Files
seaweedFS/weed/pb/filer_pb_direct_read.go
Chris Lu c2c58419b8 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>
2026-03-27 11:01:29 -07:00

383 lines
9.9 KiB
Go

package pb
import (
"container/heap"
"fmt"
"io"
"strings"
"sync"
"google.golang.org/protobuf/proto"
"github.com/seaweedfs/seaweedfs/weed/glog"
"github.com/seaweedfs/seaweedfs/weed/pb/filer_pb"
"github.com/seaweedfs/seaweedfs/weed/util"
)
// LogFileReaderFn creates an io.ReadCloser for a set of file chunks.
type LogFileReaderFn func(chunks []*filer_pb.FileChunk) (io.ReadCloser, error)
// PathFilter holds subscription path filtering parameters, matching the
// server-side eachEventNotificationFn filtering logic.
type PathFilter struct {
PathPrefix string
AdditionalPathPrefixes []string
DirectoriesToWatch []string
}
// ReadLogFileRefs reads log file data directly from volume servers using the
// chunk references, merges entries from multiple filers in timestamp order
// (same algorithm as the server's OrderedLogVisitor), applies path filtering,
// and invokes processEventFn for each matching event.
//
// Filers are read in parallel (one goroutine per filer). Within each filer,
// the next file is prefetched while the current file's entries are consumed.
func ReadLogFileRefs(
refs []*filer_pb.LogFileChunkRef,
newReader LogFileReaderFn,
startTsNs, stopTsNs int64,
filter PathFilter,
processEventFn ProcessMetadataFunc,
) (lastTsNs int64, err error) {
if len(refs) == 0 {
return
}
// Group refs by filer ID, preserving order within each filer.
perFiler := make(map[string][]*filer_pb.LogFileChunkRef)
var filerOrder []string
for _, ref := range refs {
if len(ref.Chunks) == 0 {
continue
}
if _, seen := perFiler[ref.FilerId]; !seen {
filerOrder = append(filerOrder, ref.FilerId)
}
perFiler[ref.FilerId] = append(perFiler[ref.FilerId], ref)
}
if len(filerOrder) == 0 {
return
}
// Single filer fast path: no merge heap needed.
if len(filerOrder) == 1 {
return readFilerFilesWithPrefetch(perFiler[filerOrder[0]], newReader, startTsNs, stopTsNs, filter, processEventFn)
}
// Multiple filers: read each in parallel with prefetching, merge via min-heap.
return readMultiFilersMerged(filerOrder, perFiler, newReader, startTsNs, stopTsNs, filter, processEventFn)
}
// readFilerFilesWithPrefetch reads files for a single filer, prefetching the
// next file while processing entries from the current one.
func readFilerFilesWithPrefetch(
refs []*filer_pb.LogFileChunkRef,
newReader LogFileReaderFn,
startTsNs, stopTsNs int64,
filter PathFilter,
processEventFn ProcessMetadataFunc,
) (lastTsNs int64, err error) {
type prefetchResult struct {
entries []*filer_pb.LogEntry
err error
}
startPrefetch := func(ref *filer_pb.LogFileChunkRef) chan prefetchResult {
ch := make(chan prefetchResult, 1)
go func() {
entries, readErr := readLogFileEntries(newReader, ref.Chunks, startTsNs, stopTsNs)
ch <- prefetchResult{entries, readErr}
}()
return ch
}
var pendingCh chan prefetchResult
if len(refs) > 0 {
pendingCh = startPrefetch(refs[0])
}
for i, ref := range refs {
result := <-pendingCh
// Start prefetching next file while we process current
if i+1 < len(refs) {
pendingCh = startPrefetch(refs[i+1])
}
if result.err != nil {
if isChunkNotFound(result.err) {
glog.V(0).Infof("skip log file filer=%s ts=%d: %v", ref.FilerId, ref.FileTsNs, result.err)
continue
}
return lastTsNs, fmt.Errorf("read log file filer=%s ts=%d: %w", ref.FilerId, ref.FileTsNs, result.err)
}
for _, logEntry := range result.entries {
lastTsNs, err = processOneLogEntry(logEntry, filter, processEventFn)
if err != nil {
return
}
}
}
return
}
// readMultiFilersMerged reads files from multiple filers in parallel (one goroutine
// per filer with prefetching), then merges entries in timestamp order via min-heap.
func readMultiFilersMerged(
filerOrder []string,
perFiler map[string][]*filer_pb.LogFileChunkRef,
newReader LogFileReaderFn,
startTsNs, stopTsNs int64,
filter PathFilter,
processEventFn ProcessMetadataFunc,
) (lastTsNs int64, err error) {
type filerStream struct {
filerId string
entryCh chan *filer_pb.LogEntry
}
streams := make([]filerStream, len(filerOrder))
var wg sync.WaitGroup
for i, filerId := range filerOrder {
entryCh := make(chan *filer_pb.LogEntry, 512)
streams[i] = filerStream{filerId: filerId, entryCh: entryCh}
wg.Add(1)
go func(refs []*filer_pb.LogFileChunkRef, ch chan *filer_pb.LogEntry) {
defer wg.Done()
defer close(ch)
readFilerFilesToChannel(refs, newReader, startTsNs, stopTsNs, ch)
}(perFiler[filerId], entryCh)
}
// Seed the min-heap with the first entry from each filer
pq := &logEntryHeap{}
heap.Init(pq)
for i := range streams {
if entry, ok := <-streams[i].entryCh; ok {
heap.Push(pq, &logEntryHeapItem{entry: entry, filerIdx: i})
}
}
// Merge loop
for pq.Len() > 0 {
item := heap.Pop(pq).(*logEntryHeapItem)
lastTsNs, err = processOneLogEntry(item.entry, filter, processEventFn)
if err != nil {
for i := range streams {
for range streams[i].entryCh {
}
}
wg.Wait()
return
}
if entry, ok := <-streams[item.filerIdx].entryCh; ok {
heap.Push(pq, &logEntryHeapItem{entry: entry, filerIdx: item.filerIdx})
}
}
wg.Wait()
return
}
func readFilerFilesToChannel(
refs []*filer_pb.LogFileChunkRef,
newReader LogFileReaderFn,
startTsNs, stopTsNs int64,
ch chan *filer_pb.LogEntry,
) {
type prefetchResult struct {
entries []*filer_pb.LogEntry
err error
}
startPrefetch := func(ref *filer_pb.LogFileChunkRef) chan prefetchResult {
resultCh := make(chan prefetchResult, 1)
go func() {
entries, err := readLogFileEntries(newReader, ref.Chunks, startTsNs, stopTsNs)
resultCh <- prefetchResult{entries, err}
}()
return resultCh
}
var pendingCh chan prefetchResult
if len(refs) > 0 {
pendingCh = startPrefetch(refs[0])
}
for i, ref := range refs {
result := <-pendingCh
if i+1 < len(refs) {
pendingCh = startPrefetch(refs[i+1])
}
if result.err != nil {
if isChunkNotFound(result.err) {
glog.V(0).Infof("skip log file filer=%s ts=%d: %v", ref.FilerId, ref.FileTsNs, result.err)
} else {
glog.Errorf("read log file filer=%s ts=%d: %v", ref.FilerId, ref.FileTsNs, result.err)
}
continue
}
for _, entry := range result.entries {
ch <- entry
}
}
}
func processOneLogEntry(logEntry *filer_pb.LogEntry, filter PathFilter, processEventFn ProcessMetadataFunc) (int64, error) {
event := &filer_pb.SubscribeMetadataResponse{}
if err := proto.Unmarshal(logEntry.Data, event); err != nil {
glog.Errorf("unmarshal log entry: %v", err)
return 0, nil // skip corrupt entries
}
if !matchesFilter(event, filter) {
return event.TsNs, nil
}
if err := processEventFn(event); err != nil {
return event.TsNs, fmt.Errorf("process event: %w", err)
}
return event.TsNs, nil
}
// --- path filtering (mirrors server-side eachEventNotificationFn logic) ---
const systemLogDir = "/topics/.system/log"
func matchesFilter(resp *filer_pb.SubscribeMetadataResponse, filter PathFilter) bool {
var entryName string
if resp.EventNotification != nil {
if resp.EventNotification.OldEntry != nil {
entryName = resp.EventNotification.OldEntry.Name
} else if resp.EventNotification.NewEntry != nil {
entryName = resp.EventNotification.NewEntry.Name
}
}
fullpath := util.Join(resp.Directory, entryName)
// Skip internal meta log entries
if strings.HasPrefix(fullpath, systemLogDir) {
return false
}
// Check AdditionalPathPrefixes
for _, p := range filter.AdditionalPathPrefixes {
if strings.HasPrefix(fullpath, p) {
return true
}
}
// Check DirectoriesToWatch (exact directory match)
for _, dir := range filter.DirectoriesToWatch {
if resp.Directory == dir {
return true
}
}
// Check primary PathPrefix
if filter.PathPrefix == "" || filter.PathPrefix == "/" {
return true
}
if strings.HasPrefix(fullpath, filter.PathPrefix) {
return true
}
// Check rename target
if resp.EventNotification != nil && resp.EventNotification.NewParentPath != "" {
newFullPath := util.Join(resp.EventNotification.NewParentPath, entryName)
if strings.HasPrefix(newFullPath, filter.PathPrefix) {
return true
}
}
return false
}
// isChunkNotFound checks if an error indicates a missing volume chunk.
// Matches the server-side isChunkNotFoundError logic.
func isChunkNotFound(err error) bool {
if err == nil {
return false
}
s := err.Error()
return strings.Contains(s, "not found") || strings.Contains(s, "status 404")
}
// --- min-heap for merging entries across filers ---
type logEntryHeapItem struct {
entry *filer_pb.LogEntry
filerIdx int
}
type logEntryHeap []*logEntryHeapItem
func (h logEntryHeap) Len() int { return len(h) }
func (h logEntryHeap) Less(i, j int) bool { return h[i].entry.TsNs < h[j].entry.TsNs }
func (h logEntryHeap) Swap(i, j int) { h[i], h[j] = h[j], h[i] }
func (h *logEntryHeap) Push(x any) { *h = append(*h, x.(*logEntryHeapItem)) }
func (h *logEntryHeap) Pop() any {
old := *h
n := len(old)
item := old[n-1]
old[n-1] = nil
*h = old[:n-1]
return item
}
// --- log file parsing (uses io.ReadFull for correct partial-read handling) ---
func readLogFileEntries(newReader LogFileReaderFn, chunks []*filer_pb.FileChunk, startTsNs, stopTsNs int64) ([]*filer_pb.LogEntry, error) {
reader, err := newReader(chunks)
if err != nil {
return nil, fmt.Errorf("create reader: %w", err)
}
defer reader.Close()
sizeBuf := make([]byte, 4)
var entries []*filer_pb.LogEntry
for {
_, err := io.ReadFull(reader, sizeBuf)
if err != nil {
if err == io.EOF || err == io.ErrUnexpectedEOF {
break
}
return entries, err
}
size := util.BytesToUint32(sizeBuf)
entryData := make([]byte, size)
_, err = io.ReadFull(reader, entryData)
if err != nil {
return entries, err
}
logEntry := &filer_pb.LogEntry{}
if err = proto.Unmarshal(entryData, logEntry); err != nil {
return entries, err
}
if logEntry.TsNs <= startTsNs {
continue
}
if stopTsNs != 0 && logEntry.TsNs > stopTsNs {
break
}
entries = append(entries, logEntry)
}
return entries, nil
}