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

@@ -114,12 +114,23 @@ func (fs *FilerServer) doCacheRemoteObjectToLocalCluster(ctx context.Context, re
}
assignRequest, altRequest := so.ToAssignRequests(1)
// find a good chunk size
chunkSize := int64(5 * 1024 * 1024)
chunkCount := entry.Remote.RemoteSize/chunkSize + 1
for chunkCount > 1000 && chunkSize < int64(fs.option.MaxMB)*1024*1024/2 {
chunkSize *= 2
chunkCount = entry.Remote.RemoteSize/chunkSize + 1
// adaptive chunk size: target ~32 chunks per file to balance
// per-chunk overhead (volume assign, gRPC, needle write) against parallelism
chunkSize := int64(5 * 1024 * 1024) // 5MB floor
maxChunkSize := int64(fs.option.MaxMB) * 1024 * 1024
if maxChunkSize < chunkSize {
maxChunkSize = chunkSize
}
targetChunks := int64(32)
if entry.Remote.RemoteSize/targetChunks > chunkSize {
chunkSize = entry.Remote.RemoteSize / targetChunks
if chunkSize > maxChunkSize {
chunkSize = maxChunkSize
}
}
// final safety check: ensure no more than 1000 chunks
if (entry.Remote.RemoteSize+chunkSize-1)/chunkSize > 1000 {
chunkSize = (entry.Remote.RemoteSize + 999) / 1000
}
dest := util.FullPath(remoteStorageMountedLocation.Path).Child(string(util.FullPath(req.Directory).Child(req.Name))[len(localMountedDir):])
@@ -129,7 +140,20 @@ func (fs *FilerServer) doCacheRemoteObjectToLocalCluster(ctx context.Context, re
var fetchAndWriteErr error
var wg sync.WaitGroup
limitedConcurrentExecutor := util.NewLimitedConcurrentExecutor(8)
chunkConcurrency := int(req.ChunkConcurrency)
if chunkConcurrency <= 0 {
chunkConcurrency = 8
} else if chunkConcurrency > 1024 {
glog.V(0).Infof("capping chunkConcurrency from %d to 1024", chunkConcurrency)
chunkConcurrency = 1024
}
downloadConcurrency := req.DownloadConcurrency
if downloadConcurrency > 1024 {
glog.V(0).Infof("capping downloadConcurrency from %d to 1024", downloadConcurrency)
downloadConcurrency = 1024
}
limitedConcurrentExecutor := util.NewLimitedConcurrentExecutor(chunkConcurrency)
for offset := int64(0); offset < entry.Remote.RemoteSize; offset += chunkSize {
localOffset := offset
@@ -183,14 +207,15 @@ func (fs *FilerServer) doCacheRemoteObjectToLocalCluster(ctx context.Context, re
var etag string
err = operation.WithVolumeServerClient(false, assignedServerAddress, fs.grpcDialOption, func(volumeServerClient volume_server_pb.VolumeServerClient) error {
resp, fetchErr := volumeServerClient.FetchAndWriteNeedle(context.Background(), &volume_server_pb.FetchAndWriteNeedleRequest{
VolumeId: uint32(fileId.VolumeId),
NeedleId: uint64(fileId.Key),
Cookie: uint32(fileId.Cookie),
Offset: localOffset,
Size: size,
Replicas: replicas,
Auth: string(assignResult.Auth),
RemoteConf: storageConf,
VolumeId: uint32(fileId.VolumeId),
NeedleId: uint64(fileId.Key),
Cookie: uint32(fileId.Cookie),
Offset: localOffset,
Size: size,
Replicas: replicas,
Auth: string(assignResult.Auth),
DownloadConcurrency: downloadConcurrency,
RemoteConf: storageConf,
RemoteLocation: &remote_pb.RemoteStorageLocation{
Name: remoteStorageMountedLocation.Name,
Bucket: remoteStorageMountedLocation.Bucket,

View File

@@ -34,9 +34,21 @@ func (vs *VolumeServer) FetchAndWriteNeedle(ctx context.Context, req *volume_ser
remoteStorageLocation := req.RemoteLocation
data, ReadRemoteErr := client.ReadFile(remoteStorageLocation, req.Offset, req.Size)
if ReadRemoteErr != nil {
return nil, fmt.Errorf("read from remote %+v: %v", remoteStorageLocation, ReadRemoteErr)
var data []byte
var readRemoteErr error
if cr, ok := client.(remote_storage.RemoteStorageConcurrentReader); ok {
concurrency := int(req.DownloadConcurrency)
if concurrency <= 0 {
concurrency = 0 // let the implementation choose its default
} else if concurrency > 64 {
concurrency = 64
}
data, readRemoteErr = cr.ReadFileWithConcurrency(remoteStorageLocation, req.Offset, req.Size, concurrency)
} else {
data, readRemoteErr = client.ReadFile(remoteStorageLocation, req.Offset, req.Size)
}
if readRemoteErr != nil {
return nil, fmt.Errorf("read from remote %+v: %w", remoteStorageLocation, readRemoteErr)
}
var wg sync.WaitGroup