You can not select more than 25 topics Topics must start with a letter or number, can include dashes ('-') and can be up to 35 characters long.
 
 
 
 
 
 

404 lines
15 KiB

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)
}