You can not select more than 25 topics
Topics must start with a letter or number, can include dashes ('-') and can be up to 35 characters long.
329 lines
10 KiB
329 lines
10 KiB
package log_buffer
|
|
|
|
import (
|
|
"context"
|
|
"sync"
|
|
"testing"
|
|
"time"
|
|
|
|
"github.com/seaweedfs/seaweedfs/weed/pb/filer_pb"
|
|
"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
|
|
)
|
|
|
|
// TestLoopProcessLogDataWithOffset_ClientDisconnect tests that the loop exits
|
|
// when the client disconnects (waitForDataFn returns false)
|
|
func TestLoopProcessLogDataWithOffset_ClientDisconnect(t *testing.T) {
|
|
flushFn := func(logBuffer *LogBuffer, startTime, stopTime time.Time, buf []byte, minOffset, maxOffset int64) {}
|
|
logBuffer := NewLogBuffer("test", 1*time.Minute, flushFn, nil, nil)
|
|
defer logBuffer.ShutdownLogBuffer()
|
|
|
|
// Simulate client disconnect after 100ms
|
|
ctx, cancel := context.WithTimeout(context.Background(), 100*time.Millisecond)
|
|
defer cancel()
|
|
|
|
waitForDataFn := func() bool {
|
|
select {
|
|
case <-ctx.Done():
|
|
return false // Client disconnected
|
|
default:
|
|
return true
|
|
}
|
|
}
|
|
|
|
eachLogEntryFn := func(logEntry *filer_pb.LogEntry, offset int64) (bool, error) {
|
|
return true, nil
|
|
}
|
|
|
|
startPosition := NewMessagePositionFromOffset(0)
|
|
startTime := time.Now()
|
|
|
|
// This should exit within 200ms (100ms timeout + some buffer)
|
|
_, isDone, _ := logBuffer.LoopProcessLogDataWithOffset("test-client", startPosition, 0, waitForDataFn, eachLogEntryFn)
|
|
|
|
elapsed := time.Since(startTime)
|
|
|
|
if !isDone {
|
|
t.Errorf("Expected isDone=true when client disconnects, got false")
|
|
}
|
|
|
|
if elapsed > 500*time.Millisecond {
|
|
t.Errorf("Loop took too long to exit: %v (expected < 500ms)", elapsed)
|
|
}
|
|
|
|
t.Logf("Loop exited cleanly in %v after client disconnect", elapsed)
|
|
}
|
|
|
|
// TestLoopProcessLogDataWithOffset_EmptyBuffer tests that the loop doesn't
|
|
// busy-wait when the buffer is empty
|
|
func TestLoopProcessLogDataWithOffset_EmptyBuffer(t *testing.T) {
|
|
flushFn := func(logBuffer *LogBuffer, startTime, stopTime time.Time, buf []byte, minOffset, maxOffset int64) {}
|
|
logBuffer := NewLogBuffer("test", 1*time.Minute, flushFn, nil, nil)
|
|
defer logBuffer.ShutdownLogBuffer()
|
|
|
|
callCount := 0
|
|
maxCalls := 10
|
|
mu := sync.Mutex{}
|
|
|
|
waitForDataFn := func() bool {
|
|
mu.Lock()
|
|
defer mu.Unlock()
|
|
callCount++
|
|
// Disconnect after maxCalls to prevent infinite loop
|
|
return callCount < maxCalls
|
|
}
|
|
|
|
eachLogEntryFn := func(logEntry *filer_pb.LogEntry, offset int64) (bool, error) {
|
|
return true, nil
|
|
}
|
|
|
|
startPosition := NewMessagePositionFromOffset(0)
|
|
startTime := time.Now()
|
|
|
|
_, isDone, _ := logBuffer.LoopProcessLogDataWithOffset("test-client", startPosition, 0, waitForDataFn, eachLogEntryFn)
|
|
|
|
elapsed := time.Since(startTime)
|
|
|
|
if !isDone {
|
|
t.Errorf("Expected isDone=true when waitForDataFn returns false, got false")
|
|
}
|
|
|
|
// With 10ms sleep per iteration, 10 iterations should take ~100ms minimum
|
|
minExpectedTime := time.Duration(maxCalls-1) * 10 * time.Millisecond
|
|
if elapsed < minExpectedTime {
|
|
t.Errorf("Loop exited too quickly (%v), expected at least %v (suggests busy-waiting)", elapsed, minExpectedTime)
|
|
}
|
|
|
|
// But shouldn't take more than 2x expected (allows for some overhead)
|
|
maxExpectedTime := time.Duration(maxCalls) * 30 * time.Millisecond
|
|
if elapsed > maxExpectedTime {
|
|
t.Errorf("Loop took too long: %v (expected < %v)", elapsed, maxExpectedTime)
|
|
}
|
|
|
|
mu.Lock()
|
|
finalCallCount := callCount
|
|
mu.Unlock()
|
|
|
|
if finalCallCount != maxCalls {
|
|
t.Errorf("Expected exactly %d calls to waitForDataFn, got %d", maxCalls, finalCallCount)
|
|
}
|
|
|
|
t.Logf("Loop exited cleanly in %v after %d iterations (no busy-waiting detected)", elapsed, finalCallCount)
|
|
}
|
|
|
|
// TestLoopProcessLogDataWithOffset_NoDataResumeFromDisk tests that the loop
|
|
// properly handles ResumeFromDiskError without busy-waiting
|
|
func TestLoopProcessLogDataWithOffset_NoDataResumeFromDisk(t *testing.T) {
|
|
readFromDiskFn := func(startPosition MessagePosition, stopTsNs int64, eachLogEntryFn EachLogEntryFuncType) (lastReadPosition MessagePosition, isDone bool, err error) {
|
|
// No data on disk
|
|
return startPosition, false, nil
|
|
}
|
|
flushFn := func(logBuffer *LogBuffer, startTime, stopTime time.Time, buf []byte, minOffset, maxOffset int64) {}
|
|
logBuffer := NewLogBuffer("test", 1*time.Minute, flushFn, readFromDiskFn, nil)
|
|
defer logBuffer.ShutdownLogBuffer()
|
|
|
|
callCount := 0
|
|
maxCalls := 5
|
|
mu := sync.Mutex{}
|
|
|
|
waitForDataFn := func() bool {
|
|
mu.Lock()
|
|
defer mu.Unlock()
|
|
callCount++
|
|
// Disconnect after maxCalls
|
|
return callCount < maxCalls
|
|
}
|
|
|
|
eachLogEntryFn := func(logEntry *filer_pb.LogEntry, offset int64) (bool, error) {
|
|
return true, nil
|
|
}
|
|
|
|
startPosition := NewMessagePositionFromOffset(0)
|
|
startTime := time.Now()
|
|
|
|
_, isDone, _ := logBuffer.LoopProcessLogDataWithOffset("test-client", startPosition, 0, waitForDataFn, eachLogEntryFn)
|
|
|
|
elapsed := time.Since(startTime)
|
|
|
|
if !isDone {
|
|
t.Errorf("Expected isDone=true when waitForDataFn returns false, got false")
|
|
}
|
|
|
|
// Should take at least (maxCalls-1) * 10ms due to sleep in ResumeFromDiskError path
|
|
minExpectedTime := time.Duration(maxCalls-1) * 10 * time.Millisecond
|
|
if elapsed < minExpectedTime {
|
|
t.Errorf("Loop exited too quickly (%v), expected at least %v (suggests missing sleep)", elapsed, minExpectedTime)
|
|
}
|
|
|
|
t.Logf("Loop exited cleanly in %v after %d iterations (proper sleep detected)", elapsed, callCount)
|
|
}
|
|
|
|
// TestLoopProcessLogDataWithOffset_WithData tests normal operation with data
|
|
func TestLoopProcessLogDataWithOffset_WithData(t *testing.T) {
|
|
flushFn := func(logBuffer *LogBuffer, startTime, stopTime time.Time, buf []byte, minOffset, maxOffset int64) {}
|
|
logBuffer := NewLogBuffer("test", 1*time.Minute, flushFn, nil, nil)
|
|
defer logBuffer.ShutdownLogBuffer()
|
|
|
|
// Add some test data to the buffer
|
|
testMessages := []*mq_pb.DataMessage{
|
|
{Key: []byte("key1"), Value: []byte("message1"), TsNs: 1},
|
|
{Key: []byte("key2"), Value: []byte("message2"), TsNs: 2},
|
|
{Key: []byte("key3"), Value: []byte("message3"), TsNs: 3},
|
|
}
|
|
|
|
for _, msg := range testMessages {
|
|
logBuffer.AddToBuffer(msg)
|
|
}
|
|
|
|
receivedCount := 0
|
|
mu := sync.Mutex{}
|
|
|
|
// Disconnect after receiving at least 1 message to test that data processing works
|
|
waitForDataFn := func() bool {
|
|
mu.Lock()
|
|
defer mu.Unlock()
|
|
return receivedCount == 0 // Disconnect after first message
|
|
}
|
|
|
|
eachLogEntryFn := func(logEntry *filer_pb.LogEntry, offset int64) (bool, error) {
|
|
mu.Lock()
|
|
receivedCount++
|
|
mu.Unlock()
|
|
return true, nil // Continue processing
|
|
}
|
|
|
|
startPosition := NewMessagePositionFromOffset(0)
|
|
startTime := time.Now()
|
|
|
|
_, isDone, _ := logBuffer.LoopProcessLogDataWithOffset("test-client", startPosition, 0, waitForDataFn, eachLogEntryFn)
|
|
|
|
elapsed := time.Since(startTime)
|
|
|
|
if !isDone {
|
|
t.Errorf("Expected isDone=true after client disconnect, got false")
|
|
}
|
|
|
|
mu.Lock()
|
|
finalCount := receivedCount
|
|
mu.Unlock()
|
|
|
|
if finalCount < 1 {
|
|
t.Errorf("Expected to receive at least 1 message, got %d", finalCount)
|
|
}
|
|
|
|
// Should complete quickly since data is available
|
|
if elapsed > 1*time.Second {
|
|
t.Errorf("Processing took too long: %v (expected < 1s)", elapsed)
|
|
}
|
|
|
|
t.Logf("Successfully processed %d message(s) in %v", finalCount, elapsed)
|
|
}
|
|
|
|
// TestLoopProcessLogDataWithOffset_ConcurrentDisconnect tests that the loop
|
|
// handles concurrent client disconnects without panicking
|
|
func TestLoopProcessLogDataWithOffset_ConcurrentDisconnect(t *testing.T) {
|
|
flushFn := func(logBuffer *LogBuffer, startTime, stopTime time.Time, buf []byte, minOffset, maxOffset int64) {}
|
|
logBuffer := NewLogBuffer("test", 1*time.Minute, flushFn, nil, nil)
|
|
defer logBuffer.ShutdownLogBuffer()
|
|
|
|
numClients := 10
|
|
var wg sync.WaitGroup
|
|
|
|
for i := 0; i < numClients; i++ {
|
|
wg.Add(1)
|
|
go func(clientID int) {
|
|
defer wg.Done()
|
|
|
|
ctx, cancel := context.WithTimeout(context.Background(), 50*time.Millisecond)
|
|
defer cancel()
|
|
|
|
waitForDataFn := func() bool {
|
|
select {
|
|
case <-ctx.Done():
|
|
return false
|
|
default:
|
|
return true
|
|
}
|
|
}
|
|
|
|
eachLogEntryFn := func(logEntry *filer_pb.LogEntry, offset int64) (bool, error) {
|
|
return true, nil
|
|
}
|
|
|
|
startPosition := NewMessagePositionFromOffset(0)
|
|
_, _, _ = logBuffer.LoopProcessLogDataWithOffset("test-client", startPosition, 0, waitForDataFn, eachLogEntryFn)
|
|
}(i)
|
|
}
|
|
|
|
// Wait for all clients to finish with a timeout
|
|
done := make(chan struct{})
|
|
go func() {
|
|
wg.Wait()
|
|
close(done)
|
|
}()
|
|
|
|
select {
|
|
case <-done:
|
|
t.Logf("All %d concurrent clients exited cleanly", numClients)
|
|
case <-time.After(5 * time.Second):
|
|
t.Errorf("Timeout waiting for concurrent clients to exit (possible deadlock or stuck loop)")
|
|
}
|
|
}
|
|
|
|
// TestLoopProcessLogDataWithOffset_StopTime tests that the loop respects stopTsNs
|
|
func TestLoopProcessLogDataWithOffset_StopTime(t *testing.T) {
|
|
flushFn := func(logBuffer *LogBuffer, startTime, stopTime time.Time, buf []byte, minOffset, maxOffset int64) {}
|
|
logBuffer := NewLogBuffer("test", 1*time.Minute, flushFn, nil, nil)
|
|
defer logBuffer.ShutdownLogBuffer()
|
|
|
|
callCount := 0
|
|
waitForDataFn := func() bool {
|
|
callCount++
|
|
// Prevent infinite loop in case of test failure
|
|
return callCount < 10
|
|
}
|
|
|
|
eachLogEntryFn := func(logEntry *filer_pb.LogEntry, offset int64) (bool, error) {
|
|
t.Errorf("Should not process any entries when stopTsNs is in the past")
|
|
return false, nil
|
|
}
|
|
|
|
startPosition := NewMessagePositionFromOffset(0)
|
|
stopTsNs := time.Now().Add(-1 * time.Hour).UnixNano() // Stop time in the past
|
|
|
|
startTime := time.Now()
|
|
_, isDone, _ := logBuffer.LoopProcessLogDataWithOffset("test-client", startPosition, stopTsNs, waitForDataFn, eachLogEntryFn)
|
|
elapsed := time.Since(startTime)
|
|
|
|
if !isDone {
|
|
t.Errorf("Expected isDone=true when stopTsNs is in the past, got false")
|
|
}
|
|
|
|
if elapsed > 1*time.Second {
|
|
t.Errorf("Loop should exit quickly when stopTsNs is in the past, took %v", elapsed)
|
|
}
|
|
|
|
t.Logf("Loop correctly exited for past stopTsNs in %v (waitForDataFn called %d times)", elapsed, callCount)
|
|
}
|
|
|
|
// BenchmarkLoopProcessLogDataWithOffset_EmptyBuffer benchmarks the performance
|
|
// of the loop with an empty buffer to ensure no busy-waiting
|
|
func BenchmarkLoopProcessLogDataWithOffset_EmptyBuffer(b *testing.B) {
|
|
flushFn := func(logBuffer *LogBuffer, startTime, stopTime time.Time, buf []byte, minOffset, maxOffset int64) {}
|
|
logBuffer := NewLogBuffer("test", 1*time.Minute, flushFn, nil, nil)
|
|
defer logBuffer.ShutdownLogBuffer()
|
|
|
|
for i := 0; i < b.N; i++ {
|
|
callCount := 0
|
|
waitForDataFn := func() bool {
|
|
callCount++
|
|
return callCount < 3 // Exit after 3 calls
|
|
}
|
|
|
|
eachLogEntryFn := func(logEntry *filer_pb.LogEntry, offset int64) (bool, error) {
|
|
return true, nil
|
|
}
|
|
|
|
startPosition := NewMessagePositionFromOffset(0)
|
|
logBuffer.LoopProcessLogDataWithOffset("test-client", startPosition, 0, waitForDataFn, eachLogEntryFn)
|
|
}
|
|
}
|