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

@@ -5414,14 +5414,15 @@ func (*VolumeServerLeaveResponse) Descriptor() ([]byte, []int) {
// remote storage
type FetchAndWriteNeedleRequest struct {
state protoimpl.MessageState `protogen:"open.v1"`
VolumeId uint32 `protobuf:"varint,1,opt,name=volume_id,json=volumeId,proto3" json:"volume_id,omitempty"`
NeedleId uint64 `protobuf:"varint,2,opt,name=needle_id,json=needleId,proto3" json:"needle_id,omitempty"`
Cookie uint32 `protobuf:"varint,3,opt,name=cookie,proto3" json:"cookie,omitempty"`
Offset int64 `protobuf:"varint,4,opt,name=offset,proto3" json:"offset,omitempty"`
Size int64 `protobuf:"varint,5,opt,name=size,proto3" json:"size,omitempty"`
Replicas []*FetchAndWriteNeedleRequest_Replica `protobuf:"bytes,6,rep,name=replicas,proto3" json:"replicas,omitempty"`
Auth string `protobuf:"bytes,7,opt,name=auth,proto3" json:"auth,omitempty"`
state protoimpl.MessageState `protogen:"open.v1"`
VolumeId uint32 `protobuf:"varint,1,opt,name=volume_id,json=volumeId,proto3" json:"volume_id,omitempty"`
NeedleId uint64 `protobuf:"varint,2,opt,name=needle_id,json=needleId,proto3" json:"needle_id,omitempty"`
Cookie uint32 `protobuf:"varint,3,opt,name=cookie,proto3" json:"cookie,omitempty"`
Offset int64 `protobuf:"varint,4,opt,name=offset,proto3" json:"offset,omitempty"`
Size int64 `protobuf:"varint,5,opt,name=size,proto3" json:"size,omitempty"`
Replicas []*FetchAndWriteNeedleRequest_Replica `protobuf:"bytes,6,rep,name=replicas,proto3" json:"replicas,omitempty"`
Auth string `protobuf:"bytes,7,opt,name=auth,proto3" json:"auth,omitempty"`
DownloadConcurrency int32 `protobuf:"varint,8,opt,name=download_concurrency,json=downloadConcurrency,proto3" json:"download_concurrency,omitempty"` // multipart download concurrency if supported by the remote storage client; for S3, 0 = default (5)
// remote conf
RemoteConf *remote_pb.RemoteConf `protobuf:"bytes,15,opt,name=remote_conf,json=remoteConf,proto3" json:"remote_conf,omitempty"`
RemoteLocation *remote_pb.RemoteStorageLocation `protobuf:"bytes,16,opt,name=remote_location,json=remoteLocation,proto3" json:"remote_location,omitempty"`
@@ -5508,6 +5509,13 @@ func (x *FetchAndWriteNeedleRequest) GetAuth() string {
return ""
}
func (x *FetchAndWriteNeedleRequest) GetDownloadConcurrency() int32 {
if x != nil {
return x.DownloadConcurrency
}
return 0
}
func (x *FetchAndWriteNeedleRequest) GetRemoteConf() *remote_pb.RemoteConf {
if x != nil {
return x.RemoteConf
@@ -7119,7 +7127,7 @@ const file_volume_server_proto_rawDesc = "" +
"\x04rack\x18\x05 \x01(\tR\x04rack\x129\n" +
"\x05state\x18\x06 \x01(\v2#.volume_server_pb.VolumeServerStateR\x05state\"\x1a\n" +
"\x18VolumeServerLeaveRequest\"\x1b\n" +
"\x19VolumeServerLeaveResponse\"\xdc\x03\n" +
"\x19VolumeServerLeaveResponse\"\x8f\x04\n" +
"\x1aFetchAndWriteNeedleRequest\x12\x1b\n" +
"\tvolume_id\x18\x01 \x01(\rR\bvolumeId\x12\x1b\n" +
"\tneedle_id\x18\x02 \x01(\x04R\bneedleId\x12\x16\n" +
@@ -7127,7 +7135,8 @@ const file_volume_server_proto_rawDesc = "" +
"\x06offset\x18\x04 \x01(\x03R\x06offset\x12\x12\n" +
"\x04size\x18\x05 \x01(\x03R\x04size\x12P\n" +
"\breplicas\x18\x06 \x03(\v24.volume_server_pb.FetchAndWriteNeedleRequest.ReplicaR\breplicas\x12\x12\n" +
"\x04auth\x18\a \x01(\tR\x04auth\x126\n" +
"\x04auth\x18\a \x01(\tR\x04auth\x121\n" +
"\x14download_concurrency\x18\b \x01(\x05R\x13downloadConcurrency\x126\n" +
"\vremote_conf\x18\x0f \x01(\v2\x15.remote_pb.RemoteConfR\n" +
"remoteConf\x12I\n" +
"\x0fremote_location\x18\x10 \x01(\v2 .remote_pb.RemoteStorageLocationR\x0eremoteLocation\x1aW\n" +