improve: large file sync throughput for remote.cache and filer.sync (#8676)

* improve large file sync throughput for remote.cache and filer.sync

Three main throughput improvements:

1. Adaptive chunk sizing for remote.cache: targets ~32 chunks per file
   instead of always starting at 5MB. A 500MB file now uses ~16MB chunks
   (32 chunks) instead of 5MB chunks (100 chunks), reducing per-chunk
   overhead (volume assign, gRPC call, needle write) by 3x.

2. Configurable concurrency at every layer:
   - remote.cache chunk concurrency: -chunkConcurrency flag (default 8)
   - remote.cache S3 download concurrency: -downloadConcurrency flag
     (default raised from 1 to 5 per chunk)
   - filer.sync chunk concurrency: -chunkConcurrency flag (default 32)

3. S3 multipart download concurrency raised from 1 to 5: the S3 manager
   downloader was using Concurrency=1, serializing all part downloads
   within each chunk. This alone can 5x per-chunk download speed.

The concurrency values flow through the gRPC request chain:
  shell command → CacheRemoteObjectToLocalClusterRequest →
  FetchAndWriteNeedleRequest → S3 downloader

Zero values in the request mean "use server defaults", maintaining
full backward compatibility with existing callers.

Ref #8481

* fix: use full maxMB for chunk size cap and remove loop guard

Address review feedback:
- Use full maxMB instead of maxMB/2 for maxChunkSize to avoid
  unnecessarily limiting chunk size for very large files.
- Remove chunkSize < maxChunkSize guard from the safety loop so it
  can always grow past maxChunkSize when needed to stay under 1000
  chunks (e.g., extremely large files with small maxMB).

* address review feedback: help text, validation, naming, docs

- Fix help text for -chunkConcurrency and -downloadConcurrency flags
  to say "0 = server default" instead of advertising specific numeric
  defaults that could drift from the server implementation.
- Validate chunkConcurrency and downloadConcurrency are within int32
  range before narrowing, returning a user-facing error if out of range.
- Rename ReadRemoteErr to readRemoteErr to follow Go naming conventions.
- Add doc comment to SetChunkConcurrency noting it must be called
  during initialization before replication goroutines start.
- Replace doubling loop in chunk size safety check with direct
  ceil(remoteSize/1000) computation to guarantee the 1000-chunk cap.

* address Copilot review: clamp concurrency, fix chunk count, clarify proto docs

- Use ceiling division for chunk count check to avoid overcounting
  when file size is an exact multiple of chunk size.
- Clamp chunkConcurrency (max 1024) and downloadConcurrency (max 1024
  at filer, max 64 at volume server) to prevent excessive goroutines.
- Always use ReadFileWithConcurrency when the client supports it,
  falling back to the implementation's default when value is 0.
- Clarify proto comments that download_concurrency only applies when
  the remote storage client supports it (currently S3).
- Include specific server defaults in help text (e.g., "0 = server
  default 8") so users see the actual values in -h output.

* fix data race on executionErr and use %w for error wrapping

- Protect concurrent writes to executionErr in remote.cache worker
  goroutines with a sync.Mutex to eliminate the data race.
- Use %w instead of %v in volume_grpc_remote.go error formatting
  to preserve the error chain for errors.Is/errors.As callers.
This commit is contained in:
Chris Lu
2026-03-17 16:49:56 -07:00
committed by GitHub
parent f4073107cb
commit 81369b8a83
13 changed files with 160 additions and 48 deletions

View File

@@ -48,8 +48,9 @@ type SyncOptions struct {
bProxyByFiler *bool
metricsHttpIp *string
metricsHttpPort *int
concurrency *int
aDoDeleteFiles *bool
concurrency *int
chunkConcurrency *int
aDoDeleteFiles *bool
bDoDeleteFiles *bool
clientId int32
clientEpoch atomic.Int32
@@ -104,6 +105,7 @@ func init() {
syncOptions.aFromTsMs = cmdFilerSynchronize.Flag.Int64("a.fromTsMs", 0, "synchronization from timestamp on filer A. The unit is millisecond")
syncOptions.bFromTsMs = cmdFilerSynchronize.Flag.Int64("b.fromTsMs", 0, "synchronization from timestamp on filer B. The unit is millisecond")
syncOptions.concurrency = cmdFilerSynchronize.Flag.Int("concurrency", DefaultConcurrencyLimit, "The maximum number of files that will be synced concurrently.")
syncOptions.chunkConcurrency = cmdFilerSynchronize.Flag.Int("chunkConcurrency", 32, "The maximum number of chunks that will be replicated concurrently per file.")
syncCpuProfile = cmdFilerSynchronize.Flag.String("cpuprofile", "", "cpu profile output file")
syncMemProfile = cmdFilerSynchronize.Flag.String("memprofile", "", "memory profile output file")
syncOptions.metricsHttpIp = cmdFilerSynchronize.Flag.String("metricsIp", "", "metrics listen ip")
@@ -210,6 +212,7 @@ func runFilerSynchronize(cmd *Command, args []string) bool {
*syncOptions.bDiskType,
*syncOptions.bDebug,
*syncOptions.concurrency,
*syncOptions.chunkConcurrency,
*syncOptions.bDoDeleteFiles,
aFilerSignature,
bFilerSignature,
@@ -249,6 +252,7 @@ func runFilerSynchronize(cmd *Command, args []string) bool {
*syncOptions.aDiskType,
*syncOptions.aDebug,
*syncOptions.concurrency,
*syncOptions.chunkConcurrency,
*syncOptions.aDoDeleteFiles,
bFilerSignature,
aFilerSignature,
@@ -281,7 +285,7 @@ func initOffsetFromTsMs(grpcDialOption grpc.DialOption, targetFiler pb.ServerAdd
}
func doSubscribeFilerMetaChanges(clientId int32, clientEpoch int32, grpcDialOption grpc.DialOption, sourceFiler pb.ServerAddress, sourcePath string, sourceExcludePaths []string, sourceReadChunkFromFiler bool, targetFiler pb.ServerAddress, targetPath string,
replicationStr, collection string, ttlSec int, sinkWriteChunkByFiler bool, diskType string, debug bool, concurrency int, doDeleteFiles bool, sourceFilerSignature int32, targetFilerSignature int32, statePtr *atomic.Pointer[syncState]) error {
replicationStr, collection string, ttlSec int, sinkWriteChunkByFiler bool, diskType string, debug bool, concurrency int, chunkConcurrency int, doDeleteFiles bool, sourceFilerSignature int32, targetFilerSignature int32, statePtr *atomic.Pointer[syncState]) error {
// if first time, start from now
// if has previously synced, resume from that point of time
@@ -297,6 +301,7 @@ func doSubscribeFilerMetaChanges(clientId int32, clientEpoch int32, grpcDialOpti
filerSource.DoInitialize(sourceFiler.ToHttpAddress(), sourceFiler.ToGrpcAddress(), sourcePath, sourceReadChunkFromFiler)
filerSink := &filersink.FilerSink{}
filerSink.DoInitialize(targetFiler.ToHttpAddress(), targetFiler.ToGrpcAddress(), targetPath, replicationStr, collection, ttlSec, diskType, grpcDialOption, sinkWriteChunkByFiler)
filerSink.SetChunkConcurrency(chunkConcurrency)
filerSink.SetSourceFiler(filerSource)
persistEventFn := genProcessFunction(sourcePath, targetPath, sourceExcludePaths, nil, filerSink, doDeleteFiles, debug)