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:
Chris Lu
2026-03-26 23:55:42 -07:00
committed by GitHub
parent 8c8d21d7e2
commit d97660d0cd
9 changed files with 1039 additions and 147 deletions

View File

@@ -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
}