S3: adds FilerClient to use cached volume id (#7518)

* adds FilerClient to use cached volume id

* refactor: MasterClient embeds vidMapClient to eliminate ~150 lines of duplication

- Create masterVolumeProvider that implements VolumeLocationProvider
- MasterClient now embeds vidMapClient instead of maintaining duplicate cache logic
- Removed duplicate methods: LookupVolumeIdsWithFallback, getStableVidMap, etc.
- MasterClient still receives real-time updates via KeepConnected streaming
- Updates call inherited addLocation/deleteLocation from vidMapClient
- Benefits: DRY principle, shared singleflight, cache chain logic reused
- Zero behavioral changes - only architectural improvement

* refactor: mount uses FilerClient for efficient volume location caching

- Add configurable vidMap cache size (default: 5 historical snapshots)
- Add FilerClientOption struct for clean configuration
  * GrpcTimeout: default 5 seconds (prevents hanging requests)
  * UrlPreference: PreferUrl or PreferPublicUrl
  * CacheSize: number of historical vidMap snapshots (for volume moves)
- NewFilerClient uses option struct for better API extensibility
- Improved error handling in filerVolumeProvider.LookupVolumeIds:
  * Distinguish genuine 'not found' from communication failures
  * Log volumes missing from filer response
  * Return proper error context with volume count
  * Document that filer Locations lacks Error field (unlike master)
- FilerClient.GetLookupFileIdFunction() handles URL preference automatically
- Mount (WFS) creates FilerClient with appropriate options
- Benefits for weed mount:
  * Singleflight: Deduplicates concurrent volume lookups
  * Cache history: Old volume locations available briefly when volumes move
  * Configurable cache depth: Tune for different deployment environments
  * Battle-tested vidMap cache with cache chain
  * Better concurrency handling with timeout protection
  * Improved error visibility and debugging
- Old filer.LookupFn() kept for backward compatibility
- Performance improvement for mount operations with high concurrency

* fix: prevent vidMap swap race condition in LookupFileIdWithFallback

- Hold vidMapLock.RLock() during entire vm.LookupFileId() call
- Prevents resetVidMap() from swapping vidMap mid-operation
- Ensures atomic access to the current vidMap instance
- Added documentation warnings to getStableVidMap() about swap risks
- Enhanced withCurrentVidMap() documentation for clarity

This fixes a subtle race condition where:
1. Thread A: acquires lock, gets vm pointer, releases lock
2. Thread B: calls resetVidMap(), swaps vc.vidMap
3. Thread A: calls vm.LookupFileId() on old/stale vidMap

While the old vidMap remains valid (in cache chain), holding the lock
ensures we consistently use the current vidMap for the entire operation.

* fix: FilerClient supports multiple filer addresses for high availability

Critical fix: FilerClient now accepts []ServerAddress instead of single address
- Prevents mount failure when first filer is down (regression fix)
- Implements automatic failover to remaining filers
- Uses round-robin with atomic index tracking (same pattern as WFS.WithFilerClient)
- Retries all configured filers before giving up
- Updates successful filer index for future requests

Changes:
- NewFilerClient([]pb.ServerAddress, ...) instead of (pb.ServerAddress, ...)
- filerVolumeProvider references FilerClient for failover access
- LookupVolumeIds tries all filers with util.Retry pattern
- Mount passes all option.FilerAddresses for HA
- S3 wraps single filer in slice for API consistency

This restores the high availability that existed in the old implementation
where mount would automatically failover between configured filers.

* fix: restore leader change detection in KeepConnected stream loop

Critical fix: Leader change detection was accidentally removed from the streaming loop
- Master can announce leader changes during an active KeepConnected stream
- Without this check, client continues talking to non-leader until connection breaks
- This can lead to stale data or operational errors

The check needs to be in TWO places:
1. Initial response (lines 178-187): Detect redirect on first connect
2. Stream loop (lines 203-209): Detect leader changes during active stream

Restored the loop check that was accidentally removed during refactoring.
This ensures the client immediately reconnects to new leader when announced.

* improve: address code review findings on error handling and documentation

1. Master provider now preserves per-volume errors
   - Surface detailed errors from master (e.g., misconfiguration, deletion)
   - Return partial results with aggregated errors using errors.Join
   - Callers can now distinguish specific volume failures from general errors
   - Addresses issue of losing vidLoc.Error details

2. Document GetMaster initialization contract
   - Add comprehensive documentation explaining blocking behavior
   - Clarify that KeepConnectedToMaster must be started first
   - Provide typical initialization pattern example
   - Prevent confusing timeouts during warm-up

3. Document partial results API contract
   - LookupVolumeIdsWithFallback explicitly documents partial results
   - Clear examples of how to handle result + error combinations
   - Helps prevent callers from discarding valid partial results

4. Add safeguards to legacy filer.LookupFn
   - Add deprecation warning with migration guidance
   - Implement simple 10,000 entry cache limit
   - Log warning when limit reached
   - Recommend wdclient.FilerClient for new code
   - Prevents unbounded memory growth in long-running processes

These changes improve API clarity and operational safety while maintaining
backward compatibility.

* fix: handle partial results correctly in LookupVolumeIdsWithFallback callers

Two callers were discarding partial results by checking err before processing
the result map. While these are currently single-volume lookups (so partial
results aren't possible), the code was fragile and would break if we ever
batched multiple volumes together.

Changes:
- Check result map FIRST, then conditionally check error
- If volume is found in result, use it (ignore errors about other volumes)
- If volume is NOT found and err != nil, include error context with %w
- Add defensive comments explaining the pattern for future maintainers

This makes the code:
1. Correct for future batched lookups
2. More informative (preserves underlying error details)
3. Consistent with filer_grpc_server.go which already handles this correctly

Example: If looking up ["1", "2", "999"] and only 999 fails, callers
looking for volumes 1 or 2 will succeed instead of failing unnecessarily.

* improve: address remaining code review findings

1. Lazy initialize FilerClient in mount for proxy-only setups
   - Only create FilerClient when VolumeServerAccess != "filerProxy"
   - Avoids wasted work when all reads proxy through filer
   - filerClient is nil for proxy mode, initialized for direct access

2. Fix inaccurate deprecation comment in filer.LookupFn
   - Updated comment to reflect current behavior (10k bounded cache)
   - Removed claim of "unbounded growth" after adding size limit
   - Still directs new code to wdclient.FilerClient for better features

3. Audit all MasterClient usages for KeepConnectedToMaster
   - Verified all production callers start KeepConnectedToMaster early
   - Filer, Shell, Master, Broker, Benchmark, Admin all correct
   - IAM creates MasterClient but never uses it (harmless)
   - Test code doesn't need KeepConnectedToMaster (mocks)

All callers properly follow the initialization pattern documented in
GetMaster(), preventing unexpected blocking or timeouts.

* fix: restore observability instrumentation in MasterClient

During the refactoring, several important stats counters and logging
statements were accidentally removed from tryConnectToMaster. These are
critical for monitoring and debugging the health of master client connections.

Restored instrumentation:
1. stats.MasterClientConnectCounter("total") - tracks all connection attempts
2. stats.MasterClientConnectCounter(FailedToKeepConnected) - when KeepConnected stream fails
3. stats.MasterClientConnectCounter(FailedToReceive) - when Recv() fails in loop
4. stats.MasterClientConnectCounter(Failed) - when overall gprcErr occurs
5. stats.MasterClientConnectCounter(OnPeerUpdate) - when peer updates detected

Additionally restored peer update logging:
- "+ filer@host noticed group.type address" for node additions
- "- filer@host noticed group.type address" for node removals
- Only logs updates matching the client's FilerGroup for noise reduction

This information is valuable for:
- Monitoring cluster health and connection stability
- Debugging cluster membership changes
- Tracking master failover and reconnection patterns
- Identifying network issues between clients and masters

No functional changes - purely observability restoration.

* improve: implement gRPC-aware retry for FilerClient volume lookups

The previous implementation used util.Retry which only retries errors
containing the string "transport". This is insufficient for handling
the full range of transient gRPC errors.

Changes:
1. Added isRetryableGrpcError() to properly inspect gRPC status codes
   - Retries: Unavailable, DeadlineExceeded, ResourceExhausted, Aborted
   - Falls back to string matching for non-gRPC network errors

2. Replaced util.Retry with custom retry loop
   - 3 attempts with exponential backoff (1s, 1.5s, 2.25s)
   - Tries all N filers on each attempt (N*3 total attempts max)
   - Fast-fails on non-retryable errors (NotFound, PermissionDenied, etc.)

3. Improved logging
   - Shows both filer attempt (x/N) and retry attempt (y/3)
   - Logs retry reason and wait time for debugging

Benefits:
- Better handling of transient gRPC failures (server restarts, load spikes)
- Faster failure for permanent errors (no wasted retries)
- More informative logs for troubleshooting
- Maintains existing HA failover across multiple filers

Example: If all 3 filers return Unavailable (server overload):
- Attempt 1: try all 3 filers, wait 1s
- Attempt 2: try all 3 filers, wait 1.5s
- Attempt 3: try all 3 filers, fail

Example: If filer returns NotFound (volume doesn't exist):
- Attempt 1: try all 3 filers, fast-fail (no retry)

* fmt

* improve: add circuit breaker to skip known-unhealthy filers

The previous implementation tried all filers on every failure, including
known-unhealthy ones. This wasted time retrying permanently down filers.

Problem scenario (3 filers, filer0 is down):
- Last successful: filer1 (saved as filerIndex=1)
- Next lookup when filer1 fails:
  Retry 1: filer1(fail) → filer2(fail) → filer0(fail, wastes 5s timeout)
  Retry 2: filer1(fail) → filer2(fail) → filer0(fail, wastes 5s timeout)
  Retry 3: filer1(fail) → filer2(fail) → filer0(fail, wastes 5s timeout)
  Total wasted: 15 seconds on known-bad filer!

Solution: Circuit breaker pattern
- Track consecutive failures per filer (atomic int32)
- Skip filers with 3+ consecutive failures
- Re-check unhealthy filers every 30 seconds
- Reset failure count on success

New behavior:
- filer0 fails 3 times → marked unhealthy
- Future lookups skip filer0 for 30 seconds
- After 30s, re-check filer0 (allows recovery)
- If filer0 succeeds, reset failure count to 0

Benefits:
1. Avoids wasting time on known-down filers
2. Still sticks to last healthy filer (via filerIndex)
3. Allows recovery (30s re-check window)
4. No configuration needed (automatic)

Implementation details:
- filerHealth struct tracks failureCount (atomic) + lastFailureTime
- shouldSkipUnhealthyFiler(): checks if we should skip this filer
- recordFilerSuccess(): resets failure count to 0
- recordFilerFailure(): increments count, updates timestamp
- Logs when skipping unhealthy filers (V(2) level)

Example with circuit breaker:
- filer0 down, saved filerIndex=1 (filer1 healthy)
- Lookup 1: filer1(ok) → Done (0.01s)
- Lookup 2: filer1(fail) → filer2(ok) → Done, save filerIndex=2 (0.01s)
- Lookup 3: filer2(fail) → skip filer0 (unhealthy) → filer1(ok) → Done (0.01s)

Much better than wasting 15s trying filer0 repeatedly!

* fix: OnPeerUpdate should only process updates for matching FilerGroup

Critical bug: The OnPeerUpdate callback was incorrectly moved outside the
FilerGroup check when restoring observability instrumentation. This caused
clients to process peer updates for ALL filer groups, not just their own.

Problem:
  Before: mc.OnPeerUpdate only called for update.FilerGroup == mc.FilerGroup
  Bug:    mc.OnPeerUpdate called for ALL updates regardless of FilerGroup

Impact:
- Multi-tenant deployments with separate filer groups would see cross-group
  updates (e.g., group A clients processing group B updates)
- Could cause incorrect cluster membership tracking
- OnPeerUpdate handlers (like Filer's DLM ring updates) would receive
  irrelevant updates from other groups

Example scenario:
  Cluster has two filer groups: "production" and "staging"
  Production filer connects with FilerGroup="production"

  Incorrect behavior (bug):
    - Receives "staging" group updates
    - Incorrectly adds staging filers to production DLM ring
    - Cross-tenant data access issues

  Correct behavior (fixed):
    - Only receives "production" group updates
    - Only adds production filers to production DLM ring
    - Proper isolation between groups

Fix:
  Moved mc.OnPeerUpdate(update, time.Now()) back INSIDE the FilerGroup check
  where it belongs, matching the original implementation.

The logging and stats counter were already correctly scoped to matching
FilerGroup, so they remain inside the if block as intended.

* improve: clarify Aborted error handling in volume lookups

Added documentation and logging to address the concern that codes.Aborted
might not always be retryable in all contexts.

Context-specific justification for treating Aborted as retryable:

Volume location lookups (LookupVolume RPC) are simple, read-only operations:
  - No transactions
  - No write conflicts
  - No application-level state changes
  - Idempotent (safe to retry)

In this context, Aborted is most likely caused by:
  - Filer restarting/recovering (transient)
  - Connection interrupted mid-request (transient)
  - Server-side resource cleanup (transient)

NOT caused by:
  - Application-level conflicts (no writes)
  - Transaction failures (no transactions)
  - Logical errors (read-only lookup)

Changes:
1. Added detailed comment explaining the context-specific reasoning
2. Added V(1) logging when treating Aborted as retryable
   - Helps detect misclassification if it occurs
   - Visible in verbose logs for troubleshooting
3. Split switch statement for clarity (one case per line)

If future analysis shows Aborted should not be retried, operators will
now have visibility via logs to make that determination. The logging
provides evidence for future tuning decisions.

Alternative approaches considered but not implemented:
  - Removing Aborted entirely (too conservative for read-only ops)
  - Message content inspection (adds complexity, no known patterns yet)
  - Different handling per RPC type (premature optimization)

* fix: IAM server must start KeepConnectedToMaster for masterClient usage

The IAM server creates and uses a MasterClient but never started
KeepConnectedToMaster, which could cause blocking if IAM config files
have chunks requiring volume lookups.

Problem flow:
  NewIamApiServerWithStore()
    → creates masterClient
    →  NEVER starts KeepConnectedToMaster

  GetS3ApiConfigurationFromFiler()
    → filer.ReadEntry(iama.masterClient, ...)
      → StreamContent(masterClient, ...) if file has chunks
        → masterClient.GetLookupFileIdFunction()
          → GetMaster(ctx) ← BLOCKS indefinitely waiting for connection!

While IAM config files (identity & policies) are typically small and
stored inline without chunks, the code path exists and would block
if the files ever had chunks.

Fix:
  Start KeepConnectedToMaster in background goroutine right after
  creating masterClient, following the documented pattern:

    mc := wdclient.NewMasterClient(...)
    go mc.KeepConnectedToMaster(ctx)

This ensures masterClient is usable if ReadEntry ever needs to
stream chunked content from volume servers.

Note: This bug was dormant because IAM config files are small (<256 bytes)
and SeaweedFS stores small files inline in Entry.Content, not as chunks.
The bug would only manifest if:
  - IAM config grew > 256 bytes (inline threshold)
  - Config was stored as chunks on volume servers
  - ReadEntry called StreamContent
  - GetMaster blocked indefinitely

Now all 9 production MasterClient instances correctly follow the pattern.

* fix: data race on filerHealth.lastFailureTime in circuit breaker

The circuit breaker tracked lastFailureTime as time.Time, which was
written in recordFilerFailure and read in shouldSkipUnhealthyFiler
without synchronization, causing a data race.

Data race scenario:
  Goroutine 1: recordFilerFailure(0)
    health.lastFailureTime = time.Now()  //  unsynchronized write

  Goroutine 2: shouldSkipUnhealthyFiler(0)
    time.Since(health.lastFailureTime)   //  unsynchronized read

  → RACE DETECTED by -race detector

Fix:
  Changed lastFailureTime from time.Time to int64 (lastFailureTimeNs)
  storing Unix nanoseconds for atomic access:

  Write side (recordFilerFailure):
    atomic.StoreInt64(&health.lastFailureTimeNs, time.Now().UnixNano())

  Read side (shouldSkipUnhealthyFiler):
    lastFailureNs := atomic.LoadInt64(&health.lastFailureTimeNs)
    if lastFailureNs == 0 { return false }  // Never failed
    lastFailureTime := time.Unix(0, lastFailureNs)
    time.Since(lastFailureTime) > 30*time.Second

Benefits:
  - Atomic reads/writes (no data race)
  - Efficient (int64 is 8 bytes, always atomic on 64-bit systems)
  - Zero value (0) naturally means "never failed"
  - No mutex needed (lock-free circuit breaker)

Note: sync/atomic was already imported for failureCount, so no new
import needed.

* fix: create fresh timeout context for each filer retry attempt

The timeout context was created once at function start and reused across
all retry attempts, causing subsequent retries to run with progressively
shorter (or expired) deadlines.

Problem flow:
  Line 244: timeoutCtx, cancel := context.WithTimeout(ctx, 5s)
  defer cancel()

  Retry 1, filer 0: client.LookupVolume(timeoutCtx, ...) ← 5s available 
  Retry 1, filer 1: client.LookupVolume(timeoutCtx, ...) ← 3s left
  Retry 1, filer 2: client.LookupVolume(timeoutCtx, ...) ← 0.5s left
  Retry 2, filer 0: client.LookupVolume(timeoutCtx, ...) ← EXPIRED! 

Result: Retries always fail with DeadlineExceeded, defeating the purpose
of retries.

Fix:
  Moved context.WithTimeout inside the per-filer loop, creating a fresh
  timeout context for each attempt:

    for x := 0; x < n; x++ {
      timeoutCtx, cancel := context.WithTimeout(ctx, fc.grpcTimeout)
      err := pb.WithGrpcFilerClient(..., func(client) {
        resp, err := client.LookupVolume(timeoutCtx, ...)
        ...
      })
      cancel()  // Clean up immediately after call
    }

Benefits:
  - Each filer attempt gets full fc.grpcTimeout (default 5s)
  - Retries actually have time to complete
  - No context leaks (cancel called after each attempt)
  - More predictable timeout behavior

Example with fix:
  Retry 1, filer 0: fresh 5s timeout 
  Retry 1, filer 1: fresh 5s timeout 
  Retry 2, filer 0: fresh 5s timeout 

Total max time: 3 retries × 3 filers × 5s = 45s (plus backoff)

Note: The outer ctx (from caller) still provides overall cancellation if
the caller cancels or times out the entire operation.

* fix: always reset vidMap cache on master reconnection

The previous refactoring removed the else block that resets vidMap when
the first message from a newly connected master is not a VolumeLocation.

Problem scenario:
  1. Client connects to master-1 and builds vidMap cache
  2. Master-1 fails, client connects to master-2
  3. First message from master-2 is a ClusterNodeUpdate (not VolumeLocation)
  4. Old code: vidMap is reset and updated 
  5. New code: vidMap is NOT reset 
  6. Result: Client uses stale cache from master-1 → data access errors

Example flow with bug:
  Connect to master-2
  First message: ClusterNodeUpdate {filer.x added}
  → No resetVidMap() call
  → vidMap still has master-1's stale volume locations
  → Client reads from wrong volume servers → 404 errors

Fix:
  Restored the else block that resets vidMap when first message is not
  a VolumeLocation:

    if resp.VolumeLocation != nil {
      // ... check leader, reset, and update ...
    } else {
      // First message is ClusterNodeUpdate or other type
      // Must still reset to avoid stale data
      mc.resetVidMap()
    }

This ensures the cache is always cleared when establishing a new master
connection, regardless of what the first message type is.

Root cause:
  During the vidMapClient refactoring, this else block was accidentally
  dropped, making failover behavior fragile and non-deterministic (depends
  on which message type arrives first from the new master).

Impact:
  - High severity for master failover scenarios
  - Could cause read failures, 404s, or wrong data access
  - Only manifests when first message is not VolumeLocation

* fix: goroutine and connection leak in IAM server shutdown

The IAM server's KeepConnectedToMaster goroutine used context.Background(),
which is non-cancellable, causing the goroutine and its gRPC connections
to leak on server shutdown.

Problem:
  go masterClient.KeepConnectedToMaster(context.Background())

  - context.Background() never cancels
  - KeepConnectedToMaster goroutine runs forever
  - gRPC connection to master stays open
  - No way to stop cleanly on server shutdown

Result: Resource leaks when IAM server is stopped

Fix:
  1. Added shutdownContext and shutdownCancel to IamApiServer struct
  2. Created cancellable context in NewIamApiServerWithStore:
       shutdownCtx, shutdownCancel := context.WithCancel(context.Background())
  3. Pass shutdownCtx to KeepConnectedToMaster:
       go masterClient.KeepConnectedToMaster(shutdownCtx)
  4. Added Shutdown() method to invoke cancel:
       func (iama *IamApiServer) Shutdown() {
           if iama.shutdownCancel != nil {
               iama.shutdownCancel()
           }
       }

  5. Stored masterClient reference on IamApiServer for future use

Benefits:
  - Goroutine stops cleanly when Shutdown() is called
  - gRPC connections are closed properly
  - No resource leaks on server restart/stop
  - Shutdown() is idempotent (safe to call multiple times)

Usage (for future graceful shutdown):
  iamServer, _ := iamapi.NewIamApiServer(...)
  defer iamServer.Shutdown()

  // or in signal handler:
  sigChan := make(chan os.Signal, 1)
  signal.Notify(sigChan, syscall.SIGTERM, syscall.SIGINT)
  go func() {
      <-sigChan
      iamServer.Shutdown()
      os.Exit(0)
  }()

Note: Current command implementations (weed/command/iam.go) don't have
shutdown paths yet, but this makes IAM server ready for proper lifecycle
management when that infrastructure is added.

* refactor: remove unnecessary KeepMasterClientConnected wrapper in filer

The Filer.KeepMasterClientConnected() method was an unnecessary wrapper that
just forwarded to MasterClient.KeepConnectedToMaster(). This wrapper added
no value and created inconsistency with other components that call
KeepConnectedToMaster directly.

Removed:
  filer.go:178-180
    func (fs *Filer) KeepMasterClientConnected(ctx context.Context) {
        fs.MasterClient.KeepConnectedToMaster(ctx)
    }

Updated caller:
  filer_server.go:181
    - go fs.filer.KeepMasterClientConnected(context.Background())
    + go fs.filer.MasterClient.KeepConnectedToMaster(context.Background())

Benefits:
  - Consistent with other components (S3, IAM, Shell, Mount)
  - Removes unnecessary indirection
  - Clearer that KeepConnectedToMaster runs in background goroutine
  - Follows the documented pattern from MasterClient.GetMaster()

Note: shell/commands.go was verified and already correctly starts
KeepConnectedToMaster in a background goroutine (shell_liner.go:51):
  go commandEnv.MasterClient.KeepConnectedToMaster(ctx)

* fix: use client ID instead of timeout for gRPC signature parameter

The pb.WithGrpcFilerClient signature parameter is meant to be a client
identifier for logging and tracking (added as 'sw-client-id' gRPC metadata
in streaming mode), not a timeout value.

Problem:
  timeoutMs := int32(fc.grpcTimeout.Milliseconds())  // 5000 (5 seconds)
  err := pb.WithGrpcFilerClient(false, timeoutMs, filerAddress, ...)

  - Passing timeout (5000ms) as signature/client ID
  - Misuse of API: signature should be a unique client identifier
  - Timeout is already handled by timeoutCtx passed to gRPC call
  - Inconsistent with other callers (all use 0 or proper client ID)

How WithGrpcFilerClient uses signature parameter:
  func WithGrpcClient(..., signature int32, ...) {
    if streamingMode && signature != 0 {
      md := metadata.New(map[string]string{"sw-client-id": fmt.Sprintf("%d", signature)})
      ctx = metadata.NewOutgoingContext(ctx, md)
    }
    ...
  }

It's for client identification, not timeout control!

Fix:
  1. Added clientId int32 field to FilerClient struct
  2. Initialize with rand.Int31() in NewFilerClient for unique ID
  3. Removed timeoutMs variable (and misleading comment)
  4. Use fc.clientId in pb.WithGrpcFilerClient call

Before:
  err := pb.WithGrpcFilerClient(false, timeoutMs, ...)
                                      ^^^^^^^^^ Wrong! (5000)

After:
  err := pb.WithGrpcFilerClient(false, fc.clientId, ...)
                                      ^^^^^^^^^^^^ Correct! (random int31)

Benefits:
  - Correct API usage (signature = client ID, not timeout)
  - Timeout still works via timeoutCtx (unchanged)
  - Consistent with other pb.WithGrpcFilerClient callers
  - Enables proper client tracking on filer side via gRPC metadata
  - Each FilerClient instance has unique ID for debugging

Examples of correct usage elsewhere:
  weed/iamapi/iamapi_server.go:145     pb.WithGrpcFilerClient(false, 0, ...)
  weed/command/s3.go:215               pb.WithGrpcFilerClient(false, 0, ...)
  weed/shell/commands.go:110           pb.WithGrpcFilerClient(streamingMode, 0, ...)

All use 0 (or a proper signature), not a timeout value.

* fix: add timeout to master volume lookup to prevent indefinite blocking

The masterVolumeProvider.LookupVolumeIds method was using the context
directly without a timeout, which could cause it to block indefinitely
if the master is slow to respond or unreachable.

Problem:
  err := pb.WithMasterClient(false, p.masterClient.GetMaster(ctx), ...)
  resp, err := client.LookupVolume(ctx, &master_pb.LookupVolumeRequest{...})

  - No timeout on gRPC call to master
  - Could block indefinitely if master is unresponsive
  - Inconsistent with FilerClient which uses 5s timeout
  - This is a fallback path (cache miss) but still needs protection

Scenarios where this could hang:
  1. Master server under heavy load (slow response)
  2. Network issues between client and master
  3. Master server hung or deadlocked
  4. Master in process of shutting down

Fix:
  timeoutCtx, cancel := context.WithTimeout(ctx, 5*time.Second)
  defer cancel()

  err := pb.WithMasterClient(false, p.masterClient.GetMaster(timeoutCtx), ...)
  resp, err := client.LookupVolume(timeoutCtx, &master_pb.LookupVolumeRequest{...})

Benefits:
  - Prevents indefinite blocking on master lookup
  - Consistent with FilerClient timeout pattern (5 seconds)
  - Faster failure detection when master is unresponsive
  - Caller's context still honored (timeout is in addition, not replacement)
  - Improves overall system resilience

Note: 5 seconds is a reasonable default for volume lookups:
  - Long enough for normal master response (~10-50ms)
  - Short enough to fail fast on issues
  - Matches FilerClient's grpcTimeout default

* purge

* refactor: address code review feedback on comments and style

Fixed several code quality issues identified during review:

1. Corrected backoff algorithm description in filer_client.go:
   - Changed "Exponential backoff" to "Multiplicative backoff with 1.5x factor"
   - The formula waitTime * 3/2 produces 1s, 1.5s, 2.25s, not exponential 2^n
   - More accurate terminology prevents confusion

2. Removed redundant nil check in vidmap_client.go:
   - After the for loop, node is guaranteed to be non-nil
   - Loop either returns early or assigns non-nil value to node
   - Simplified: if node != nil { node.cache.Store(nil) } → node.cache.Store(nil)

3. Added startup logging to IAM server for consistency:
   - Log when master client connection starts
   - Matches pattern in S3ApiServer (line 100 in s3api_server.go)
   - Improves operational visibility during startup
   - Added missing glog import

4. Fixed indentation in filer/reader_at.go:
   - Lines 76-91 had incorrect indentation (extra tab level)
   - Line 93 also misaligned
   - Now properly aligned with surrounding code

5. Updated deprecation comment to follow Go convention:
   - Changed "DEPRECATED:" to "Deprecated:" (standard Go format)
   - Tools like staticcheck and IDEs recognize the standard format
   - Enables automated deprecation warnings in tooling
   - Better developer experience

All changes are cosmetic and do not affect functionality.

* fmt

* refactor: make circuit breaker parameters configurable in FilerClient

The circuit breaker failure threshold (3) and reset timeout (30s) were
hardcoded, making it difficult to tune the client's behavior in different
deployment environments without modifying the code.

Problem:
  func shouldSkipUnhealthyFiler(index int32) bool {
    if failureCount < 3 {              // Hardcoded threshold
      return false
    }
    if time.Since(lastFailureTime) > 30*time.Second {  // Hardcoded timeout
      return false
    }
  }

Different environments have different needs:
  - High-traffic production: may want lower threshold (2) for faster failover
  - Development/testing: may want higher threshold (5) to tolerate flaky networks
  - Low-latency services: may want shorter reset timeout (10s)
  - Batch processing: may want longer reset timeout (60s)

Solution:
  1. Added fields to FilerClientOption:
     - FailureThreshold int32 (default: 3)
     - ResetTimeout time.Duration (default: 30s)

  2. Added fields to FilerClient:
     - failureThreshold int32
     - resetTimeout time.Duration

  3. Applied defaults in NewFilerClient with option override:
     failureThreshold := int32(3)
     resetTimeout := 30 * time.Second
     if opt.FailureThreshold > 0 {
       failureThreshold = opt.FailureThreshold
     }
     if opt.ResetTimeout > 0 {
       resetTimeout = opt.ResetTimeout
     }

  4. Updated shouldSkipUnhealthyFiler to use configurable values:
     if failureCount < fc.failureThreshold { ... }
     if time.Since(lastFailureTime) > fc.resetTimeout { ... }

Benefits:
  ✓ Tunable for different deployment environments
  ✓ Backward compatible (defaults match previous hardcoded values)
  ✓ No breaking changes to existing code
  ✓ Better maintainability and flexibility

Example usage:
  // Aggressive failover for low-latency production
  fc := wdclient.NewFilerClient(filers, dialOpt, dc, &wdclient.FilerClientOption{
    FailureThreshold: 2,
    ResetTimeout:     10 * time.Second,
  })

  // Tolerant of flaky networks in development
  fc := wdclient.NewFilerClient(filers, dialOpt, dc, &wdclient.FilerClientOption{
    FailureThreshold: 5,
    ResetTimeout:     60 * time.Second,
  })

* retry parameters

* refactor: make retry and timeout parameters configurable

Made retry logic and gRPC timeouts configurable across FilerClient and
MasterClient to support different deployment environments and network
conditions.

Problem 1: Hardcoded retry parameters in FilerClient
  waitTime := time.Second          // Fixed at 1s
  maxRetries := 3                  // Fixed at 3 attempts
  waitTime = waitTime * 3 / 2      // Fixed 1.5x multiplier

Different environments have different needs:
  - Unstable networks: may want more retries (5) with longer waits (2s)
  - Low-latency production: may want fewer retries (2) with shorter waits (500ms)
  - Batch processing: may want exponential backoff (2x) instead of 1.5x

Problem 2: Hardcoded gRPC timeout in MasterClient
  timeoutCtx, cancel := context.WithTimeout(ctx, 5*time.Second)

Master lookups may need different timeouts:
  - High-latency cross-region: may need 10s timeout
  - Local network: may use 2s timeout for faster failure detection

Solution for FilerClient:
  1. Added fields to FilerClientOption:
     - MaxRetries int (default: 3)
     - InitialRetryWait time.Duration (default: 1s)
     - RetryBackoffFactor float64 (default: 1.5)

  2. Added fields to FilerClient:
     - maxRetries int
     - initialRetryWait time.Duration
     - retryBackoffFactor float64

  3. Updated LookupVolumeIds to use configurable values:
     waitTime := fc.initialRetryWait
     maxRetries := fc.maxRetries
     for retry := 0; retry < maxRetries; retry++ {
       ...
       waitTime = time.Duration(float64(waitTime) * fc.retryBackoffFactor)
     }

Solution for MasterClient:
  1. Added grpcTimeout field to MasterClient (default: 5s)
  2. Initialize in NewMasterClient with 5 * time.Second default
  3. Updated masterVolumeProvider to use p.masterClient.grpcTimeout

Benefits:
  ✓ Tunable for different network conditions and deployment scenarios
  ✓ Backward compatible (defaults match previous hardcoded values)
  ✓ No breaking changes to existing code
  ✓ Consistent configuration pattern across FilerClient and MasterClient

Example usage:
  // Fast-fail for low-latency production with stable network
  fc := wdclient.NewFilerClient(filers, dialOpt, dc, &wdclient.FilerClientOption{
    MaxRetries:         2,
    InitialRetryWait:   500 * time.Millisecond,
    RetryBackoffFactor: 2.0,  // Exponential backoff
    GrpcTimeout:        2 * time.Second,
  })

  // Patient retries for unstable network or batch processing
  fc := wdclient.NewFilerClient(filers, dialOpt, dc, &wdclient.FilerClientOption{
    MaxRetries:         5,
    InitialRetryWait:   2 * time.Second,
    RetryBackoffFactor: 1.5,
    GrpcTimeout:        10 * time.Second,
  })

Note: MasterClient timeout is currently set at construction time and not
user-configurable via NewMasterClient parameters. Future enhancement could
add a MasterClientOption struct similar to FilerClientOption.

* fix: rename vicCacheLock to vidCacheLock for consistency

Fixed typo in variable name for better code consistency and readability.

Problem:
  vidCache := make(map[string]*filer_pb.Locations)
  var vicCacheLock sync.RWMutex  // Typo: vic instead of vid

  vicCacheLock.RLock()
  locations, found := vidCache[vid]
  vicCacheLock.RUnlock()

The variable name 'vicCacheLock' is inconsistent with 'vidCache'.
Both should use 'vid' prefix (volume ID) not 'vic'.

Fix:
  Renamed all 5 occurrences:
  - var vicCacheLock → var vidCacheLock (line 56)
  - vicCacheLock.RLock() → vidCacheLock.RLock() (line 62)
  - vicCacheLock.RUnlock() → vidCacheLock.RUnlock() (line 64)
  - vicCacheLock.Lock() → vidCacheLock.Lock() (line 81)
  - vicCacheLock.Unlock() → vidCacheLock.Unlock() (line 91)

Benefits:
  ✓ Consistent variable naming convention
  ✓ Clearer intent (volume ID cache lock)
  ✓ Better code readability
  ✓ Easier code navigation

* fix: use defer cancel() with anonymous function for proper context cleanup

Fixed context cancellation to use defer pattern correctly in loop iteration.

Problem:
  for x := 0; x < n; x++ {
    timeoutCtx, cancel := context.WithTimeout(ctx, fc.grpcTimeout)
    err := pb.WithGrpcFilerClient(...)
    cancel() // Only called on normal return, not on panic
  }

Issues with original approach:
  1. If pb.WithGrpcFilerClient panics, cancel() is never called → context leak
  2. If callback returns early (though unlikely here), cleanup might be missed
  3. Not following Go best practices for context.WithTimeout usage

Problem with naive defer in loop:
  for x := 0; x < n; x++ {
    timeoutCtx, cancel := context.WithTimeout(ctx, fc.grpcTimeout)
    defer cancel() //  WRONG: All defers accumulate until function returns
  }

In Go, defer executes when the surrounding *function* returns, not when
the loop iteration ends. This would accumulate n deferred cancel() calls
and leak contexts until LookupVolumeIds returns.

Solution: Wrap in anonymous function
  for x := 0; x < n; x++ {
    err := func() error {
      timeoutCtx, cancel := context.WithTimeout(ctx, fc.grpcTimeout)
      defer cancel() //  Executes when anonymous function returns (per iteration)
      return pb.WithGrpcFilerClient(...)
    }()
  }

Benefits:
  ✓ Context always cancelled, even on panic
  ✓ defer executes after each iteration (not accumulated)
  ✓ Follows Go best practices for context.WithTimeout
  ✓ No resource leaks during retry loop execution
  ✓ Cleaner error handling

Reference:
  Go documentation for context.WithTimeout explicitly shows:
    ctx, cancel := context.WithTimeout(...)
    defer cancel()

This is the idiomatic pattern that should always be followed.

* Can't use defer directly in loop

* improve: add data center preference and URL shuffling for consistent performance

Added missing data center preference and load distribution (URL shuffling)
to ensure consistent performance and behavior across all code paths.

Problem 1: PreferPublicUrl path missing DC preference and shuffling
Location: weed/wdclient/filer_client.go lines 184-192

The custom PreferPublicUrl implementation was simply iterating through
locations and building URLs without considering:
  1. Data center proximity (latency optimization)
  2. Load distribution across volume servers

Before:
  for _, loc := range locations {
    url := loc.PublicUrl
    if url == "" { url = loc.Url }
    fullUrls = append(fullUrls, "http://"+url+"/"+fileId)
  }
  return fullUrls, nil

After:
  var sameDcUrls, otherDcUrls []string
  dataCenter := fc.GetDataCenter()
  for _, loc := range locations {
    url := loc.PublicUrl
    if url == "" { url = loc.Url }
    httpUrl := "http://" + url + "/" + fileId
    if dataCenter != "" && dataCenter == loc.DataCenter {
      sameDcUrls = append(sameDcUrls, httpUrl)
    } else {
      otherDcUrls = append(otherDcUrls, httpUrl)
    }
  }
  rand.Shuffle(len(sameDcUrls), ...)
  rand.Shuffle(len(otherDcUrls), ...)
  fullUrls = append(sameDcUrls, otherDcUrls...)

Problem 2: Cache miss path missing URL shuffling
Location: weed/wdclient/vidmap_client.go lines 95-108

The cache miss path (fallback lookup) was missing URL shuffling, while
the cache hit path (vm.LookupFileId) already shuffles URLs. This
inconsistency meant:
  - Cache hit: URLs shuffled → load distributed
  - Cache miss: URLs not shuffled → first server always hit

Before:
  var sameDcUrls, otherDcUrls []string
  // ... build URLs ...
  fullUrls = append(sameDcUrls, otherDcUrls...)
  return fullUrls, nil

After:
  var sameDcUrls, otherDcUrls []string
  // ... build URLs ...
  rand.Shuffle(len(sameDcUrls), ...)
  rand.Shuffle(len(otherDcUrls), ...)
  fullUrls = append(sameDcUrls, otherDcUrls...)
  return fullUrls, nil

Benefits:
  ✓ Reduced latency by preferring same-DC volume servers
  ✓ Even load distribution across all volume servers
  ✓ Consistent behavior between cache hit/miss paths
  ✓ Consistent behavior between PreferUrl and PreferPublicUrl
  ✓ Matches behavior of existing vidMap.LookupFileId implementation

Impact on performance:
  - Lower read latency (same-DC preference)
  - Better volume server utilization (load spreading)
  - No single volume server becomes a hotspot

Note: Added math/rand import to vidmap_client.go for shuffle support.

* Update weed/wdclient/masterclient.go

Co-authored-by: Copilot <175728472+Copilot@users.noreply.github.com>

* improve: call IAM server Shutdown() for best-effort cleanup

Added call to iamApiServer.Shutdown() to ensure cleanup happens when possible,
and documented the limitations of the current approach.

Problem:
  The Shutdown() method was defined in IamApiServer but never called anywhere,
  meaning the KeepConnectedToMaster goroutine would continue running even when
  the IAM server stopped, causing resource leaks.

Changes:
  1. Store iamApiServer instance in weed/command/iam.go
     - Changed: _, iamApiServer_err := iamapi.NewIamApiServer(...)
     - To: iamApiServer, iamApiServer_err := iamapi.NewIamApiServer(...)

  2. Added defer call for best-effort cleanup
     - defer iamApiServer.Shutdown()
     - This will execute if startIamServer() returns normally

  3. Added logging in Shutdown() method
     - Log when shutdown is triggered for visibility

  4. Documented limitations and future improvements
     - Added note that defer only works for normal function returns
     - SeaweedFS commands don't currently have signal handling
     - Suggested future enhancement: add SIGTERM/SIGINT handling

Current behavior:
  - ✓ Cleanup happens if HTTP server fails to start (glog.Fatalf path)
  - ✓ Cleanup happens if Serve() returns with error (unlikely)
  - ✗ Cleanup does NOT happen on SIGTERM/SIGINT (process killed)

The last case is a limitation of the current command architecture - all
SeaweedFS commands (s3, filer, volume, master, iam) lack signal handling
for graceful shutdown. This is a systemic issue that affects all services.

Future enhancement:
  To properly handle SIGTERM/SIGINT, the command layer would need:

    sigChan := make(chan os.Signal, 1)
    signal.Notify(sigChan, syscall.SIGTERM, syscall.SIGINT)

    go func() {
      httpServer.Serve(listener) // Non-blocking
    }()

    <-sigChan
    glog.V(0).Infof("Received shutdown signal")
    iamApiServer.Shutdown()
    httpServer.Shutdown(context.Background())

This would require refactoring the command structure for all services,
which is out of scope for this change.

Benefits of current approach:
  ✓ Best-effort cleanup (better than nothing)
  ✓ Proper cleanup in error paths
  ✓ Documented for future improvement
  ✓ Consistent with how other SeaweedFS services handle lifecycle

* data racing in test

---------

Co-authored-by: Copilot <175728472+Copilot@users.noreply.github.com>
This commit is contained in:
Chris Lu
2025-11-20 20:50:26 -08:00
committed by GitHub
parent 5f77f87335
commit c1b8d4bf0d
13 changed files with 1082 additions and 436 deletions

View File

@@ -705,13 +705,23 @@ func (f *S3IAMTestFramework) CreateBucketWithCleanup(s3Client *s3.S3, bucketName
f.t.Logf("Warning: Failed to delete existing bucket %s: %v", bucketName, deleteErr)
}
// Add a small delay to allow deletion to propagate
time.Sleep(100 * time.Millisecond)
// Now create it fresh
_, err = s3Client.CreateBucket(&s3.CreateBucketInput{
Bucket: aws.String(bucketName),
})
if err != nil {
// If it still says bucket exists after cleanup, it might be in an inconsistent state
// In this case, just use the existing bucket since we emptied it
if awsErr, ok := err.(awserr.Error); ok && (awsErr.Code() == "BucketAlreadyExists" || awsErr.Code() == "BucketAlreadyOwnedByYou") {
f.t.Logf("Bucket %s still exists after cleanup, reusing it", bucketName)
// Bucket exists and is empty, so we can proceed
} else {
return fmt.Errorf("failed to recreate bucket after cleanup: %v", err)
}
}
} else {
return err
}

View File

@@ -1,10 +1,11 @@
package cluster
import (
"github.com/seaweedfs/seaweedfs/weed/pb"
"github.com/seaweedfs/seaweedfs/weed/pb/master_pb"
"sync"
"time"
"github.com/seaweedfs/seaweedfs/weed/pb"
"github.com/seaweedfs/seaweedfs/weed/pb/master_pb"
)
const (

View File

@@ -76,7 +76,7 @@ func (iamopt *IamOptions) startIamServer() bool {
masters := pb.ServerAddresses(*iamopt.masters).ToAddressMap()
router := mux.NewRouter().SkipClean(true)
_, iamApiServer_err := iamapi.NewIamApiServer(router, &iamapi.IamServerOption{
iamApiServer, iamApiServer_err := iamapi.NewIamApiServer(router, &iamapi.IamServerOption{
Masters: masters,
Filer: filerAddress,
Port: *iamopt.port,
@@ -87,6 +87,9 @@ func (iamopt *IamOptions) startIamServer() bool {
glog.Fatalf("IAM API Server startup error: %v", iamApiServer_err)
}
// Ensure cleanup on shutdown
defer iamApiServer.Shutdown()
listenAddress := fmt.Sprintf(":%d", *iamopt.port)
iamApiListener, iamApiLocalListener, err := util.NewIpAndLocalListeners(*iamopt.ip, *iamopt.port, time.Duration(10)*time.Second)
if err != nil {

View File

@@ -175,10 +175,6 @@ func (fs *Filer) GetMaster(ctx context.Context) pb.ServerAddress {
return fs.MasterClient.GetMaster(ctx)
}
func (fs *Filer) KeepMasterClientConnected(ctx context.Context) {
fs.MasterClient.KeepConnectedToMaster(ctx)
}
func (f *Filer) BeginTransaction(ctx context.Context) (context.Context, error) {
return f.Store.BeginTransaction(ctx)
}

View File

@@ -26,15 +26,42 @@ type ChunkReadAt struct {
var _ = io.ReaderAt(&ChunkReadAt{})
var _ = io.Closer(&ChunkReadAt{})
// LookupFn creates a basic volume location lookup function with simple caching.
//
// Deprecated: Use wdclient.FilerClient instead. This function has several limitations compared to wdclient.FilerClient:
// - Simple bounded cache (10k entries, no eviction policy or TTL for stale entries)
// - No singleflight deduplication (concurrent requests for same volume will duplicate work)
// - No cache history for volume moves (no fallback chain when volumes migrate)
// - No high availability (single filer address, no automatic failover)
//
// For NEW code, especially mount operations, use wdclient.FilerClient instead:
// filerClient := wdclient.NewFilerClient(filerAddresses, grpcDialOption, dataCenter, opts)
// lookupFn := filerClient.GetLookupFileIdFunction()
//
// This provides:
// - Bounded cache with configurable size
// - Singleflight deduplication of concurrent lookups
// - Cache history when volumes move
// - Battle-tested vidMap with cache chain
//
// This function is kept for backward compatibility with existing code paths
// (shell commands, streaming, etc.) but should be avoided in long-running processes
// or multi-tenant deployments where unbounded memory growth is a concern.
//
// Maximum recommended cache entries: ~10,000 volumes per process.
// Beyond this, consider migrating to wdclient.FilerClient.
func LookupFn(filerClient filer_pb.FilerClient) wdclient.LookupFileIdFunctionType {
vidCache := make(map[string]*filer_pb.Locations)
var vicCacheLock sync.RWMutex
var vidCacheLock sync.RWMutex
cacheSize := 0
const maxCacheSize = 10000 // Simple bound to prevent unbounded growth
return func(ctx context.Context, fileId string) (targetUrls []string, err error) {
vid := VolumeId(fileId)
vicCacheLock.RLock()
vidCacheLock.RLock()
locations, found := vidCache[vid]
vicCacheLock.RUnlock()
vidCacheLock.RUnlock()
if !found {
util.Retry("lookup volume "+vid, func() error {
@@ -51,9 +78,17 @@ func LookupFn(filerClient filer_pb.FilerClient) wdclient.LookupFileIdFunctionTyp
glog.V(0).InfofCtx(ctx, "failed to locate %s", fileId)
return fmt.Errorf("failed to locate %s", fileId)
}
vicCacheLock.Lock()
vidCacheLock.Lock()
// Simple size limit to prevent unbounded growth
// For proper cache management, use wdclient.FilerClient instead
if cacheSize < maxCacheSize {
vidCache[vid] = locations
vicCacheLock.Unlock()
cacheSize++
} else if cacheSize == maxCacheSize {
glog.Warningf("filer.LookupFn cache reached limit of %d volumes, not caching new entries. Consider migrating to wdclient.FilerClient for bounded cache management.", maxCacheSize)
cacheSize++ // Only log once
}
vidCacheLock.Unlock()
return nil
})

View File

@@ -12,6 +12,7 @@ import (
"github.com/gorilla/mux"
"github.com/seaweedfs/seaweedfs/weed/credential"
"github.com/seaweedfs/seaweedfs/weed/filer"
"github.com/seaweedfs/seaweedfs/weed/glog"
"github.com/seaweedfs/seaweedfs/weed/pb"
"github.com/seaweedfs/seaweedfs/weed/pb/filer_pb"
"github.com/seaweedfs/seaweedfs/weed/pb/iam_pb"
@@ -47,6 +48,9 @@ type IamServerOption struct {
type IamApiServer struct {
s3ApiConfig IamS3ApiConfig
iam *s3api.IdentityAccessManagement
shutdownContext context.Context
shutdownCancel context.CancelFunc
masterClient *wdclient.MasterClient
}
var s3ApiConfigure IamS3ApiConfig
@@ -56,9 +60,21 @@ func NewIamApiServer(router *mux.Router, option *IamServerOption) (iamApiServer
}
func NewIamApiServerWithStore(router *mux.Router, option *IamServerOption, explicitStore string) (iamApiServer *IamApiServer, err error) {
masterClient := wdclient.NewMasterClient(option.GrpcDialOption, "", "iam", "", "", "", *pb.NewServiceDiscoveryFromMap(option.Masters))
// Create a cancellable context for the master client connection
// This allows graceful shutdown via Shutdown() method
shutdownCtx, shutdownCancel := context.WithCancel(context.Background())
// Start KeepConnectedToMaster for volume location lookups
// IAM config files are typically small and inline, but if they ever have chunks,
// ReadEntry→StreamContent needs masterClient for volume lookups
glog.V(0).Infof("IAM API starting master client connection for volume location lookups")
go masterClient.KeepConnectedToMaster(shutdownCtx)
configure := &IamS3ApiConfigure{
option: option,
masterClient: wdclient.NewMasterClient(option.GrpcDialOption, "", "iam", "", "", "", *pb.NewServiceDiscoveryFromMap(option.Masters)),
masterClient: masterClient,
}
s3ApiConfigure = configure
@@ -74,6 +90,9 @@ func NewIamApiServerWithStore(router *mux.Router, option *IamServerOption, expli
iamApiServer = &IamApiServer{
s3ApiConfig: s3ApiConfigure,
iam: iam,
shutdownContext: shutdownCtx,
shutdownCancel: shutdownCancel,
masterClient: masterClient,
}
iamApiServer.registerRouter(router)
@@ -93,6 +112,20 @@ func (iama *IamApiServer) registerRouter(router *mux.Router) {
apiRouter.NotFoundHandler = http.HandlerFunc(s3err.NotFoundHandler)
}
// Shutdown gracefully stops the IAM API server and releases resources.
// It cancels the master client connection goroutine and closes gRPC connections.
// This method is safe to call multiple times.
//
// Note: This method is called via defer in weed/command/iam.go for best-effort cleanup.
// For proper graceful shutdown on SIGTERM/SIGINT, signal handling should be added to
// the command layer to call this method before process exit.
func (iama *IamApiServer) Shutdown() {
if iama.shutdownCancel != nil {
glog.V(0).Infof("IAM API server shutting down, stopping master client connection")
iama.shutdownCancel()
}
}
func (iama *IamS3ApiConfigure) GetS3ApiConfiguration(s3cfg *iam_pb.S3ApiConfiguration) (err error) {
return iama.GetS3ApiConfigurationFromCredentialManager(s3cfg)
}

View File

@@ -97,9 +97,32 @@ type WFS struct {
fhLockTable *util.LockTable[FileHandleId]
rdmaClient *RDMAMountClient
FilerConf *filer.FilerConf
filerClient *wdclient.FilerClient // Cached volume location client
}
func NewSeaweedFileSystem(option *Option) *WFS {
// Only create FilerClient for direct volume access modes
// When VolumeServerAccess == "filerProxy", all reads go through filer, so no volume lookup needed
var filerClient *wdclient.FilerClient
if option.VolumeServerAccess != "filerProxy" {
// Create FilerClient for efficient volume location caching
// Pass all filer addresses for high availability with automatic failover
// Configure URL preference based on VolumeServerAccess option
var opts *wdclient.FilerClientOption
if option.VolumeServerAccess == "publicUrl" {
opts = &wdclient.FilerClientOption{
UrlPreference: wdclient.PreferPublicUrl,
}
}
filerClient = wdclient.NewFilerClient(
option.FilerAddresses, // Pass all filer addresses for HA
option.GrpcDialOption,
option.DataCenter,
opts,
)
}
wfs := &WFS{
RawFileSystem: fuse.NewDefaultRawFileSystem(),
option: option,
@@ -107,6 +130,7 @@ func NewSeaweedFileSystem(option *Option) *WFS {
inodeToPath: NewInodeToPath(util.FullPath(option.FilerMountRootPath), option.CacheMetaTTlSec),
fhMap: NewFileHandleToInode(),
dhMap: NewDirectoryHandleToInode(),
filerClient: filerClient, // nil for proxy mode, initialized for direct access
fhLockTable: util.NewLockTable[FileHandleId](),
}
@@ -253,7 +277,8 @@ func (wfs *WFS) LookupFn() wdclient.LookupFileIdFunctionType {
return []string{"http://" + wfs.getCurrentFiler().ToHttpAddress() + "/?proxyChunkId=" + fileId}, nil
}
}
return filer.LookupFn(wfs)
// Use the cached FilerClient for efficient lookups with singleflight and cache history
return wfs.filerClient.GetLookupFileIdFunction()
}
func (wfs *WFS) getCurrentFiler() pb.ServerAddress {

View File

@@ -24,7 +24,6 @@ import (
"github.com/seaweedfs/seaweedfs/weed/s3api/s3_constants"
"github.com/seaweedfs/seaweedfs/weed/s3api/s3err"
util_http "github.com/seaweedfs/seaweedfs/weed/util/http"
"github.com/seaweedfs/seaweedfs/weed/util/mem"
"github.com/seaweedfs/seaweedfs/weed/glog"
@@ -994,36 +993,10 @@ var volumeServerHTTPClient = &http.Client{
}
// createLookupFileIdFunction creates a reusable lookup function for resolving volume URLs
// Uses FilerClient's vidMap cache to eliminate per-chunk gRPC overhead
func (s3a *S3ApiServer) createLookupFileIdFunction() func(context.Context, string) ([]string, error) {
return func(ctx context.Context, fileId string) ([]string, error) {
var urls []string
err := s3a.WithFilerClient(false, func(client filer_pb.SeaweedFilerClient) error {
vid := filer.VolumeId(fileId)
resp, err := client.LookupVolume(ctx, &filer_pb.LookupVolumeRequest{
VolumeIds: []string{vid},
})
if err != nil {
return err
}
if locs, found := resp.LocationsMap[vid]; found {
for _, loc := range locs.Locations {
// Build complete URL with volume server address and fileId
// The fileId parameter contains the full "volumeId,fileKey" identifier (e.g., "3,01637037d6")
// This constructs URLs like: http://127.0.0.1:8080/3,01637037d6 (or https:// if configured)
// NormalizeUrl ensures the proper scheme (http:// or https://) is used based on configuration
normalizedUrl, err := util_http.NormalizeUrl(loc.Url)
if err != nil {
glog.Warningf("Failed to normalize URL for %s: %v", loc.Url, err)
continue
}
urls = append(urls, normalizedUrl+"/"+fileId)
}
}
return nil
})
glog.V(3).Infof("createLookupFileIdFunction: fileId=%s, resolved urls=%v", fileId, urls)
return urls, err
}
// Return the FilerClient's lookup function which uses the battle-tested vidMap cache
return s3a.filerClient.GetLookupFileIdFunction()
}
// streamFromVolumeServersWithSSE handles streaming with inline SSE decryption

View File

@@ -19,6 +19,7 @@ import (
"github.com/seaweedfs/seaweedfs/weed/iam/sts"
"github.com/seaweedfs/seaweedfs/weed/pb/s3_pb"
"github.com/seaweedfs/seaweedfs/weed/util/grace"
"github.com/seaweedfs/seaweedfs/weed/wdclient"
"github.com/gorilla/mux"
"github.com/seaweedfs/seaweedfs/weed/pb"
@@ -55,6 +56,7 @@ type S3ApiServer struct {
cb *CircuitBreaker
randomClientId int32
filerGuard *security.Guard
filerClient *wdclient.FilerClient
client util_http_client.HTTPClientInterface
bucketRegistry *BucketRegistry
credentialManager *credential.CredentialManager
@@ -91,11 +93,18 @@ func NewS3ApiServerWithStore(router *mux.Router, option *S3ApiServerOption, expl
// Initialize bucket policy engine first
policyEngine := NewBucketPolicyEngine()
// Initialize FilerClient for volume location caching
// Uses the battle-tested vidMap with filer-based lookups
// S3 API typically connects to a single filer, but wrap in slice for consistency
filerClient := wdclient.NewFilerClient([]pb.ServerAddress{option.Filer}, option.GrpcDialOption, option.DataCenter)
glog.V(0).Infof("S3 API initialized FilerClient for volume location caching")
s3ApiServer = &S3ApiServer{
option: option,
iam: iam,
randomClientId: util.RandomInt32(),
filerGuard: security.NewGuard([]string{}, signingKey, expiresAfterSec, readSigningKey, readExpiresAfterSec),
filerClient: filerClient,
cb: NewCircuitBreaker(option),
credentialManager: iam.credentialManager,
bucketConfigCache: NewBucketConfigCache(60 * time.Minute), // Increased TTL since cache is now event-driven

View File

@@ -178,7 +178,7 @@ func NewFilerServer(defaultMux, readonlyMux *http.ServeMux, option *FilerOption)
fs.checkWithMaster()
go stats.LoopPushingMetric("filer", string(fs.option.Host), fs.metricsAddress, fs.metricsIntervalSec)
go fs.filer.KeepMasterClientConnected(context.Background())
go fs.filer.MasterClient.KeepConnectedToMaster(context.Background())
fs.option.recursiveDelete = v.GetBool("filer.options.recursive_delete")
v.SetDefault("filer.options.buckets_folder", "/buckets")

View File

@@ -0,0 +1,404 @@
package wdclient
import (
"context"
"fmt"
"math/rand"
"strings"
"sync/atomic"
"time"
"google.golang.org/grpc"
"google.golang.org/grpc/codes"
"google.golang.org/grpc/status"
"github.com/seaweedfs/seaweedfs/weed/glog"
"github.com/seaweedfs/seaweedfs/weed/pb"
"github.com/seaweedfs/seaweedfs/weed/pb/filer_pb"
)
// UrlPreference controls which URL to use for volume access
type UrlPreference string
const (
PreferUrl UrlPreference = "url" // Use private URL (default)
PreferPublicUrl UrlPreference = "publicUrl" // Use public URL
)
// filerHealth tracks the health status of a filer
type filerHealth struct {
failureCount int32 // atomic: consecutive failures
lastFailureTimeNs int64 // atomic: last failure time in Unix nanoseconds
}
// FilerClient provides volume location services by querying a filer
// It uses the shared vidMap cache for efficient lookups
// Supports multiple filer addresses with automatic failover for high availability
// Tracks filer health to avoid repeatedly trying known-unhealthy filers
type FilerClient struct {
*vidMapClient
filerAddresses []pb.ServerAddress
filerIndex int32 // atomic: current filer index for round-robin
filerHealth []*filerHealth // health status per filer (same order as filerAddresses)
grpcDialOption grpc.DialOption
urlPreference UrlPreference
grpcTimeout time.Duration
cacheSize int // Number of historical vidMap snapshots to keep
clientId int32 // Unique client identifier for gRPC metadata
failureThreshold int32 // Circuit breaker: consecutive failures before circuit opens
resetTimeout time.Duration // Circuit breaker: time before re-checking unhealthy filer
maxRetries int // Retry: maximum retry attempts for transient failures
initialRetryWait time.Duration // Retry: initial wait time before first retry
retryBackoffFactor float64 // Retry: backoff multiplier for wait time
}
// filerVolumeProvider implements VolumeLocationProvider by querying filer
// Supports multiple filer addresses with automatic failover
type filerVolumeProvider struct {
filerClient *FilerClient
}
// FilerClientOption holds optional configuration for FilerClient
type FilerClientOption struct {
GrpcTimeout time.Duration
UrlPreference UrlPreference
CacheSize int // Number of historical vidMap snapshots (0 = use default)
FailureThreshold int32 // Circuit breaker: consecutive failures before skipping filer (0 = use default of 3)
ResetTimeout time.Duration // Circuit breaker: time before re-checking unhealthy filer (0 = use default of 30s)
MaxRetries int // Retry: maximum retry attempts for transient failures (0 = use default of 3)
InitialRetryWait time.Duration // Retry: initial wait time before first retry (0 = use default of 1s)
RetryBackoffFactor float64 // Retry: backoff multiplier for wait time (0 = use default of 1.5)
}
// NewFilerClient creates a new client that queries filer(s) for volume locations
// Supports multiple filer addresses for high availability with automatic failover
// Uses sensible defaults: 5-second gRPC timeout, PreferUrl, DefaultVidMapCacheSize
func NewFilerClient(filerAddresses []pb.ServerAddress, grpcDialOption grpc.DialOption, dataCenter string, opts ...*FilerClientOption) *FilerClient {
if len(filerAddresses) == 0 {
glog.Fatal("NewFilerClient requires at least one filer address")
}
// Apply defaults
grpcTimeout := 5 * time.Second
urlPref := PreferUrl
cacheSize := DefaultVidMapCacheSize
failureThreshold := int32(3) // Default: 3 consecutive failures before circuit opens
resetTimeout := 30 * time.Second // Default: 30 seconds before re-checking unhealthy filer
maxRetries := 3 // Default: 3 retry attempts for transient failures
initialRetryWait := time.Second // Default: 1 second initial retry wait
retryBackoffFactor := 1.5 // Default: 1.5x backoff multiplier
// Override with provided options
if len(opts) > 0 && opts[0] != nil {
opt := opts[0]
if opt.GrpcTimeout > 0 {
grpcTimeout = opt.GrpcTimeout
}
if opt.UrlPreference != "" {
urlPref = opt.UrlPreference
}
if opt.CacheSize > 0 {
cacheSize = opt.CacheSize
}
if opt.FailureThreshold > 0 {
failureThreshold = opt.FailureThreshold
}
if opt.ResetTimeout > 0 {
resetTimeout = opt.ResetTimeout
}
if opt.MaxRetries > 0 {
maxRetries = opt.MaxRetries
}
if opt.InitialRetryWait > 0 {
initialRetryWait = opt.InitialRetryWait
}
if opt.RetryBackoffFactor > 0 {
retryBackoffFactor = opt.RetryBackoffFactor
}
}
// Initialize health tracking for each filer
health := make([]*filerHealth, len(filerAddresses))
for i := range health {
health[i] = &filerHealth{}
}
fc := &FilerClient{
filerAddresses: filerAddresses,
filerIndex: 0,
filerHealth: health,
grpcDialOption: grpcDialOption,
urlPreference: urlPref,
grpcTimeout: grpcTimeout,
cacheSize: cacheSize,
clientId: rand.Int31(), // Random client ID for gRPC metadata tracking
failureThreshold: failureThreshold,
resetTimeout: resetTimeout,
maxRetries: maxRetries,
initialRetryWait: initialRetryWait,
retryBackoffFactor: retryBackoffFactor,
}
// Create provider that references this FilerClient for failover support
provider := &filerVolumeProvider{
filerClient: fc,
}
fc.vidMapClient = newVidMapClient(provider, dataCenter, cacheSize)
return fc
}
// GetLookupFileIdFunction returns a lookup function with URL preference handling
func (fc *FilerClient) GetLookupFileIdFunction() LookupFileIdFunctionType {
if fc.urlPreference == PreferUrl {
// Use the default implementation from vidMapClient
return fc.vidMapClient.GetLookupFileIdFunction()
}
// Custom implementation that prefers PublicUrl
return func(ctx context.Context, fileId string) (fullUrls []string, err error) {
// Parse file ID to extract volume ID
parts := strings.Split(fileId, ",")
if len(parts) != 2 {
return nil, fmt.Errorf("invalid fileId format: %s", fileId)
}
volumeIdStr := parts[0]
// First try the cache using LookupVolumeIdsWithFallback
vidLocations, err := fc.LookupVolumeIdsWithFallback(ctx, []string{volumeIdStr})
// Check for partial results first (important for multi-volume batched lookups)
locations, found := vidLocations[volumeIdStr]
if !found || len(locations) == 0 {
// Volume not found - return specific error with context from lookup if available
if err != nil {
return nil, fmt.Errorf("volume %s not found for fileId %s: %w", volumeIdStr, fileId, err)
}
return nil, fmt.Errorf("volume %s not found for fileId %s", volumeIdStr, fileId)
}
// Volume found successfully - ignore any errors about other volumes
// (not relevant for single-volume lookup, but defensive for future batching)
// Build URLs with publicUrl preference, and also prefer same DC
var sameDcUrls, otherDcUrls []string
dataCenter := fc.GetDataCenter()
for _, loc := range locations {
url := loc.PublicUrl
if url == "" {
url = loc.Url
}
httpUrl := "http://" + url + "/" + fileId
if dataCenter != "" && dataCenter == loc.DataCenter {
sameDcUrls = append(sameDcUrls, httpUrl)
} else {
otherDcUrls = append(otherDcUrls, httpUrl)
}
}
// Shuffle to distribute load across volume servers
rand.Shuffle(len(sameDcUrls), func(i, j int) { sameDcUrls[i], sameDcUrls[j] = sameDcUrls[j], sameDcUrls[i] })
rand.Shuffle(len(otherDcUrls), func(i, j int) { otherDcUrls[i], otherDcUrls[j] = otherDcUrls[j], otherDcUrls[i] })
// Prefer same data center
fullUrls = append(sameDcUrls, otherDcUrls...)
return fullUrls, nil
}
}
// isRetryableGrpcError checks if a gRPC error is transient and should be retried
//
// Note on codes.Aborted: While Aborted can indicate application-level conflicts
// (e.g., transaction failures), in the context of volume location lookups (which
// are simple read-only operations with no transactions), Aborted is more likely
// to indicate transient server issues during restart/recovery. We include it here
// for volume lookups but log it for visibility in case misclassification occurs.
func isRetryableGrpcError(err error) bool {
if err == nil {
return false
}
// Check gRPC status code
st, ok := status.FromError(err)
if ok {
switch st.Code() {
case codes.Unavailable: // Server unavailable (temporary)
return true
case codes.DeadlineExceeded: // Request timeout
return true
case codes.ResourceExhausted: // Rate limited or overloaded
return true
case codes.Aborted:
// Aborted during read-only volume lookups is likely transient
// (e.g., filer restarting), but log for visibility
glog.V(1).Infof("Treating Aborted as retryable for volume lookup: %v", err)
return true
}
}
// Fallback to string matching for non-gRPC errors (e.g., network errors)
errStr := err.Error()
return strings.Contains(errStr, "transport") ||
strings.Contains(errStr, "connection") ||
strings.Contains(errStr, "timeout") ||
strings.Contains(errStr, "unavailable")
}
// shouldSkipUnhealthyFiler checks if we should skip a filer based on recent failures
// Circuit breaker pattern: skip filers with multiple recent consecutive failures
func (fc *FilerClient) shouldSkipUnhealthyFiler(index int32) bool {
health := fc.filerHealth[index]
failureCount := atomic.LoadInt32(&health.failureCount)
// Check if failure count exceeds threshold
if failureCount < fc.failureThreshold {
return false
}
// Re-check unhealthy filers after reset timeout
lastFailureNs := atomic.LoadInt64(&health.lastFailureTimeNs)
if lastFailureNs == 0 {
return false // Never failed, shouldn't skip
}
lastFailureTime := time.Unix(0, lastFailureNs)
if time.Since(lastFailureTime) > fc.resetTimeout {
return false // Time to re-check
}
return true // Skip this unhealthy filer
}
// recordFilerSuccess resets failure tracking for a successful filer
func (fc *FilerClient) recordFilerSuccess(index int32) {
health := fc.filerHealth[index]
atomic.StoreInt32(&health.failureCount, 0)
}
// recordFilerFailure increments failure count for an unhealthy filer
func (fc *FilerClient) recordFilerFailure(index int32) {
health := fc.filerHealth[index]
atomic.AddInt32(&health.failureCount, 1)
atomic.StoreInt64(&health.lastFailureTimeNs, time.Now().UnixNano())
}
// LookupVolumeIds queries the filer for volume locations with automatic failover
// Tries all configured filer addresses until one succeeds (high availability)
// Retries transient gRPC errors (Unavailable, DeadlineExceeded, etc.) with exponential backoff
// Note: Unlike master's VolumeIdLocation, filer's Locations message doesn't currently have
// an Error field. This implementation handles the current structure while being prepared
// for future error reporting enhancements.
func (p *filerVolumeProvider) LookupVolumeIds(ctx context.Context, volumeIds []string) (map[string][]Location, error) {
fc := p.filerClient
result := make(map[string][]Location)
// Retry transient failures with configurable backoff
var lastErr error
waitTime := fc.initialRetryWait
maxRetries := fc.maxRetries
for retry := 0; retry < maxRetries; retry++ {
// Try all filer addresses with round-robin starting from current index
// Skip known-unhealthy filers (circuit breaker pattern)
i := atomic.LoadInt32(&fc.filerIndex)
n := int32(len(fc.filerAddresses))
for x := int32(0); x < n; x++ {
// Circuit breaker: skip unhealthy filers
if fc.shouldSkipUnhealthyFiler(i) {
glog.V(2).Infof("FilerClient: skipping unhealthy filer %s (consecutive failures: %d)",
fc.filerAddresses[i], atomic.LoadInt32(&fc.filerHealth[i].failureCount))
i++
if i >= n {
i = 0
}
continue
}
filerAddress := fc.filerAddresses[i]
// Use anonymous function to ensure defer cancel() is called per iteration, not accumulated
err := func() error {
// Create a fresh timeout context for each filer attempt
// This ensures each retry gets the full grpcTimeout, not a diminishing deadline
timeoutCtx, cancel := context.WithTimeout(ctx, fc.grpcTimeout)
defer cancel() // Always clean up context, even on panic or early return
return pb.WithGrpcFilerClient(false, fc.clientId, filerAddress, fc.grpcDialOption, func(client filer_pb.SeaweedFilerClient) error {
resp, err := client.LookupVolume(timeoutCtx, &filer_pb.LookupVolumeRequest{
VolumeIds: volumeIds,
})
if err != nil {
return fmt.Errorf("filer.LookupVolume failed: %w", err)
}
// Process each volume in the response
for vid, locs := range resp.LocationsMap {
// Convert locations from protobuf to internal format
var locations []Location
for _, loc := range locs.Locations {
locations = append(locations, Location{
Url: loc.Url,
PublicUrl: loc.PublicUrl,
DataCenter: loc.DataCenter,
GrpcPort: int(loc.GrpcPort),
})
}
// Only add to result if we have locations
// Empty locations with no gRPC error means "not found" (volume doesn't exist)
if len(locations) > 0 {
result[vid] = locations
glog.V(4).Infof("FilerClient: volume %s found with %d location(s)", vid, len(locations))
} else {
glog.V(2).Infof("FilerClient: volume %s not found (no locations in response)", vid)
}
}
// Check for volumes that weren't in the response at all
// This could indicate a problem with the filer
for _, vid := range volumeIds {
if _, found := resp.LocationsMap[vid]; !found {
glog.V(1).Infof("FilerClient: volume %s missing from filer response", vid)
}
}
return nil
})
}()
if err != nil {
glog.V(1).Infof("FilerClient: filer %s lookup failed (attempt %d/%d, retry %d/%d): %v", filerAddress, x+1, n, retry+1, maxRetries, err)
fc.recordFilerFailure(i)
lastErr = err
i++
if i >= n {
i = 0
}
continue
}
// Success - update the preferred filer index and reset health tracking
atomic.StoreInt32(&fc.filerIndex, i)
fc.recordFilerSuccess(i)
glog.V(3).Infof("FilerClient: looked up %d volumes on %s, found %d", len(volumeIds), filerAddress, len(result))
return result, nil
}
// All filers failed on this attempt
// Check if the error is retryable (transient gRPC error)
if !isRetryableGrpcError(lastErr) {
// Non-retryable error (e.g., NotFound, PermissionDenied) - fail immediately
return nil, fmt.Errorf("all %d filer(s) failed with non-retryable error: %w", n, lastErr)
}
// Transient error - retry if we have attempts left
if retry < maxRetries-1 {
glog.V(1).Infof("FilerClient: all %d filer(s) failed with retryable error (attempt %d/%d), retrying in %v: %v",
n, retry+1, maxRetries, waitTime, lastErr)
time.Sleep(waitTime)
waitTime = time.Duration(float64(waitTime) * fc.retryBackoffFactor)
}
}
// All retries exhausted
return nil, fmt.Errorf("all %d filer(s) failed after %d attempts, last error: %w", len(fc.filerAddresses), maxRetries, lastErr)
}

View File

@@ -5,189 +5,53 @@ import (
"errors"
"fmt"
"math/rand"
"sort"
"strconv"
"strings"
"sync"
"time"
"golang.org/x/sync/singleflight"
"github.com/seaweedfs/seaweedfs/weed/util/version"
"github.com/seaweedfs/seaweedfs/weed/stats"
"github.com/seaweedfs/seaweedfs/weed/util"
"google.golang.org/grpc"
"github.com/seaweedfs/seaweedfs/weed/glog"
"github.com/seaweedfs/seaweedfs/weed/pb"
"github.com/seaweedfs/seaweedfs/weed/pb/master_pb"
"github.com/seaweedfs/seaweedfs/weed/stats"
"github.com/seaweedfs/seaweedfs/weed/util"
"github.com/seaweedfs/seaweedfs/weed/util/version"
)
type MasterClient struct {
FilerGroup string
clientType string
clientHost pb.ServerAddress
rack string
currentMaster pb.ServerAddress
currentMasterLock sync.RWMutex
masters pb.ServerDiscovery
grpcDialOption grpc.DialOption
// vidMap stores volume location mappings
// Protected by vidMapLock to prevent race conditions during pointer swaps in resetVidMap
vidMap *vidMap
vidMapLock sync.RWMutex
vidMapCacheSize int
OnPeerUpdate func(update *master_pb.ClusterNodeUpdate, startFrom time.Time)
OnPeerUpdateLock sync.RWMutex
// Per-batch in-flight tracking to prevent duplicate lookups for the same set of volumes
vidLookupGroup singleflight.Group
// masterVolumeProvider implements VolumeLocationProvider by querying master
// This is rarely called since master pushes updates proactively via KeepConnected stream
type masterVolumeProvider struct {
masterClient *MasterClient
}
func NewMasterClient(grpcDialOption grpc.DialOption, filerGroup string, clientType string, clientHost pb.ServerAddress, clientDataCenter string, rack string, masters pb.ServerDiscovery) *MasterClient {
return &MasterClient{
FilerGroup: filerGroup,
clientType: clientType,
clientHost: clientHost,
rack: rack,
masters: masters,
grpcDialOption: grpcDialOption,
vidMap: newVidMap(clientDataCenter),
vidMapCacheSize: 5,
}
}
func (mc *MasterClient) SetOnPeerUpdateFn(onPeerUpdate func(update *master_pb.ClusterNodeUpdate, startFrom time.Time)) {
mc.OnPeerUpdateLock.Lock()
mc.OnPeerUpdate = onPeerUpdate
mc.OnPeerUpdateLock.Unlock()
}
func (mc *MasterClient) GetLookupFileIdFunction() LookupFileIdFunctionType {
return mc.LookupFileIdWithFallback
}
func (mc *MasterClient) LookupFileIdWithFallback(ctx context.Context, fileId string) (fullUrls []string, err error) {
// Try cache first using the fast path - grab both vidMap and dataCenter in one lock
mc.vidMapLock.RLock()
vm := mc.vidMap
dataCenter := vm.DataCenter
mc.vidMapLock.RUnlock()
fullUrls, err = vm.LookupFileId(ctx, fileId)
if err == nil && len(fullUrls) > 0 {
return
}
// Extract volume ID from file ID (format: "volumeId,needle_id_cookie")
parts := strings.Split(fileId, ",")
if len(parts) != 2 {
return nil, fmt.Errorf("invalid fileId %s", fileId)
}
volumeId := parts[0]
// Use shared lookup logic with batching and singleflight
vidLocations, err := mc.LookupVolumeIdsWithFallback(ctx, []string{volumeId})
if err != nil {
return nil, fmt.Errorf("LookupVolume %s failed: %v", fileId, err)
}
locations, found := vidLocations[volumeId]
if !found || len(locations) == 0 {
return nil, fmt.Errorf("volume %s not found for fileId %s", volumeId, fileId)
}
// Build HTTP URLs from locations, preferring same data center
var sameDcUrls, otherDcUrls []string
for _, loc := range locations {
httpUrl := "http://" + loc.Url + "/" + fileId
if dataCenter != "" && dataCenter == loc.DataCenter {
sameDcUrls = append(sameDcUrls, httpUrl)
} else {
otherDcUrls = append(otherDcUrls, httpUrl)
}
}
// Prefer same data center
fullUrls = append(sameDcUrls, otherDcUrls...)
return fullUrls, nil
}
// LookupVolumeIdsWithFallback looks up volume locations, querying master if not in cache
// Uses singleflight to coalesce concurrent requests for the same batch of volumes
func (mc *MasterClient) LookupVolumeIdsWithFallback(ctx context.Context, volumeIds []string) (map[string][]Location, error) {
// LookupVolumeIds queries the master for volume locations (fallback when cache misses)
// Returns partial results with aggregated errors for volumes that failed
func (p *masterVolumeProvider) LookupVolumeIds(ctx context.Context, volumeIds []string) (map[string][]Location, error) {
result := make(map[string][]Location)
var needsLookup []string
var lookupErrors []error
// Check cache first and parse volume IDs once
vidStringToUint := make(map[string]uint32, len(volumeIds))
glog.V(2).Infof("Looking up %d volumes from master: %v", len(volumeIds), volumeIds)
// Get stable pointer to vidMap with minimal lock hold time
vm := mc.getStableVidMap()
// Use a timeout for the master lookup to prevent indefinite blocking
timeoutCtx, cancel := context.WithTimeout(ctx, p.masterClient.grpcTimeout)
defer cancel()
for _, vidString := range volumeIds {
vid, err := strconv.ParseUint(vidString, 10, 32)
if err != nil {
return nil, fmt.Errorf("invalid volume id %s: %v", vidString, err)
}
vidStringToUint[vidString] = uint32(vid)
locations, found := vm.GetLocations(uint32(vid))
if found && len(locations) > 0 {
result[vidString] = locations
} else {
needsLookup = append(needsLookup, vidString)
}
}
if len(needsLookup) == 0 {
return result, nil
}
// Batch query all missing volumes using singleflight on the batch key
// Sort for stable key to coalesce identical batches
sort.Strings(needsLookup)
batchKey := strings.Join(needsLookup, ",")
sfResult, err, _ := mc.vidLookupGroup.Do(batchKey, func() (interface{}, error) {
// Double-check cache for volumes that might have been populated while waiting
stillNeedLookup := make([]string, 0, len(needsLookup))
batchResult := make(map[string][]Location)
// Get stable pointer with minimal lock hold time
vm := mc.getStableVidMap()
for _, vidString := range needsLookup {
vid := vidStringToUint[vidString] // Use pre-parsed value
if locations, found := vm.GetLocations(vid); found && len(locations) > 0 {
batchResult[vidString] = locations
} else {
stillNeedLookup = append(stillNeedLookup, vidString)
}
}
if len(stillNeedLookup) == 0 {
return batchResult, nil
}
// Query master with batched volume IDs
glog.V(2).Infof("Looking up %d volumes from master: %v", len(stillNeedLookup), stillNeedLookup)
err := pb.WithMasterClient(false, mc.GetMaster(ctx), mc.grpcDialOption, false, func(client master_pb.SeaweedClient) error {
resp, err := client.LookupVolume(ctx, &master_pb.LookupVolumeRequest{
VolumeOrFileIds: stillNeedLookup,
err := pb.WithMasterClient(false, p.masterClient.GetMaster(ctx), p.masterClient.grpcDialOption, false, func(client master_pb.SeaweedClient) error {
resp, err := client.LookupVolume(timeoutCtx, &master_pb.LookupVolumeRequest{
VolumeOrFileIds: volumeIds,
})
if err != nil {
return fmt.Errorf("master lookup failed: %v", err)
}
for _, vidLoc := range resp.VolumeIdLocations {
// Preserve per-volume errors from master response
// These could indicate misconfiguration, volume deletion, etc.
if vidLoc.Error != "" {
glog.V(0).Infof("volume %s lookup error: %s", vidLoc.VolumeOrFileId, vidLoc.Error)
lookupErrors = append(lookupErrors, fmt.Errorf("volume %s: %s", vidLoc.VolumeOrFileId, vidLoc.Error))
glog.V(1).Infof("volume %s lookup error from master: %s", vidLoc.VolumeOrFileId, vidLoc.Error)
continue
}
@@ -196,6 +60,7 @@ func (mc *MasterClient) LookupVolumeIdsWithFallback(ctx context.Context, volumeI
vidOnly := parts[0]
vid, err := strconv.ParseUint(vidOnly, 10, 32)
if err != nil {
lookupErrors = append(lookupErrors, fmt.Errorf("volume %s: invalid volume ID format: %w", vidLoc.VolumeOrFileId, err))
glog.Warningf("Failed to parse volume id '%s' from master response '%s': %v", vidOnly, vidLoc.VolumeOrFileId, err)
continue
}
@@ -208,125 +73,75 @@ func (mc *MasterClient) LookupVolumeIdsWithFallback(ctx context.Context, volumeI
GrpcPort: int(masterLoc.GrpcPort),
DataCenter: masterLoc.DataCenter,
}
mc.addLocation(uint32(vid), loc)
// Update cache with the location
p.masterClient.addLocation(uint32(vid), loc)
locations = append(locations, loc)
}
if len(locations) > 0 {
batchResult[vidOnly] = locations
result[vidOnly] = locations
}
}
return nil
})
if err != nil {
return batchResult, err
}
return batchResult, nil
})
if err != nil {
lookupErrors = append(lookupErrors, err)
return nil, err
}
// Merge singleflight batch results
if batchLocations, ok := sfResult.(map[string][]Location); ok {
for vid, locs := range batchLocations {
result[vid] = locs
}
// Return partial results with detailed errors
// Callers should check both result map and error
if len(lookupErrors) > 0 {
glog.V(2).Infof("MasterClient: looked up %d volumes, found %d, %d errors", len(volumeIds), len(result), len(lookupErrors))
return result, fmt.Errorf("master volume lookup errors: %w", errors.Join(lookupErrors...))
}
// Check for volumes that still weren't found
for _, vidString := range needsLookup {
if _, found := result[vidString]; !found {
lookupErrors = append(lookupErrors, fmt.Errorf("volume %s not found", vidString))
}
}
// Return aggregated errors using errors.Join to preserve error types
return result, errors.Join(lookupErrors...)
glog.V(3).Infof("MasterClient: looked up %d volumes, found %d", len(volumeIds), len(result))
return result, nil
}
func (mc *MasterClient) getCurrentMaster() pb.ServerAddress {
mc.currentMasterLock.RLock()
defer mc.currentMasterLock.RUnlock()
return mc.currentMaster
// MasterClient connects to master servers and maintains volume location cache
// It receives real-time updates via KeepConnected streaming and uses vidMapClient for caching
type MasterClient struct {
*vidMapClient // Embedded cache with shared logic
FilerGroup string
clientType string
clientHost pb.ServerAddress
rack string
currentMaster pb.ServerAddress
currentMasterLock sync.RWMutex
masters pb.ServerDiscovery
grpcDialOption grpc.DialOption
grpcTimeout time.Duration // Timeout for gRPC calls to master
OnPeerUpdate func(update *master_pb.ClusterNodeUpdate, startFrom time.Time)
OnPeerUpdateLock sync.RWMutex
}
func (mc *MasterClient) setCurrentMaster(master pb.ServerAddress) {
mc.currentMasterLock.Lock()
mc.currentMaster = master
mc.currentMasterLock.Unlock()
func NewMasterClient(grpcDialOption grpc.DialOption, filerGroup string, clientType string, clientHost pb.ServerAddress, clientDataCenter string, rack string, masters pb.ServerDiscovery) *MasterClient {
mc := &MasterClient{
FilerGroup: filerGroup,
clientType: clientType,
clientHost: clientHost,
rack: rack,
masters: masters,
grpcDialOption: grpcDialOption,
grpcTimeout: 5 * time.Second, // Default: 5 seconds for gRPC calls to master
}
// Create provider that references this MasterClient
provider := &masterVolumeProvider{masterClient: mc}
// Initialize embedded vidMapClient with the provider and default cache size
mc.vidMapClient = newVidMapClient(provider, clientDataCenter, DefaultVidMapCacheSize)
return mc
}
func (mc *MasterClient) GetMaster(ctx context.Context) pb.ServerAddress {
mc.WaitUntilConnected(ctx)
return mc.getCurrentMaster()
}
func (mc *MasterClient) GetMasters(ctx context.Context) []pb.ServerAddress {
mc.WaitUntilConnected(ctx)
return mc.masters.GetInstances()
}
func (mc *MasterClient) WaitUntilConnected(ctx context.Context) {
attempts := 0
for {
select {
case <-ctx.Done():
return
default:
currentMaster := mc.getCurrentMaster()
if currentMaster != "" {
return
}
attempts++
if attempts%100 == 0 { // Log every 100 attempts (roughly every 20 seconds)
glog.V(0).Infof("%s.%s WaitUntilConnected still waiting for master connection (attempt %d)...", mc.FilerGroup, mc.clientType, attempts)
}
time.Sleep(time.Duration(rand.Int31n(200)) * time.Millisecond)
}
}
}
func (mc *MasterClient) KeepConnectedToMaster(ctx context.Context) {
glog.V(1).Infof("%s.%s masterClient bootstraps with masters %v", mc.FilerGroup, mc.clientType, mc.masters)
for {
select {
case <-ctx.Done():
glog.V(0).Infof("Connection to masters stopped: %v", ctx.Err())
return
default:
mc.tryAllMasters(ctx)
time.Sleep(time.Second)
}
}
}
func (mc *MasterClient) FindLeaderFromOtherPeers(myMasterAddress pb.ServerAddress) (leader string) {
for _, master := range mc.masters.GetInstances() {
if master == myMasterAddress {
continue
}
if grpcErr := pb.WithMasterClient(false, master, mc.grpcDialOption, false, func(client master_pb.SeaweedClient) error {
ctx, cancel := context.WithTimeout(context.Background(), 120*time.Millisecond)
defer cancel()
resp, err := client.GetMasterConfiguration(ctx, &master_pb.GetMasterConfigurationRequest{})
if err != nil {
return err
}
leader = resp.Leader
return nil
}); grpcErr != nil {
glog.V(0).Infof("connect to %s: %v", master, grpcErr)
}
if leader != "" {
glog.V(0).Infof("existing leader is %s", leader)
return
}
}
glog.V(0).Infof("No existing leader found!")
return
func (mc *MasterClient) SetOnPeerUpdateFn(onPeerUpdate func(update *master_pb.ClusterNodeUpdate, startFrom time.Time)) {
mc.OnPeerUpdateLock.Lock()
mc.OnPeerUpdate = onPeerUpdate
mc.OnPeerUpdateLock.Unlock()
}
func (mc *MasterClient) tryAllMasters(ctx context.Context) {
@@ -393,6 +208,8 @@ func (mc *MasterClient) tryConnectToMaster(ctx context.Context, master pb.Server
mc.resetVidMap()
mc.updateVidMap(resp)
} else {
// First message from master is not VolumeLocation (e.g., ClusterNodeUpdate)
// Still need to reset cache to ensure we don't use stale data from previous master
mc.resetVidMap()
}
mc.setCurrentMaster(master)
@@ -406,7 +223,8 @@ func (mc *MasterClient) tryConnectToMaster(ctx context.Context, master pb.Server
}
if resp.VolumeLocation != nil {
// maybe the leader is changed
// Check for leader change during the stream
// If master announces a new leader, reconnect to it
if resp.VolumeLocation.Leader != "" && string(mc.GetMaster(ctx)) != resp.VolumeLocation.Leader {
glog.V(0).Infof("currentMaster %v redirected to leader %v", mc.GetMaster(ctx), resp.VolumeLocation.Leader)
nextHintedLeader = pb.ServerAddress(resp.VolumeLocation.Leader)
@@ -415,7 +233,6 @@ func (mc *MasterClient) tryConnectToMaster(ctx context.Context, master pb.Server
}
mc.updateVidMap(resp)
}
if resp.ClusterNodeUpdate != nil {
update := resp.ClusterNodeUpdate
mc.OnPeerUpdateLock.RLock()
@@ -442,7 +259,7 @@ func (mc *MasterClient) tryConnectToMaster(ctx context.Context, master pb.Server
stats.MasterClientConnectCounter.WithLabelValues(stats.Failed).Inc()
glog.V(1).Infof("%s.%s masterClient failed to connect with master %v: %v", mc.FilerGroup, mc.clientType, master, gprcErr)
}
return
return nextHintedLeader
}
func (mc *MasterClient) updateVidMap(resp *master_pb.KeepConnectedResponse) {
@@ -494,110 +311,103 @@ func (mc *MasterClient) WithClientCustomGetMaster(getMasterF func() pb.ServerAdd
})
}
// getStableVidMap gets a stable pointer to the vidMap, releasing the lock immediately.
// This is safe for read operations as the returned pointer is a stable snapshot,
// and the underlying vidMap methods have their own internal locking.
func (mc *MasterClient) getStableVidMap() *vidMap {
mc.vidMapLock.RLock()
vm := mc.vidMap
mc.vidMapLock.RUnlock()
return vm
func (mc *MasterClient) getCurrentMaster() pb.ServerAddress {
mc.currentMasterLock.RLock()
defer mc.currentMasterLock.RUnlock()
return mc.currentMaster
}
// withCurrentVidMap executes a function with the current vidMap under a read lock.
// This is for methods that modify vidMap's internal state, ensuring the pointer
// is not swapped by resetVidMap during the operation. The actual map mutations
// are protected by vidMap's internal mutex.
func (mc *MasterClient) withCurrentVidMap(f func(vm *vidMap)) {
mc.vidMapLock.RLock()
defer mc.vidMapLock.RUnlock()
f(mc.vidMap)
func (mc *MasterClient) setCurrentMaster(master pb.ServerAddress) {
mc.currentMasterLock.Lock()
mc.currentMaster = master
mc.currentMasterLock.Unlock()
}
// Public methods for external packages to access vidMap safely
// GetLocations safely retrieves volume locations
func (mc *MasterClient) GetLocations(vid uint32) (locations []Location, found bool) {
return mc.getStableVidMap().GetLocations(vid)
// GetMaster returns the current master address, blocking until connected.
//
// IMPORTANT: This method blocks until KeepConnectedToMaster successfully establishes
// a connection to a master server. If KeepConnectedToMaster hasn't been started in a
// background goroutine, this will block indefinitely (or until ctx is canceled).
//
// Typical initialization pattern:
//
// mc := wdclient.NewMasterClient(...)
// go mc.KeepConnectedToMaster(ctx) // Start connection management
// // ... later ...
// master := mc.GetMaster(ctx) // Will block until connected
//
// If called before KeepConnectedToMaster establishes a connection, this may cause
// unexpected timeouts in LookupVolumeIds and other operations that depend on it.
func (mc *MasterClient) GetMaster(ctx context.Context) pb.ServerAddress {
mc.WaitUntilConnected(ctx)
return mc.getCurrentMaster()
}
// GetLocationsClone safely retrieves a clone of volume locations
func (mc *MasterClient) GetLocationsClone(vid uint32) (locations []Location, found bool) {
return mc.getStableVidMap().GetLocationsClone(vid)
// GetMasters returns all configured master addresses, blocking until connected.
// See GetMaster() for important initialization contract details.
func (mc *MasterClient) GetMasters(ctx context.Context) []pb.ServerAddress {
mc.WaitUntilConnected(ctx)
return mc.masters.GetInstances()
}
// GetVidLocations safely retrieves volume locations by string ID
func (mc *MasterClient) GetVidLocations(vid string) (locations []Location, err error) {
return mc.getStableVidMap().GetVidLocations(vid)
}
// LookupFileId safely looks up URLs for a file ID
func (mc *MasterClient) LookupFileId(ctx context.Context, fileId string) (fullUrls []string, err error) {
return mc.getStableVidMap().LookupFileId(ctx, fileId)
}
// LookupVolumeServerUrl safely looks up volume server URLs
func (mc *MasterClient) LookupVolumeServerUrl(vid string) (serverUrls []string, err error) {
return mc.getStableVidMap().LookupVolumeServerUrl(vid)
}
// GetDataCenter safely retrieves the data center
func (mc *MasterClient) GetDataCenter() string {
return mc.getStableVidMap().DataCenter
}
// Thread-safe helpers for vidMap operations
// addLocation adds a volume location
func (mc *MasterClient) addLocation(vid uint32, location Location) {
mc.withCurrentVidMap(func(vm *vidMap) {
vm.addLocation(vid, location)
})
}
// deleteLocation removes a volume location
func (mc *MasterClient) deleteLocation(vid uint32, location Location) {
mc.withCurrentVidMap(func(vm *vidMap) {
vm.deleteLocation(vid, location)
})
}
// addEcLocation adds an EC volume location
func (mc *MasterClient) addEcLocation(vid uint32, location Location) {
mc.withCurrentVidMap(func(vm *vidMap) {
vm.addEcLocation(vid, location)
})
}
// deleteEcLocation removes an EC volume location
func (mc *MasterClient) deleteEcLocation(vid uint32, location Location) {
mc.withCurrentVidMap(func(vm *vidMap) {
vm.deleteEcLocation(vid, location)
})
}
func (mc *MasterClient) resetVidMap() {
mc.vidMapLock.Lock()
defer mc.vidMapLock.Unlock()
// Preserve the existing vidMap in the cache chain
// No need to clone - the existing vidMap has its own mutex for thread safety
tail := mc.vidMap
nvm := newVidMap(tail.DataCenter)
nvm.cache.Store(tail)
mc.vidMap = nvm
// Trim cache chain to vidMapCacheSize by traversing to the last node
// that should remain and cutting the chain after it
node := tail
for i := 0; i < mc.vidMapCacheSize-1; i++ {
if node.cache.Load() == nil {
// WaitUntilConnected blocks until a master connection is established or ctx is canceled.
// This does NOT initiate connections - it only waits for KeepConnectedToMaster to succeed.
func (mc *MasterClient) WaitUntilConnected(ctx context.Context) {
attempts := 0
for {
select {
case <-ctx.Done():
return
default:
currentMaster := mc.getCurrentMaster()
if currentMaster != "" {
return
}
node = node.cache.Load()
attempts++
if attempts%100 == 0 { // Log every 100 attempts (roughly every 20 seconds)
glog.V(0).Infof("%s.%s WaitUntilConnected still waiting for master connection (attempt %d)...", mc.FilerGroup, mc.clientType, attempts)
}
time.Sleep(time.Duration(rand.Int31n(200)) * time.Millisecond)
}
if node != nil {
node.cache.Store(nil)
}
}
func (mc *MasterClient) KeepConnectedToMaster(ctx context.Context) {
glog.V(1).Infof("%s.%s masterClient bootstraps with masters %v", mc.FilerGroup, mc.clientType, mc.masters)
for {
select {
case <-ctx.Done():
glog.V(0).Infof("Connection to masters stopped: %v", ctx.Err())
return
default:
mc.tryAllMasters(ctx)
time.Sleep(time.Second)
}
}
}
func (mc *MasterClient) FindLeaderFromOtherPeers(myMasterAddress pb.ServerAddress) (leader string) {
for _, master := range mc.masters.GetInstances() {
if master == myMasterAddress {
continue
}
if grpcErr := pb.WithMasterClient(false, master, mc.grpcDialOption, false, func(client master_pb.SeaweedClient) error {
ctx, cancel := context.WithTimeout(context.Background(), 120*time.Millisecond)
defer cancel()
resp, err := client.GetMasterConfiguration(ctx, &master_pb.GetMasterConfigurationRequest{})
if err != nil {
return err
}
leader = resp.Leader
return nil
}); grpcErr != nil {
glog.V(0).Infof("connect to %s: %v", master, grpcErr)
}
if leader != "" {
glog.V(0).Infof("existing leader is %s", leader)
return
}
}
glog.V(0).Infof("No existing leader found!")
return
}

View File

@@ -0,0 +1,347 @@
package wdclient
import (
"context"
"errors"
"fmt"
"math/rand"
"sort"
"strconv"
"strings"
"sync"
"golang.org/x/sync/singleflight"
"github.com/seaweedfs/seaweedfs/weed/glog"
)
// VolumeLocationProvider is the interface for looking up volume locations
// This allows different implementations (master subscription, filer queries, etc.)
type VolumeLocationProvider interface {
// LookupVolumeIds looks up volume locations for the given volume IDs
// Returns a map of volume ID to locations
LookupVolumeIds(ctx context.Context, volumeIds []string) (map[string][]Location, error)
}
// vidMapClient provides volume location caching with pluggable lookup
// It wraps the battle-tested vidMap with customizable volume lookup strategies
type vidMapClient struct {
vidMap *vidMap
vidMapLock sync.RWMutex
vidMapCacheSize int
provider VolumeLocationProvider
vidLookupGroup singleflight.Group
}
const (
// DefaultVidMapCacheSize is the default number of historical vidMap snapshots to keep
// This provides cache history when volumes move between servers
DefaultVidMapCacheSize = 5
)
// newVidMapClient creates a new client with the given provider and data center
func newVidMapClient(provider VolumeLocationProvider, dataCenter string, cacheSize int) *vidMapClient {
if cacheSize <= 0 {
cacheSize = DefaultVidMapCacheSize
}
return &vidMapClient{
vidMap: newVidMap(dataCenter),
vidMapCacheSize: cacheSize,
provider: provider,
}
}
// GetLookupFileIdFunction returns a function that can be used to lookup file IDs
func (vc *vidMapClient) GetLookupFileIdFunction() LookupFileIdFunctionType {
return vc.LookupFileIdWithFallback
}
// LookupFileIdWithFallback looks up a file ID, checking cache first, then using provider
func (vc *vidMapClient) LookupFileIdWithFallback(ctx context.Context, fileId string) (fullUrls []string, err error) {
// Try cache first - hold read lock during entire vidMap access to prevent swap during operation
vc.vidMapLock.RLock()
vm := vc.vidMap
dataCenter := vm.DataCenter
fullUrls, err = vm.LookupFileId(ctx, fileId)
vc.vidMapLock.RUnlock()
// Cache hit - return immediately
if err == nil && len(fullUrls) > 0 {
return
}
// Cache miss - extract volume ID from file ID (format: "volumeId,needle_id_cookie")
parts := strings.Split(fileId, ",")
if len(parts) != 2 {
return nil, fmt.Errorf("invalid fileId %s", fileId)
}
volumeId := parts[0]
// Use shared lookup logic with batching and singleflight
vidLocations, err := vc.LookupVolumeIdsWithFallback(ctx, []string{volumeId})
// Check for partial results first (important for multi-volume batched lookups)
locations, found := vidLocations[volumeId]
if !found || len(locations) == 0 {
// Volume not found - return specific error with context from lookup if available
if err != nil {
return nil, fmt.Errorf("volume %s not found for fileId %s: %w", volumeId, fileId, err)
}
return nil, fmt.Errorf("volume %s not found for fileId %s", volumeId, fileId)
}
// Volume found successfully - ignore any errors about other volumes
// (not relevant for single-volume lookup, but defensive for future batching)
// Build HTTP URLs from locations, preferring same data center
var sameDcUrls, otherDcUrls []string
for _, loc := range locations {
httpUrl := "http://" + loc.Url + "/" + fileId
if dataCenter != "" && dataCenter == loc.DataCenter {
sameDcUrls = append(sameDcUrls, httpUrl)
} else {
otherDcUrls = append(otherDcUrls, httpUrl)
}
}
// Shuffle to distribute load across volume servers
rand.Shuffle(len(sameDcUrls), func(i, j int) { sameDcUrls[i], sameDcUrls[j] = sameDcUrls[j], sameDcUrls[i] })
rand.Shuffle(len(otherDcUrls), func(i, j int) { otherDcUrls[i], otherDcUrls[j] = otherDcUrls[j], otherDcUrls[i] })
// Prefer same data center
fullUrls = append(sameDcUrls, otherDcUrls...)
return fullUrls, nil
}
// LookupVolumeIdsWithFallback looks up volume locations, querying provider if not in cache.
// Uses singleflight to coalesce concurrent requests for the same batch of volumes.
//
// IMPORTANT: This function may return PARTIAL results with a non-nil error.
// The result map contains successfully looked up volumes, while the error aggregates
// failures for volumes that couldn't be found or had lookup errors.
//
// Callers MUST check both the result map AND the error:
// - result != nil && err == nil: All volumes found successfully
// - result != nil && err != nil: Some volumes found, some failed (check both)
// - result == nil && err != nil: Complete failure (connection error, etc.)
//
// Example usage:
//
// locs, err := mc.LookupVolumeIdsWithFallback(ctx, []string{"1", "2", "999"})
// if len(locs) > 0 {
// // Process successfully found volumes
// }
// if err != nil {
// // Log/handle failed volumes
// }
func (vc *vidMapClient) LookupVolumeIdsWithFallback(ctx context.Context, volumeIds []string) (map[string][]Location, error) {
result := make(map[string][]Location)
var needsLookup []string
var lookupErrors []error
// Check cache first and parse volume IDs once
vidStringToUint := make(map[string]uint32, len(volumeIds))
// Get stable pointer to vidMap with minimal lock hold time
vm := vc.getStableVidMap()
for _, vidString := range volumeIds {
vid, err := strconv.ParseUint(vidString, 10, 32)
if err != nil {
return nil, fmt.Errorf("invalid volume id %s: %v", vidString, err)
}
vidStringToUint[vidString] = uint32(vid)
locations, found := vm.GetLocations(uint32(vid))
if found && len(locations) > 0 {
result[vidString] = locations
} else {
needsLookup = append(needsLookup, vidString)
}
}
if len(needsLookup) == 0 {
return result, nil
}
// Batch query all missing volumes using singleflight on the batch key
// Sort for stable key to coalesce identical batches
sort.Strings(needsLookup)
batchKey := strings.Join(needsLookup, ",")
sfResult, err, _ := vc.vidLookupGroup.Do(batchKey, func() (interface{}, error) {
// Double-check cache for volumes that might have been populated while waiting
stillNeedLookup := make([]string, 0, len(needsLookup))
batchResult := make(map[string][]Location)
// Get stable pointer with minimal lock hold time
vm := vc.getStableVidMap()
for _, vidString := range needsLookup {
vid := vidStringToUint[vidString] // Use pre-parsed value
if locations, found := vm.GetLocations(vid); found && len(locations) > 0 {
batchResult[vidString] = locations
} else {
stillNeedLookup = append(stillNeedLookup, vidString)
}
}
if len(stillNeedLookup) == 0 {
return batchResult, nil
}
// Query provider with batched volume IDs
glog.V(2).Infof("Looking up %d volumes from provider: %v", len(stillNeedLookup), stillNeedLookup)
providerResults, err := vc.provider.LookupVolumeIds(ctx, stillNeedLookup)
if err != nil {
return batchResult, fmt.Errorf("provider lookup failed: %v", err)
}
// Update cache with results
for vidString, locations := range providerResults {
vid, err := strconv.ParseUint(vidString, 10, 32)
if err != nil {
glog.Warningf("Failed to parse volume id '%s': %v", vidString, err)
continue
}
for _, loc := range locations {
vc.addLocation(uint32(vid), loc)
}
if len(locations) > 0 {
batchResult[vidString] = locations
}
}
return batchResult, nil
})
if err != nil {
lookupErrors = append(lookupErrors, err)
}
// Merge singleflight batch results
if batchLocations, ok := sfResult.(map[string][]Location); ok {
for vid, locs := range batchLocations {
result[vid] = locs
}
}
// Check for volumes that still weren't found
for _, vidString := range needsLookup {
if _, found := result[vidString]; !found {
lookupErrors = append(lookupErrors, fmt.Errorf("volume %s not found", vidString))
}
}
// Return aggregated errors
return result, errors.Join(lookupErrors...)
}
// getStableVidMap gets a stable pointer to the vidMap, releasing the lock immediately.
// WARNING: Use with caution. The returned vidMap pointer is stable (won't be garbage collected
// due to cache chain), but the vidMapClient.vidMap field may be swapped by resetVidMap().
// For operations that must use the current vidMap atomically, use withCurrentVidMap() instead.
func (vc *vidMapClient) getStableVidMap() *vidMap {
vc.vidMapLock.RLock()
vm := vc.vidMap
vc.vidMapLock.RUnlock()
return vm
}
// withCurrentVidMap executes a function with the current vidMap under a read lock.
// This guarantees the vidMap instance cannot be swapped during the function execution.
// Use this when you need atomic access to the current vidMap for multiple operations.
func (vc *vidMapClient) withCurrentVidMap(f func(vm *vidMap)) {
vc.vidMapLock.RLock()
defer vc.vidMapLock.RUnlock()
f(vc.vidMap)
}
// Public methods for external access
// GetLocations safely retrieves volume locations
func (vc *vidMapClient) GetLocations(vid uint32) (locations []Location, found bool) {
return vc.getStableVidMap().GetLocations(vid)
}
// GetLocationsClone safely retrieves a clone of volume locations
func (vc *vidMapClient) GetLocationsClone(vid uint32) (locations []Location, found bool) {
return vc.getStableVidMap().GetLocationsClone(vid)
}
// GetVidLocations safely retrieves volume locations by string ID
func (vc *vidMapClient) GetVidLocations(vid string) (locations []Location, err error) {
return vc.getStableVidMap().GetVidLocations(vid)
}
// LookupFileId safely looks up URLs for a file ID
func (vc *vidMapClient) LookupFileId(ctx context.Context, fileId string) (fullUrls []string, err error) {
return vc.getStableVidMap().LookupFileId(ctx, fileId)
}
// LookupVolumeServerUrl safely looks up volume server URLs
func (vc *vidMapClient) LookupVolumeServerUrl(vid string) (serverUrls []string, err error) {
return vc.getStableVidMap().LookupVolumeServerUrl(vid)
}
// GetDataCenter safely retrieves the data center
func (vc *vidMapClient) GetDataCenter() string {
return vc.getStableVidMap().DataCenter
}
// Thread-safe helpers for vidMap operations
// addLocation adds a volume location
func (vc *vidMapClient) addLocation(vid uint32, location Location) {
vc.withCurrentVidMap(func(vm *vidMap) {
vm.addLocation(vid, location)
})
}
// deleteLocation removes a volume location
func (vc *vidMapClient) deleteLocation(vid uint32, location Location) {
vc.withCurrentVidMap(func(vm *vidMap) {
vm.deleteLocation(vid, location)
})
}
// addEcLocation adds an EC volume location
func (vc *vidMapClient) addEcLocation(vid uint32, location Location) {
vc.withCurrentVidMap(func(vm *vidMap) {
vm.addEcLocation(vid, location)
})
}
// deleteEcLocation removes an EC volume location
func (vc *vidMapClient) deleteEcLocation(vid uint32, location Location) {
vc.withCurrentVidMap(func(vm *vidMap) {
vm.deleteEcLocation(vid, location)
})
}
// resetVidMap resets the volume ID map
func (vc *vidMapClient) resetVidMap() {
vc.vidMapLock.Lock()
defer vc.vidMapLock.Unlock()
// Preserve the existing vidMap in the cache chain
tail := vc.vidMap
nvm := newVidMap(tail.DataCenter)
nvm.cache.Store(tail)
vc.vidMap = nvm
// Trim cache chain to vidMapCacheSize
node := tail
for i := 0; i < vc.vidMapCacheSize-1; i++ {
if node.cache.Load() == nil {
return
}
node = node.cache.Load()
}
// node is guaranteed to be non-nil after the loop
node.cache.Store(nil)
}