filer.sync: send log file chunk fids to clients for direct volume server reads (#8792)
* filer.sync: send log file chunk fids to clients for direct volume server reads Instead of the server reading persisted log files from volume servers, parsing entries, and streaming them over gRPC (serial bottleneck), clients that opt in via client_supports_metadata_chunks receive log file chunk references (fids) and read directly from volume servers in parallel. New proto messages: - LogFileChunkRef: chunk fids + timestamp + filer ID for one log file - SubscribeMetadataRequest.client_supports_metadata_chunks: client opt-in - SubscribeMetadataResponse.log_file_refs: server sends refs during backlog Server changes: - CollectLogFileRefs: lists log files and returns chunk refs without any volume server I/O (metadata-only operation) - SubscribeMetadata/SubscribeLocalMetadata: when client opts in, sends refs during persisted log phase, then falls back to normal streaming for in-memory events Client changes: - ReadLogFileRefs: reads log files from volume servers, parses entries, filters by path prefix, invokes processEventFn - MetadataFollowOption.LogFileReaderFn: factory for chunk readers, enables metadata chunks when non-nil - Both filer_pb_tail.go and meta_aggregator.go recv loops accumulate refs then process them at the disk→memory transition Backward compatible: old clients don't set the flag, get existing behavior. Ref: #8771 * filer.sync: merge entries across filers in timestamp order on client side ReadLogFileRefs now groups refs by filer ID and merges entries from multiple filers using a min-heap priority queue — the same algorithm the server uses in OrderedLogVisitor + LogEntryItemPriorityQueue. This ensures events are processed in correct timestamp order even when log files from different filers have interleaved timestamps. Single-filer case takes the fast path (no heap allocation). * filer.sync: integration tests for direct-read metadata chunks Three test categories: 1. Merge correctness (TestReadLogFileRefsMergeOrder): Verifies entries from 3 filers are delivered in strict timestamp order, matching the server-side OrderedLogVisitor guarantee. 2. Path filtering (TestReadLogFileRefsPathFilter): Verifies client-side path prefix filtering works correctly. 3. Throughput comparison (TestDirectReadVsServerSideThroughput): 3 filers × 7 files × 300 events = 6300 events, 2ms per file read: server-side: 6300 events 218ms 28,873 events/sec direct-read: 6300 events 51ms 123,566 events/sec (4.3x) parallel: 6300 events 17ms 378,628 events/sec (13.1x) Direct-read eliminates gRPC send overhead per event (4.3x). Parallel per-filer reading eliminates serial file I/O (13.1x). * filer.sync: parallel per-filer reads with prefetching in ReadLogFileRefs ReadLogFileRefs now has two levels of I/O overlap: 1. Cross-filer parallelism: one goroutine per filer reads its files concurrently. Entries feed into per-filer channels, merged by the main goroutine via min-heap (same ordering guarantee as the server's OrderedLogVisitor). 2. Within-filer prefetching: while the current file's entries are being consumed by the merge heap, the next file is already being read from the volume server in a background goroutine. Single-filer fast path avoids the heap and channels. Test results (3 filers × 7 files × 300 events, 2ms per file read): server-side sequential: 6300 events 212ms 29,760 events/sec parallel + prefetch: 6300 events 36ms 177,443 events/sec Speedup: 6.0x * filer.sync: address all review comments on metadata chunks PR Critical fixes: - sendLogFileRefs: bypass pipelinedSender, send directly on gRPC stream. Ref messages have TsNs=0 and were being incorrectly batched into the Events field by the adaptive batching logic, corrupting ref delivery. - readLogFileEntries: use io.ReadFull instead of reader.Read to prevent partial reads from corrupting size values or protobuf data. - Error handling: only skip chunk-not-found errors (matching server-side isChunkNotFoundError). Other I/O or decode failures are propagated so the follower can retry. High-priority fixes: - CollectLogFileRefs: remove incorrect +24h padding from stopTime. The extra day caused unnecessary log file refs to be collected. - Path filtering: ReadLogFileRefs now accepts PathFilter struct with PathPrefix, AdditionalPathPrefixes, and DirectoriesToWatch. Uses util.Join for path construction (avoids "//foo" on root). Excludes /.system/log/ internal entries. Matches server-side eachEventNotificationFn filtering logic. Medium-priority fixes: - CollectLogFileRefs: accept context.Context, propagate to ListDirectoryEntries calls for cancellation support. - NewChunkStreamReaderFromLookup: accept context.Context, propagate to doNewChunkStreamReader. Test fixes: - Check error returns from ReadLogFileRefs in all test call sites. --------- Co-authored-by: Copilot <copilot@github.com>
This commit is contained in:
@@ -1,6 +1,7 @@
|
||||
package weed_server
|
||||
|
||||
import (
|
||||
"context"
|
||||
"errors"
|
||||
"fmt"
|
||||
"strings"
|
||||
@@ -196,7 +197,11 @@ func (fs *FilerServer) SubscribeMetadata(req *filer_pb.SubscribeMetadataRequest,
|
||||
|
||||
glog.V(4).Infof("read on disk %v aggregated subscribe %s from %+v", clientName, req.PathPrefix, lastReadTime)
|
||||
|
||||
processedTsNs, isDone, readPersistedLogErr = fs.filer.ReadPersistedLogBuffer(lastReadTime, req.UntilNs, eachLogEntryFn)
|
||||
if req.ClientSupportsMetadataChunks {
|
||||
processedTsNs, isDone, readPersistedLogErr = fs.sendLogFileRefs(ctx, stream, lastReadTime, req.UntilNs)
|
||||
} else {
|
||||
processedTsNs, isDone, readPersistedLogErr = fs.filer.ReadPersistedLogBuffer(lastReadTime, req.UntilNs, eachLogEntryFn)
|
||||
}
|
||||
if readPersistedLogErr != nil {
|
||||
return fmt.Errorf("reading from persisted logs: %w", readPersistedLogErr)
|
||||
}
|
||||
@@ -331,7 +336,11 @@ func (fs *FilerServer) SubscribeLocalMetadata(req *filer_pb.SubscribeMetadataReq
|
||||
// Record the position we are about to read from
|
||||
lastDiskReadTsNs = currentReadTsNs
|
||||
glog.V(4).Infof("read on disk %v local subscribe %s from %+v (lastFlushed: %v)", clientName, req.PathPrefix, lastReadTime, time.Unix(0, currentFlushTsNs))
|
||||
processedTsNs, isDone, readPersistedLogErr = fs.filer.ReadPersistedLogBuffer(lastReadTime, req.UntilNs, eachLogEntryFn)
|
||||
if req.ClientSupportsMetadataChunks {
|
||||
processedTsNs, isDone, readPersistedLogErr = fs.sendLogFileRefs(ctx, stream, lastReadTime, req.UntilNs)
|
||||
} else {
|
||||
processedTsNs, isDone, readPersistedLogErr = fs.filer.ReadPersistedLogBuffer(lastReadTime, req.UntilNs, eachLogEntryFn)
|
||||
}
|
||||
if readPersistedLogErr != nil {
|
||||
glog.V(0).Infof("read on disk %v local subscribe %s from %+v: %v", clientName, req.PathPrefix, lastReadTime, readPersistedLogErr)
|
||||
return fmt.Errorf("reading from persisted logs: %w", readPersistedLogErr)
|
||||
@@ -447,6 +456,35 @@ func eachLogEntryFn(eachEventNotificationFn func(dirPath string, eventNotificati
|
||||
}
|
||||
}
|
||||
|
||||
// sendLogFileRefs collects persisted log file chunk references and sends them
|
||||
// to the client so it can read the data directly from volume servers.
|
||||
// This does zero volume server I/O — it only lists filer store directory entries.
|
||||
// Sends directly on the gRPC stream (bypasses pipelinedSender) because ref
|
||||
// messages have TsNs=0 and must not be batched into Events by the sender.
|
||||
func (fs *FilerServer) sendLogFileRefs(ctx context.Context, stream metadataStreamSender, startPosition log_buffer.MessagePosition, stopTsNs int64) (lastTsNs int64, isDone bool, err error) {
|
||||
refs, lastTsNs, err := fs.filer.CollectLogFileRefs(ctx, startPosition, stopTsNs)
|
||||
if err != nil {
|
||||
return 0, false, err
|
||||
}
|
||||
if len(refs) == 0 {
|
||||
return 0, false, nil
|
||||
}
|
||||
|
||||
const maxRefsPerMessage = 64
|
||||
for i := 0; i < len(refs); i += maxRefsPerMessage {
|
||||
end := i + maxRefsPerMessage
|
||||
if end > len(refs) {
|
||||
end = len(refs)
|
||||
}
|
||||
if err := stream.Send(&filer_pb.SubscribeMetadataResponse{
|
||||
LogFileRefs: refs[i:end],
|
||||
}); err != nil {
|
||||
return lastTsNs, false, err
|
||||
}
|
||||
}
|
||||
return lastTsNs, false, nil
|
||||
}
|
||||
|
||||
func (fs *FilerServer) eachEventNotificationFn(req *filer_pb.SubscribeMetadataRequest, sender metadataStreamSender, clientName string) func(dirPath string, eventNotification *filer_pb.EventNotification, tsNs int64) error {
|
||||
filtered := 0
|
||||
|
||||
|
||||
Reference in New Issue
Block a user