Files
seaweedFS/weed/remote_storage/remote_storage.go
Chris Lu 81369b8a83 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.
2026-03-17 16:49:56 -07:00

175 lines
5.3 KiB
Go

package remote_storage
import (
"context"
"errors"
"fmt"
"io"
"sort"
"strings"
"sync"
"time"
"github.com/seaweedfs/seaweedfs/weed/pb/filer_pb"
"github.com/seaweedfs/seaweedfs/weed/pb/remote_pb"
"google.golang.org/protobuf/proto"
)
const slash = "/"
func ParseLocationName(remote string) (locationName string) {
remote = strings.TrimSuffix(remote, slash)
parts := strings.SplitN(remote, slash, 2)
if len(parts) >= 1 {
return parts[0]
}
return
}
func parseBucketLocation(remote string) (loc *remote_pb.RemoteStorageLocation) {
loc = &remote_pb.RemoteStorageLocation{}
remote = strings.TrimSuffix(remote, slash)
parts := strings.SplitN(remote, slash, 3)
if len(parts) >= 1 {
loc.Name = parts[0]
}
if len(parts) >= 2 {
loc.Bucket = parts[1]
}
loc.Path = remote[len(loc.Name)+1+len(loc.Bucket):]
if loc.Path == "" {
loc.Path = slash
}
return
}
func parseNoBucketLocation(remote string) (loc *remote_pb.RemoteStorageLocation) {
loc = &remote_pb.RemoteStorageLocation{}
remote = strings.TrimSuffix(remote, slash)
parts := strings.SplitN(remote, slash, 2)
if len(parts) >= 1 {
loc.Name = parts[0]
}
loc.Path = remote[len(loc.Name):]
if loc.Path == "" {
loc.Path = slash
}
return
}
func FormatLocation(loc *remote_pb.RemoteStorageLocation) string {
if loc.Bucket == "" {
return fmt.Sprintf("%s%s", loc.Name, loc.Path)
}
return fmt.Sprintf("%s/%s%s", loc.Name, loc.Bucket, loc.Path)
}
type VisitFunc func(dir string, name string, isDirectory bool, remoteEntry *filer_pb.RemoteEntry) error
type Bucket struct {
Name string
CreatedAt time.Time
}
// ErrRemoteObjectNotFound is returned by StatFile when the object does not exist in the remote storage backend.
var ErrRemoteObjectNotFound = errors.New("remote object not found")
type RemoteStorageClient interface {
Traverse(loc *remote_pb.RemoteStorageLocation, visitFn VisitFunc) error
ListDirectory(ctx context.Context, loc *remote_pb.RemoteStorageLocation, visitFn VisitFunc) error
StatFile(loc *remote_pb.RemoteStorageLocation) (remoteEntry *filer_pb.RemoteEntry, err error)
ReadFile(loc *remote_pb.RemoteStorageLocation, offset int64, size int64) (data []byte, err error)
WriteDirectory(loc *remote_pb.RemoteStorageLocation, entry *filer_pb.Entry) (err error)
RemoveDirectory(loc *remote_pb.RemoteStorageLocation) (err error)
WriteFile(loc *remote_pb.RemoteStorageLocation, entry *filer_pb.Entry, reader io.Reader) (remoteEntry *filer_pb.RemoteEntry, err error)
UpdateFileMetadata(loc *remote_pb.RemoteStorageLocation, oldEntry *filer_pb.Entry, newEntry *filer_pb.Entry) (err error)
DeleteFile(loc *remote_pb.RemoteStorageLocation) (err error)
ListBuckets() ([]*Bucket, error)
CreateBucket(name string) (err error)
DeleteBucket(name string) (err error)
}
// RemoteStorageConcurrentReader is an optional interface for remote storage clients
// that support configurable download concurrency for multipart downloads.
type RemoteStorageConcurrentReader interface {
ReadFileWithConcurrency(loc *remote_pb.RemoteStorageLocation, offset int64, size int64, concurrency int) (data []byte, err error)
}
type RemoteStorageClientMaker interface {
Make(remoteConf *remote_pb.RemoteConf) (RemoteStorageClient, error)
HasBucket() bool
}
type CachedRemoteStorageClient struct {
*remote_pb.RemoteConf
RemoteStorageClient
}
var (
RemoteStorageClientMakers = make(map[string]RemoteStorageClientMaker)
remoteStorageClients = make(map[string]CachedRemoteStorageClient)
remoteStorageClientsLock sync.Mutex
)
func GetAllRemoteStorageNames() string {
var storageNames []string
for k := range RemoteStorageClientMakers {
storageNames = append(storageNames, k)
}
sort.Strings(storageNames)
return strings.Join(storageNames, "|")
}
func GetRemoteStorageNamesHasBucket() string {
var storageNames []string
for k, m := range RemoteStorageClientMakers {
if m.HasBucket() {
storageNames = append(storageNames, k)
}
}
sort.Strings(storageNames)
return strings.Join(storageNames, "|")
}
func ParseRemoteLocation(remoteConfType string, remote string) (remoteStorageLocation *remote_pb.RemoteStorageLocation, err error) {
maker, found := RemoteStorageClientMakers[remoteConfType]
if !found {
return nil, fmt.Errorf("remote storage type %s not found", remoteConfType)
}
if !maker.HasBucket() {
return parseNoBucketLocation(remote), nil
}
return parseBucketLocation(remote), nil
}
func makeRemoteStorageClient(remoteConf *remote_pb.RemoteConf) (RemoteStorageClient, error) {
maker, found := RemoteStorageClientMakers[remoteConf.Type]
if !found {
return nil, fmt.Errorf("remote storage type %s not found", remoteConf.Type)
}
return maker.Make(remoteConf)
}
func GetRemoteStorage(remoteConf *remote_pb.RemoteConf) (RemoteStorageClient, error) {
remoteStorageClientsLock.Lock()
defer remoteStorageClientsLock.Unlock()
existingRemoteStorageClient, found := remoteStorageClients[remoteConf.Name]
if found && proto.Equal(existingRemoteStorageClient.RemoteConf, remoteConf) {
return existingRemoteStorageClient.RemoteStorageClient, nil
}
newRemoteStorageClient, err := makeRemoteStorageClient(remoteConf)
if err != nil {
return nil, fmt.Errorf("make remote storage client %s: %v", remoteConf.Name, err)
}
remoteStorageClients[remoteConf.Name] = CachedRemoteStorageClient{
RemoteConf: remoteConf,
RemoteStorageClient: newRemoteStorageClient,
}
return newRemoteStorageClient, nil
}