filer.sync: pipelined subscription with adaptive batching for faster catch-up (#8791)
* filer.sync: pipelined subscription with adaptive batching for faster catch-up The SubscribeMetadata pipeline was fully serial: reading a log entry from a volume server, unmarshaling, filtering, and calling stream.Send() all happened one-at-a-time. stream.Send() blocked the entire pipeline until the client acknowledged each event, limiting throughput to ~80 events/sec regardless of the -concurrency setting. Three server-side optimizations that stack: 1. Pipelined sender: decouple stream.Send() from the read loop via a buffered channel (1024 messages). A dedicated goroutine handles gRPC delivery while the reader continues processing the next events. 2. Adaptive batching: when event timestamps are >2min behind wall clock (backlog catch-up), drain multiple events from the channel and pack them into a single stream.Send() using a new `repeated events` field on SubscribeMetadataResponse. When events are recent (real-time), send one-by-one for low latency. Old clients ignore the new field (backward compatible). 3. Persisted log readahead: run the OrderedLogVisitor in a background goroutine so volume server I/O for the next log file overlaps with event processing and gRPC delivery. 4. Event-driven aggregated subscription: replace time.Sleep(1127ms) polling in SubscribeMetadata with notification-driven wake-up using the MetaLogBuffer subscriber mechanism, reducing real-time latency from ~1127ms to sub-millisecond. Combined, these create a 3-stage pipeline: [Volume I/O → readahead buffer] → [Filter → send buffer] → [gRPC Send] Test results (simulated backlog with 50µs gRPC latency per Send): direct (old): 2100 events 2100 sends 168ms 12,512 events/sec pipelined+batched: 2100 events 14 sends 40ms 52,856 events/sec Speedup: 4.2x single-stream throughput Ref: #8771 * filer.sync: require client opt-in for batch event delivery Add ClientSupportsBatching field to SubscribeMetadataRequest. The server only packs events into the Events batch field when the client explicitly sets this flag to true. Old clients (Java SDK, third-party) that don't set the flag get one-event-per-Send, preserving backward compatibility. All Go callers (FollowMetadata, MetaAggregator) set the flag to true since their recv loops already unpack batched events. * filer.sync: clear batch Events field after Send to release references Prevents the envelope message from holding references to the rest of the batch after gRPC serialization, allowing the GC to collect them sooner. * filer.sync: fix Send deadlock, add error propagation test, event-driven local subscribe - pipelinedSender.Send: add case <-s.done to unblock when sender goroutine exits (fixes deadlock when errCh was already consumed by a prior Send). - pipelinedSender.reportErr: remove for-range drain on sendCh that could block indefinitely. Send() now detects exit via s.done instead. - SubscribeLocalMetadata: replace remaining time.Sleep(1127ms) in the gap-detected-no-memory-data path with event-driven listenersCond.Wait(), consistent with the rest of the subscription paths. - Add TestPipelinedSenderErrorPropagation: verifies error surfaces via Send and Close when the underlying stream fails. - Replace goto with labeled break in test simulatePipeline. * filer.sync: check error returns in test code - direct_send: check slowStream.Send error return - pipelined_batched_send: check sender.Close error return - simulatePipeline: return error from sender.Close, propagate to callers --------- Co-authored-by: Copilot <copilot@github.com>
This commit is contained in:
@@ -23,6 +23,133 @@ const (
|
||||
MaxUnsyncedEvents = 1e3
|
||||
)
|
||||
|
||||
// metadataStreamSender is satisfied by both gRPC stream types and pipelinedSender.
|
||||
type metadataStreamSender interface {
|
||||
Send(*filer_pb.SubscribeMetadataResponse) error
|
||||
}
|
||||
|
||||
const (
|
||||
// batchBehindThreshold: when an event's timestamp is older than this
|
||||
// relative to wall clock, the sender switches to batch mode for throughput.
|
||||
// When events are closer to current time, they are sent one-by-one for
|
||||
// low latency.
|
||||
batchBehindThreshold = 2 * time.Minute
|
||||
maxBatchSize = 256
|
||||
)
|
||||
|
||||
// pipelinedSender decouples event reading from gRPC delivery by buffering
|
||||
// messages in a channel. A dedicated goroutine handles stream.Send(), allowing
|
||||
// the reader to continue reading ahead without waiting for the client to
|
||||
// acknowledge each event.
|
||||
//
|
||||
// When the client declares support for batching AND events are far behind
|
||||
// current time (backlog catch-up), multiple events are packed into a single
|
||||
// stream.Send() using the Events field. Otherwise events are sent one-by-one.
|
||||
type pipelinedSender struct {
|
||||
sendCh chan *filer_pb.SubscribeMetadataResponse
|
||||
errCh chan error
|
||||
done chan struct{}
|
||||
canBatch bool // true only if client set ClientSupportsBatching
|
||||
}
|
||||
|
||||
func newPipelinedSender(stream metadataStreamSender, bufSize int, clientSupportsBatching bool) *pipelinedSender {
|
||||
s := &pipelinedSender{
|
||||
sendCh: make(chan *filer_pb.SubscribeMetadataResponse, bufSize),
|
||||
errCh: make(chan error, 1),
|
||||
done: make(chan struct{}),
|
||||
canBatch: clientSupportsBatching,
|
||||
}
|
||||
go s.sendLoop(stream)
|
||||
return s
|
||||
}
|
||||
|
||||
func (s *pipelinedSender) sendLoop(stream metadataStreamSender) {
|
||||
defer close(s.done)
|
||||
for msg := range s.sendCh {
|
||||
shouldBatch := s.canBatch && time.Now().UnixNano()-msg.TsNs > int64(batchBehindThreshold)
|
||||
|
||||
if !shouldBatch {
|
||||
// Real-time: send immediately for low latency
|
||||
if err := stream.Send(msg); err != nil {
|
||||
s.reportErr(err)
|
||||
return
|
||||
}
|
||||
continue
|
||||
}
|
||||
|
||||
// Backlog: batch multiple events into one Send for throughput.
|
||||
// The first event goes in the top-level fields; additional events
|
||||
// go in the Events slice. Old clients ignore the Events field.
|
||||
batch := make([]*filer_pb.SubscribeMetadataResponse, 0, maxBatchSize)
|
||||
batch = append(batch, msg)
|
||||
drain:
|
||||
for len(batch) < maxBatchSize {
|
||||
select {
|
||||
case next, ok := <-s.sendCh:
|
||||
if !ok {
|
||||
break drain
|
||||
}
|
||||
batch = append(batch, next)
|
||||
default:
|
||||
break drain
|
||||
}
|
||||
}
|
||||
|
||||
var toSend *filer_pb.SubscribeMetadataResponse
|
||||
if len(batch) == 1 {
|
||||
toSend = batch[0]
|
||||
} else {
|
||||
// Pack batch: first event is the envelope, rest go in Events
|
||||
toSend = batch[0]
|
||||
toSend.Events = batch[1:]
|
||||
}
|
||||
if err := stream.Send(toSend); err != nil {
|
||||
s.reportErr(err)
|
||||
return
|
||||
}
|
||||
if toSend.Events != nil {
|
||||
toSend.Events = nil
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func (s *pipelinedSender) reportErr(err error) {
|
||||
select {
|
||||
case s.errCh <- err:
|
||||
default:
|
||||
}
|
||||
// Don't drain sendCh here — Send() detects the exit via <-s.done
|
||||
// and the deferred close(s.done) in sendLoop will fire after this returns.
|
||||
}
|
||||
|
||||
func (s *pipelinedSender) Send(msg *filer_pb.SubscribeMetadataResponse) error {
|
||||
select {
|
||||
case s.sendCh <- msg:
|
||||
return nil
|
||||
case err := <-s.errCh:
|
||||
return err
|
||||
case <-s.done:
|
||||
// Sender goroutine exited (stream error or shutdown).
|
||||
select {
|
||||
case err := <-s.errCh:
|
||||
return err
|
||||
default:
|
||||
return fmt.Errorf("pipelined sender closed")
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func (s *pipelinedSender) Close() error {
|
||||
close(s.sendCh)
|
||||
<-s.done
|
||||
select {
|
||||
case err := <-s.errCh:
|
||||
return err
|
||||
default:
|
||||
return nil
|
||||
}
|
||||
}
|
||||
|
||||
func (fs *FilerServer) SubscribeMetadata(req *filer_pb.SubscribeMetadataRequest, stream filer_pb.SeaweedFiler_SubscribeMetadataServer) error {
|
||||
if fs.filer.MetaAggregator == nil || !fs.filer.MetaAggregator.HasRemotePeers() {
|
||||
return fs.SubscribeLocalMetadata(req, stream)
|
||||
@@ -47,7 +174,16 @@ func (fs *FilerServer) SubscribeMetadata(req *filer_pb.SubscribeMetadataRequest,
|
||||
lastReadTime := log_buffer.NewMessagePosition(req.SinceNs, -2)
|
||||
glog.V(0).Infof(" %v starts to subscribe %s from %+v", clientName, req.PathPrefix, lastReadTime)
|
||||
|
||||
eachEventNotificationFn := fs.eachEventNotificationFn(req, stream, clientName)
|
||||
sender := newPipelinedSender(stream, 1024, req.ClientSupportsBatching)
|
||||
defer sender.Close()
|
||||
|
||||
// Register for instant notification when new data arrives in the aggregated log buffer.
|
||||
// Used to replace the 1127ms sleep with event-driven wake-up.
|
||||
aggNotifyName := "aggSubscribe:" + clientName
|
||||
aggNotifyChan := fs.filer.MetaAggregator.MetaLogBuffer.RegisterSubscriber(aggNotifyName)
|
||||
defer fs.filer.MetaAggregator.MetaLogBuffer.UnregisterSubscriber(aggNotifyName)
|
||||
|
||||
eachEventNotificationFn := fs.eachEventNotificationFn(req, sender, clientName)
|
||||
|
||||
eachLogEntryFn := eachLogEntryFn(eachEventNotificationFn)
|
||||
|
||||
@@ -128,7 +264,17 @@ func (fs *FilerServer) SubscribeMetadata(req *filer_pb.SubscribeMetadataRequest,
|
||||
return nil
|
||||
}
|
||||
|
||||
time.Sleep(1127 * time.Millisecond)
|
||||
// Wait for new data (event-driven instead of 1127ms polling).
|
||||
// Drain any stale notification first to avoid a spurious wake-up.
|
||||
select {
|
||||
case <-aggNotifyChan:
|
||||
default:
|
||||
}
|
||||
select {
|
||||
case <-aggNotifyChan:
|
||||
case <-ctx.Done():
|
||||
return nil
|
||||
}
|
||||
}
|
||||
|
||||
return readInMemoryLogErr
|
||||
@@ -158,7 +304,10 @@ func (fs *FilerServer) SubscribeLocalMetadata(req *filer_pb.SubscribeMetadataReq
|
||||
lastReadTime := log_buffer.NewMessagePosition(req.SinceNs, -2)
|
||||
glog.V(0).Infof(" + %v local subscribe %s from %+v clientId:%d", clientName, req.PathPrefix, lastReadTime, req.ClientId)
|
||||
|
||||
eachEventNotificationFn := fs.eachEventNotificationFn(req, stream, clientName)
|
||||
sender := newPipelinedSender(stream, 1024, req.ClientSupportsBatching)
|
||||
defer sender.Close()
|
||||
|
||||
eachEventNotificationFn := fs.eachEventNotificationFn(req, sender, clientName)
|
||||
|
||||
eachLogEntryFn := eachLogEntryFn(eachEventNotificationFn)
|
||||
|
||||
@@ -210,8 +359,12 @@ func (fs *FilerServer) SubscribeLocalMetadata(req *filer_pb.SubscribeMetadataReq
|
||||
lastReadTime = log_buffer.NewMessagePosition(earliestTime.UnixNano(), -2)
|
||||
readInMemoryLogErr = nil // Clear the error since we're skipping forward
|
||||
} else {
|
||||
// No memory data yet, just wait
|
||||
time.Sleep(1127 * time.Millisecond)
|
||||
// No memory data yet, wait for new data (event-driven)
|
||||
fs.listenersLock.Lock()
|
||||
atomic.AddInt64(&fs.listenersWaits, 1)
|
||||
fs.listenersCond.Wait()
|
||||
atomic.AddInt64(&fs.listenersWaits, -1)
|
||||
fs.listenersLock.Unlock()
|
||||
continue
|
||||
}
|
||||
} else {
|
||||
@@ -294,13 +447,13 @@ func eachLogEntryFn(eachEventNotificationFn func(dirPath string, eventNotificati
|
||||
}
|
||||
}
|
||||
|
||||
func (fs *FilerServer) eachEventNotificationFn(req *filer_pb.SubscribeMetadataRequest, stream filer_pb.SeaweedFiler_SubscribeMetadataServer, clientName string) func(dirPath string, eventNotification *filer_pb.EventNotification, tsNs int64) error {
|
||||
func (fs *FilerServer) eachEventNotificationFn(req *filer_pb.SubscribeMetadataRequest, sender metadataStreamSender, clientName string) func(dirPath string, eventNotification *filer_pb.EventNotification, tsNs int64) error {
|
||||
filtered := 0
|
||||
|
||||
return func(dirPath string, eventNotification *filer_pb.EventNotification, tsNs int64) error {
|
||||
defer func() {
|
||||
if filtered > MaxUnsyncedEvents {
|
||||
if err := stream.Send(&filer_pb.SubscribeMetadataResponse{
|
||||
if err := sender.Send(&filer_pb.SubscribeMetadataResponse{
|
||||
EventNotification: &filer_pb.EventNotification{},
|
||||
TsNs: tsNs,
|
||||
}); err == nil {
|
||||
@@ -364,7 +517,7 @@ func (fs *FilerServer) eachEventNotificationFn(req *filer_pb.SubscribeMetadataRe
|
||||
TsNs: tsNs,
|
||||
}
|
||||
// println("sending", dirPath, entryName)
|
||||
if err := stream.Send(message); err != nil {
|
||||
if err := sender.Send(message); err != nil {
|
||||
glog.V(0).Infof("=> client %v: %+v", clientName, err)
|
||||
return err
|
||||
}
|
||||
|
||||
334
weed/server/filer_grpc_server_sub_meta_test.go
Normal file
334
weed/server/filer_grpc_server_sub_meta_test.go
Normal file
@@ -0,0 +1,334 @@
|
||||
package weed_server
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"sync"
|
||||
"sync/atomic"
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
"github.com/seaweedfs/seaweedfs/weed/pb/filer_pb"
|
||||
)
|
||||
|
||||
// slowStream simulates a gRPC stream with configurable per-Send latency.
|
||||
// It counts individual events including those packed inside batches.
|
||||
type slowStream struct {
|
||||
sendDelay time.Duration
|
||||
sends int64 // number of stream.Send() calls
|
||||
eventsSent int64 // total events (1 + len(Events) per Send)
|
||||
}
|
||||
|
||||
func (s *slowStream) Send(msg *filer_pb.SubscribeMetadataResponse) error {
|
||||
time.Sleep(s.sendDelay)
|
||||
atomic.AddInt64(&s.sends, 1)
|
||||
atomic.AddInt64(&s.eventsSent, 1+int64(len(msg.Events)))
|
||||
return nil
|
||||
}
|
||||
|
||||
func makeEvent(dir, name string, tsNs int64) *filer_pb.SubscribeMetadataResponse {
|
||||
return &filer_pb.SubscribeMetadataResponse{
|
||||
Directory: dir,
|
||||
TsNs: tsNs,
|
||||
EventNotification: &filer_pb.EventNotification{
|
||||
NewEntry: &filer_pb.Entry{
|
||||
Name: name,
|
||||
IsDirectory: false,
|
||||
},
|
||||
},
|
||||
}
|
||||
}
|
||||
|
||||
// makeOldEvents creates events with timestamps far in the past (triggers batch mode).
|
||||
func makeOldEvents(n int) []*filer_pb.SubscribeMetadataResponse {
|
||||
baseTs := time.Now().Add(-time.Hour).UnixNano() // 1 hour ago → well past batchBehindThreshold
|
||||
events := make([]*filer_pb.SubscribeMetadataResponse, n)
|
||||
for i := range events {
|
||||
events[i] = makeEvent("/bucket/dir", fmt.Sprintf("file%06d.txt", i), baseTs+int64(i))
|
||||
}
|
||||
return events
|
||||
}
|
||||
|
||||
// makeRecentEvents creates events with timestamps close to now (sends one-by-one).
|
||||
func makeRecentEvents(n int) []*filer_pb.SubscribeMetadataResponse {
|
||||
baseTs := time.Now().UnixNano()
|
||||
events := make([]*filer_pb.SubscribeMetadataResponse, n)
|
||||
for i := range events {
|
||||
events[i] = makeEvent("/bucket/dir", fmt.Sprintf("file%06d.txt", i), baseTs+int64(i))
|
||||
}
|
||||
return events
|
||||
}
|
||||
|
||||
// TestPipelinedSenderThroughput compares direct (blocking) stream.Send with
|
||||
// the pipelinedSender with adaptive batching.
|
||||
//
|
||||
// Simulates realistic backlog catch-up: the reader loads one log file at a time
|
||||
// from a volume server (fileReadDelay per file), producing a burst of ~300
|
||||
// events. The sender has per-Send gRPC overhead (sendDelay).
|
||||
//
|
||||
// - Direct: serial — each event: send one-by-one between file reads
|
||||
// - Pipelined+batched: file I/O overlaps with batched sending
|
||||
func TestPipelinedSenderThroughput(t *testing.T) {
|
||||
const (
|
||||
eventsPerFile = 300 // events in one minute-log file
|
||||
numFiles = 7 // files to process
|
||||
totalEvents = eventsPerFile * numFiles // 2100
|
||||
fileReadDelay = 5 * time.Millisecond // volume server read per log file
|
||||
sendDelay = 50 * time.Microsecond // gRPC round-trip per Send()
|
||||
)
|
||||
|
||||
// Partition old events into file-sized bursts
|
||||
files := make([][]*filer_pb.SubscribeMetadataResponse, numFiles)
|
||||
baseTs := time.Now().Add(-time.Hour).UnixNano()
|
||||
for f := 0; f < numFiles; f++ {
|
||||
files[f] = make([]*filer_pb.SubscribeMetadataResponse, eventsPerFile)
|
||||
for i := 0; i < eventsPerFile; i++ {
|
||||
idx := f*eventsPerFile + i
|
||||
files[f][i] = makeEvent("/bucket/dir", fmt.Sprintf("file%06d.txt", idx), baseTs+int64(idx))
|
||||
}
|
||||
}
|
||||
|
||||
// --- Direct (old behavior): read file, send events one-by-one, repeat ---
|
||||
var directRate float64
|
||||
t.Run("direct_send", func(t *testing.T) {
|
||||
stream := &slowStream{sendDelay: sendDelay}
|
||||
|
||||
start := time.Now()
|
||||
for _, file := range files {
|
||||
time.Sleep(fileReadDelay) // read log file from volume server
|
||||
for _, ev := range file {
|
||||
if err := stream.Send(ev); err != nil {
|
||||
t.Fatalf("send error: %v", err)
|
||||
}
|
||||
}
|
||||
}
|
||||
elapsed := time.Since(start)
|
||||
|
||||
directRate = float64(stream.eventsSent) / elapsed.Seconds()
|
||||
t.Logf("direct: %d events %4d sends %v %6.0f events/sec",
|
||||
stream.eventsSent, stream.sends, elapsed.Round(time.Millisecond), directRate)
|
||||
})
|
||||
|
||||
// --- Pipelined + batched (new behavior): file reads overlap with batched sends ---
|
||||
var batchedRate float64
|
||||
t.Run("pipelined_batched_send", func(t *testing.T) {
|
||||
stream := &slowStream{sendDelay: sendDelay}
|
||||
sender := newPipelinedSender(stream, 1024, true)
|
||||
|
||||
start := time.Now()
|
||||
for _, file := range files {
|
||||
time.Sleep(fileReadDelay) // read log file from volume server
|
||||
for _, ev := range file {
|
||||
if err := sender.Send(ev); err != nil {
|
||||
t.Fatalf("send error: %v", err)
|
||||
}
|
||||
}
|
||||
}
|
||||
if err := sender.Close(); err != nil {
|
||||
t.Fatalf("close error: %v", err)
|
||||
}
|
||||
elapsed := time.Since(start)
|
||||
|
||||
batchedRate = float64(stream.eventsSent) / elapsed.Seconds()
|
||||
t.Logf("pipelined+batch: %d events %4d sends %v %6.0f events/sec",
|
||||
stream.eventsSent, stream.sends, elapsed.Round(time.Millisecond), batchedRate)
|
||||
})
|
||||
|
||||
if directRate > 0 {
|
||||
t.Logf("Speedup: %.1fx (pipelined+batched vs direct)", batchedRate/directRate)
|
||||
}
|
||||
}
|
||||
|
||||
// TestBatchingAdaptive verifies the adaptive behavior: old events are batched,
|
||||
// recent events are sent one-by-one.
|
||||
func TestBatchingAdaptive(t *testing.T) {
|
||||
const numEvents = 500
|
||||
|
||||
t.Run("old_events_are_batched", func(t *testing.T) {
|
||||
stream := &slowStream{sendDelay: 10 * time.Microsecond}
|
||||
sender := newPipelinedSender(stream, 1024, true)
|
||||
|
||||
// Push all events at once (no read delay) so the sender can batch aggressively
|
||||
for _, ev := range makeOldEvents(numEvents) {
|
||||
sender.Send(ev)
|
||||
}
|
||||
sender.Close()
|
||||
|
||||
t.Logf("old events: %d events in %d sends (avg batch size: %.1f)",
|
||||
stream.eventsSent, stream.sends, float64(stream.eventsSent)/float64(stream.sends))
|
||||
|
||||
if stream.sends >= int64(numEvents) {
|
||||
t.Errorf("expected batching to reduce sends below %d, got %d", numEvents, stream.sends)
|
||||
}
|
||||
})
|
||||
|
||||
t.Run("recent_events_sent_individually", func(t *testing.T) {
|
||||
stream := &slowStream{sendDelay: 10 * time.Microsecond}
|
||||
sender := newPipelinedSender(stream, 1024, true)
|
||||
|
||||
for _, ev := range makeRecentEvents(numEvents) {
|
||||
sender.Send(ev)
|
||||
}
|
||||
sender.Close()
|
||||
|
||||
t.Logf("recent events: %d events in %d sends (avg batch size: %.1f)",
|
||||
stream.eventsSent, stream.sends, float64(stream.eventsSent)/float64(stream.sends))
|
||||
|
||||
if stream.sends != int64(numEvents) {
|
||||
t.Errorf("expected 1:1 sends for recent events, got %d sends for %d events", stream.sends, numEvents)
|
||||
}
|
||||
})
|
||||
}
|
||||
|
||||
// errorStreamImpl is a metadataStreamSender that returns an error after N sends.
|
||||
type errorStreamImpl struct {
|
||||
failAfter int
|
||||
err error
|
||||
count int64
|
||||
}
|
||||
|
||||
func (s *errorStreamImpl) Send(msg *filer_pb.SubscribeMetadataResponse) error {
|
||||
n := atomic.AddInt64(&s.count, 1)
|
||||
if int(n) > s.failAfter {
|
||||
return s.err
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
// TestPipelinedSenderErrorPropagation verifies that when stream.Send fails,
|
||||
// the error propagates to pipelinedSender.Send callers and Close.
|
||||
func TestPipelinedSenderErrorPropagation(t *testing.T) {
|
||||
sendErr := fmt.Errorf("connection reset")
|
||||
|
||||
t.Run("send_returns_error", func(t *testing.T) {
|
||||
// Stream fails after 5 successful sends
|
||||
stream := &errorStreamImpl{failAfter: 5, err: sendErr}
|
||||
sender := newPipelinedSender(stream, 4, true)
|
||||
|
||||
var lastErr error
|
||||
for i := 0; i < 100; i++ {
|
||||
ev := makeOldEvents(1)[0]
|
||||
if err := sender.Send(ev); err != nil {
|
||||
lastErr = err
|
||||
break
|
||||
}
|
||||
}
|
||||
|
||||
if lastErr == nil {
|
||||
t.Fatal("expected Send to return an error after stream failure")
|
||||
}
|
||||
t.Logf("Send returned error after stream failure: %v", lastErr)
|
||||
})
|
||||
|
||||
t.Run("close_returns_error_if_not_consumed", func(t *testing.T) {
|
||||
// Stream fails on the very first send — error surfaces via Close
|
||||
// since Send may have already returned before the sender goroutine
|
||||
// processes the message.
|
||||
stream := &errorStreamImpl{failAfter: 0, err: sendErr}
|
||||
sender := newPipelinedSender(stream, 1024, true)
|
||||
|
||||
ev := makeOldEvents(1)[0]
|
||||
sender.Send(ev)
|
||||
|
||||
closeErr := sender.Close()
|
||||
if closeErr == nil {
|
||||
t.Log("Close returned nil (error was consumed by Send)")
|
||||
} else {
|
||||
t.Logf("Close returned error: %v", closeErr)
|
||||
}
|
||||
})
|
||||
}
|
||||
|
||||
// TestPipelinedSingleVsParallelStreams shows 1 pipelined+batched stream vs
|
||||
// N parallel pipelined+batched streams, using the realistic burst-read pattern.
|
||||
func TestPipelinedSingleVsParallelStreams(t *testing.T) {
|
||||
const (
|
||||
numDirs = 10
|
||||
filesPerDir = 7 // log files per directory
|
||||
eventsPerFile = 300 // events per log file
|
||||
totalEvents = numDirs * filesPerDir * eventsPerFile // 21000
|
||||
fileReadDelay = 5 * time.Millisecond
|
||||
sendDelay = 50 * time.Microsecond
|
||||
)
|
||||
|
||||
// Generate partitioned OLD events grouped into file-sized bursts
|
||||
baseTs := time.Now().Add(-time.Hour).UnixNano()
|
||||
type logFile []*filer_pb.SubscribeMetadataResponse
|
||||
// partitions[dir][file][event]
|
||||
partitions := make([][]logFile, numDirs)
|
||||
var allFiles []logFile
|
||||
idx := 0
|
||||
for d := 0; d < numDirs; d++ {
|
||||
dir := fmt.Sprintf("/bucket/dir%03d", d)
|
||||
for f := 0; f < filesPerDir; f++ {
|
||||
file := make(logFile, eventsPerFile)
|
||||
for i := 0; i < eventsPerFile; i++ {
|
||||
file[i] = makeEvent(dir, fmt.Sprintf("file%06d.txt", idx), baseTs+int64(idx))
|
||||
idx++
|
||||
}
|
||||
partitions[d] = append(partitions[d], file)
|
||||
allFiles = append(allFiles, file)
|
||||
}
|
||||
}
|
||||
|
||||
// simulatePipeline: read files with I/O delay, push events, send via pipelinedSender
|
||||
simulatePipeline := func(files []logFile) (eventsSent, sends int64, elapsed time.Duration, err error) {
|
||||
stream := &slowStream{sendDelay: sendDelay}
|
||||
sender := newPipelinedSender(stream, 1024, true)
|
||||
|
||||
start := time.Now()
|
||||
outer:
|
||||
for _, file := range files {
|
||||
time.Sleep(fileReadDelay) // volume server read
|
||||
for _, ev := range file {
|
||||
if err = sender.Send(ev); err != nil {
|
||||
break outer
|
||||
}
|
||||
}
|
||||
}
|
||||
if closeErr := sender.Close(); closeErr != nil && err == nil {
|
||||
err = closeErr
|
||||
}
|
||||
elapsed = time.Since(start)
|
||||
eventsSent = atomic.LoadInt64(&stream.eventsSent)
|
||||
sends = atomic.LoadInt64(&stream.sends)
|
||||
return
|
||||
}
|
||||
|
||||
var singleRate float64
|
||||
t.Run("1_pipelined_stream", func(t *testing.T) {
|
||||
eventsSent, sends, elapsed, err := simulatePipeline(allFiles)
|
||||
if err != nil {
|
||||
t.Fatalf("pipeline error: %v", err)
|
||||
}
|
||||
singleRate = float64(eventsSent) / elapsed.Seconds()
|
||||
t.Logf("1 stream: %5d events %4d sends %v %7.0f events/sec",
|
||||
eventsSent, sends, elapsed.Round(time.Millisecond), singleRate)
|
||||
})
|
||||
|
||||
var parallelRate float64
|
||||
t.Run("10_pipelined_streams", func(t *testing.T) {
|
||||
var totalEventsSent, totalSends int64
|
||||
var wg sync.WaitGroup
|
||||
|
||||
start := time.Now()
|
||||
for d := 0; d < numDirs; d++ {
|
||||
wg.Add(1)
|
||||
go func(files []logFile) {
|
||||
defer wg.Done()
|
||||
eventsSent, sends, _, _ := simulatePipeline(files)
|
||||
atomic.AddInt64(&totalEventsSent, eventsSent)
|
||||
atomic.AddInt64(&totalSends, sends)
|
||||
}(partitions[d])
|
||||
}
|
||||
wg.Wait()
|
||||
elapsed := time.Since(start)
|
||||
|
||||
parallelRate = float64(totalEventsSent) / elapsed.Seconds()
|
||||
t.Logf("%d streams: %5d events %4d sends %v %7.0f events/sec",
|
||||
numDirs, totalEventsSent, totalSends, elapsed.Round(time.Millisecond), parallelRate)
|
||||
})
|
||||
|
||||
if singleRate > 0 && parallelRate > 0 {
|
||||
t.Logf("Speedup: %.1fx (%d parallel pipelined streams vs 1)", parallelRate/singleRate, numDirs)
|
||||
}
|
||||
}
|
||||
Reference in New Issue
Block a user