Files
seaweedFS/weed/wdclient/masterclient.go
Chris Lu 75a6a34528 dlm: resilient distributed locks via consistent hashing + backup replication (#8860)
* dlm: replace modulo hashing with consistent hash ring

Introduce HashRing with virtual nodes (CRC32-based consistent hashing)
to replace the modulo-based hashKeyToServer. When a filer node is
removed, only keys that hashed to that node are remapped to the next
server on the ring, leaving all other mappings stable. This is the
foundation for backup replication — the successor on the ring is
always the natural takeover node.

* dlm: add Generation and IsBackup fields to Lock

Lock now carries IsBackup (whether this node holds the lock as a backup
replica) and Generation (a monotonic fencing token that increments on
each fresh acquisition, stays the same on renewal). Add helper methods:
AllLocks, PromoteLock, DemoteLock, InsertBackupLock, RemoveLock, GetLock.

* dlm: add ReplicateLock RPC and generation/is_backup proto fields

Add generation field to LockResponse for fencing tokens.
Add generation and is_backup fields to Lock message.
Add ReplicateLock RPC for primary-to-backup lock replication.
Add ReplicateLockRequest/ReplicateLockResponse messages.

* dlm: add async backup replication to DistributedLockManager

Route lock/unlock via consistent hash ring's GetPrimaryAndBackup().
After a successful lock or unlock on the primary, asynchronously
replicate the operation to the backup server via ReplicateFunc
callback. Single-server deployments skip replication.

* dlm: add ReplicateLock handler and backup-aware topology changes

Add ReplicateLock gRPC handler for primary-to-backup replication.
Revise OnDlmChangeSnapshot to handle three cases on topology change:
- Promote backup locks when this node becomes primary
- Demote primary locks when this node becomes backup
- Transfer locks when this node is neither primary nor backup
Wire up SetupDlmReplication during filer server initialization.

* dlm: expose generation fencing token in lock client

LiveLock now captures the generation from LockResponse and exposes it
via Generation() method. Consumers can use this as a fencing token to
detect stale lock holders.

* dlm: update empty folder cleaner to use consistent hash ring

Replace local modulo-based hashKeyToServer with LockRing.GetPrimary()
which uses the shared consistent hash ring for folder ownership.

* dlm: add unit tests for consistent hash ring

Test basic operations, consistency on server removal (only keys from
removed server move), backup-is-successor property (backup becomes
new primary when primary is removed), and key distribution balance.

* dlm: add integration tests for lock replication failure scenarios

Test cases:
- Primary crash with backup promotion (backup has valid token)
- Backup crash with primary continuing
- Both primary and backup crash (lock lost, re-acquirable)
- Rolling restart across all nodes
- Generation fencing token increments on new acquisition
- Replication failure (primary still works independently)
- Unlock replicates deletion to backup
- Lock survives server addition (topology change)
- Consistent hashing minimal disruption (only removed server's keys move)

* dlm: address PR review findings

1. Causal replication ordering: Add per-lock sequence number (Seq) that
   increments on every mutation. Backup rejects incoming mutations with
   seq <= current seq, preventing stale async replications from
   overwriting newer state. Unlock replication also carries seq and is
   rejected if stale.

2. Demote-after-handoff: OnDlmChangeSnapshot now transfers the lock to
   the new primary first and only demotes to backup after a successful
   TransferLocks RPC. If the transfer fails, the lock stays as primary
   on this node.

3. SetSnapshot candidateServers leak: Replace the candidateServers map
   entirely instead of appending, so removed servers don't linger.

4. TransferLocks preserves Generation and Seq: InsertLock now accepts
   generation and seq parameters. After accepting a transferred lock,
   the receiving node re-replicates to its backup.

5. Rolling restart test: Add re-replication step after promotion and
   assert survivedCount > 0. Add TestDLM_StaleReplicationRejected.

6. Mixed-version upgrade note: Add comment on HashRing documenting that
   all filer nodes must be upgraded together.

* dlm: serve renewals locally during transfer window on node join

When a new node joins and steals hash ranges from surviving nodes,
there's a window between ring update and lock transfer where the
client gets redirected to a node that doesn't have the lock yet.

Fix: if the ring says primary != self but we still hold the lock
locally (non-backup, matching token), serve the renewal/unlock here
rather than redirecting. The lock will be transferred by
OnDlmChangeSnapshot, and subsequent requests will go to the new
primary once the transfer completes.

Add tests:
- TestDLM_NodeDropAndJoin_OwnershipDisruption: measures disruption
  when a node drops and a new one joins (14/100 surviving-node locks
  disrupted, all handled by transfer logic)
- TestDLM_RenewalDuringTransferWindow: verifies renewal succeeds on
  old primary during the transfer window

* dlm: master-managed lock ring with stabilization batching

The master now owns the lock ring membership. Instead of filers
independently reacting to individual ClusterNodeUpdate add/remove
events, the master:

1. Tracks filer membership in LockRingManager
2. Batches rapid changes with a 1-second stabilization timer
   (e.g., a node drop + join within 1 second → single ring update)
3. Broadcasts the complete ring snapshot atomically via the new
   LockRingUpdate message in KeepConnectedResponse

Filers receive the ring as a complete snapshot and apply it via
SetSnapshot, ensuring all filers converge to the same ring state
without intermediate churn.

This eliminates the double-churn problem where a rapid drop+join
would fire two separate ring mutations, each triggering lock
transfers and disrupting ownership on surviving nodes.

* dlm: track ring version, reject stale updates, remove dead code

SetSnapshot now takes a version parameter from the master. Stale
updates (version < current) are rejected, preventing reordered
messages from overwriting a newer ring state. Version 0 is always
accepted for bootstrap.

Remove AddServer/RemoveServer from LockRing — the ring is now
exclusively managed by the master via SetSnapshot. Remove the
candidateServers map that was only used by those methods.

* dlm: fix SelectLocks data race, advance generation on backup insert

- SelectLocks: change RLock to Lock since the function deletes map
  entries, which is a write operation and causes a data race under RLock.
- InsertBackupLock: advance nextGeneration to at least the incoming
  generation so that after failover promotion, new lock acquisitions
  get a generation strictly greater than any replicated lock.
- Bump replication failure log from V(1) to Warningf for production
  visibility.

* dlm: fix SetSnapshot race, test reliability, timer edge cases

- SetSnapshot: hold LockRing lock through both version update and
  Ring.SetServers() so they're atomic. Prevents a concurrent caller
  from seeing the new version but applying stale servers.
- Transfer window test: search for a key that actually moves primary
  when filer4 joins, instead of relying on a fixed key that may not.
- renewLock redirect: pass the existing token to the new primary
  instead of empty string, so redirected renewals work correctly.
- scheduleBroadcast: check timer.Stop() return value. If the timer
  already fired, the callback picks up latest state.
- FlushPending: only broadcast if timer.Stop() returns true (timer
  was still pending). If false, the callback is already running.
- Fix test comment: "idempotent" → "accepted, state-changing".

* dlm: use wall-clock nanoseconds for lock ring version

The lock ring version was an in-memory counter that reset to 0 on
master restart. A filer that had seen version 5 would reject version 1
from the restarted master.

Fix: use time.Now().UnixNano() as the version. This survives master
restarts without persistence — the restarted master produces a
version greater than any pre-restart value.

* dlm: treat expired lock owners as missing

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

* dlm: reject stale lock transfers

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

* dlm: order replication by generation

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

* dlm: bootstrap lock ring on reconnect

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

---------

Co-authored-by: Copilot <223556219+Copilot@users.noreply.github.com>
2026-03-30 23:29:56 -07:00

522 lines
19 KiB
Go

package wdclient
import (
"context"
"errors"
"fmt"
"math/rand"
"strconv"
"strings"
"sync"
"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/master_pb"
"github.com/seaweedfs/seaweedfs/weed/stats"
"github.com/seaweedfs/seaweedfs/weed/util"
"github.com/seaweedfs/seaweedfs/weed/util/version"
)
// masterVolumeProvider implements VolumeLocationProvider by querying master
// This is rarely called since master pushes updates proactively via KeepConnected stream
type masterVolumeProvider struct {
masterClient *MasterClient
}
func isCanceledErr(err error) bool {
if err == nil {
return false
}
if errors.Is(err, context.Canceled) || errors.Is(err, context.DeadlineExceeded) {
return true
}
if statusErr, ok := status.FromError(err); ok {
switch statusErr.Code() {
case codes.Canceled, codes.DeadlineExceeded:
return true
}
}
return false
}
// LookupVolumeIds queries the master for volume locations (fallback when cache misses).
// Returns partial results with aggregated errors for volumes that failed.
// Retries on codes.Unavailable (e.g. master warming up after restart) with backoff.
func (p *masterVolumeProvider) LookupVolumeIds(ctx context.Context, volumeIds []string) (map[string][]Location, error) {
var result map[string][]Location
var lookupErrors []error
glog.V(2).Infof("Looking up %d volumes from master: %v", len(volumeIds), volumeIds)
retryErr := util.RetryWithBackoff(ctx, "lookup", 30*time.Second,
func(err error) bool {
st, ok := status.FromError(err)
return ok && st.Code() == codes.Unavailable
},
func() error {
result = make(map[string][]Location)
lookupErrors = nil
// Per-attempt timeout bounds both master resolution and the RPC
// so a single attempt cannot consume the entire retry budget.
timeoutCtx, cancel := context.WithTimeout(ctx, p.masterClient.grpcTimeout)
defer cancel()
master := p.masterClient.GetMaster(timeoutCtx)
if master == "" {
if ctx.Err() != nil {
return ctx.Err()
}
return status.Errorf(codes.Unavailable, "no master available")
}
return pb.WithMasterClient(false, master, p.masterClient.grpcDialOption, false, func(client master_pb.SeaweedClient) error {
resp, err := client.LookupVolume(timeoutCtx, &master_pb.LookupVolumeRequest{
VolumeOrFileIds: volumeIds,
})
if err != nil {
return err
}
for _, vidLoc := range resp.VolumeIdLocations {
// Preserve per-volume errors from master response
// These could indicate misconfiguration, volume deletion, etc.
if 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
}
// Parse volume ID from response
parts := strings.Split(vidLoc.VolumeOrFileId, ",")
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
}
var locations []Location
for _, masterLoc := range vidLoc.Locations {
loc := Location{
Url: masterLoc.Url,
PublicUrl: masterLoc.PublicUrl,
GrpcPort: int(masterLoc.GrpcPort),
DataCenter: masterLoc.DataCenter,
}
// Update cache with the location
p.masterClient.addLocation(uint32(vid), loc)
locations = append(locations, loc)
}
if len(locations) > 0 {
result[vidOnly] = locations
}
}
return nil
})
})
if retryErr != nil {
return nil, retryErr
}
// 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...))
}
glog.V(3).Infof("MasterClient: looked up %d volumes, found %d", len(volumeIds), len(result))
return result, nil
}
// 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
OnLockRingUpdate func(update *master_pb.LockRingUpdate)
OnLockRingUpdateLock sync.RWMutex
}
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) SetOnPeerUpdateFn(onPeerUpdate func(update *master_pb.ClusterNodeUpdate, startFrom time.Time)) {
mc.OnPeerUpdateLock.Lock()
mc.OnPeerUpdate = onPeerUpdate
mc.OnPeerUpdateLock.Unlock()
}
func (mc *MasterClient) SetOnLockRingUpdateFn(fn func(update *master_pb.LockRingUpdate)) {
mc.OnLockRingUpdateLock.Lock()
mc.OnLockRingUpdate = fn
mc.OnLockRingUpdateLock.Unlock()
}
func (mc *MasterClient) tryAllMasters(ctx context.Context) {
var nextHintedLeader pb.ServerAddress
failedMasters := make(map[pb.ServerAddress]struct{})
mc.masters.RefreshBySrvIfAvailable()
for _, master := range mc.masters.GetInstances() {
if _, failed := failedMasters[master]; failed {
continue
}
nextHintedLeader = mc.tryConnectToMaster(ctx, master)
for nextHintedLeader != "" {
if _, failed := failedMasters[nextHintedLeader]; failed {
break // don't follow redirect to a known-unreachable master
}
select {
case <-ctx.Done():
glog.V(0).Infof("Connection attempt to all masters stopped: %v", ctx.Err())
return
default:
target := nextHintedLeader
nextHintedLeader = mc.tryConnectToMaster(ctx, target)
if nextHintedLeader == "" {
// connection to target failed; remember it so we skip
// stale redirects pointing back to it this cycle
failedMasters[target] = struct{}{}
}
}
}
mc.setCurrentMaster("")
}
}
func (mc *MasterClient) tryConnectToMaster(ctx context.Context, master pb.ServerAddress) (nextHintedLeader pb.ServerAddress) {
glog.V(1).Infof("%s.%s masterClient Connecting to master %v", mc.FilerGroup, mc.clientType, master)
stats.MasterClientConnectCounter.WithLabelValues("total").Inc()
connectStartTime := time.Now()
gprcErr := pb.WithMasterClient(true, master, mc.grpcDialOption, false, func(client master_pb.SeaweedClient) error {
ctx, cancel := context.WithCancel(ctx)
defer cancel()
stream, err := client.KeepConnected(ctx)
if err != nil {
glog.V(1).Infof("%s.%s masterClient failed to keep connected to %s: %v", mc.FilerGroup, mc.clientType, master, err)
stats.MasterClientConnectCounter.WithLabelValues(stats.FailedToKeepConnected).Inc()
return err
}
glog.V(0).Infof("%s.%s masterClient gRPC stream established to %s in %v", mc.FilerGroup, mc.clientType, master, time.Since(connectStartTime))
if err = stream.Send(&master_pb.KeepConnectedRequest{
FilerGroup: mc.FilerGroup,
DataCenter: mc.GetDataCenter(),
Rack: mc.rack,
ClientType: mc.clientType,
ClientAddress: string(mc.clientHost),
Version: version.Version(),
}); err != nil {
glog.V(0).Infof("%s.%s masterClient failed to send to %s: %v", mc.FilerGroup, mc.clientType, master, err)
stats.MasterClientConnectCounter.WithLabelValues(stats.FailedToSend).Inc()
return err
}
glog.V(1).Infof("%s.%s masterClient Connected to %v", mc.FilerGroup, mc.clientType, master)
resp, err := stream.Recv()
if err != nil {
canceled := isCanceledErr(err) || ctx.Err() != nil
if canceled {
glog.V(1).Infof("%s.%s masterClient stream closed from %s: %v", mc.FilerGroup, mc.clientType, master, err)
} else {
glog.V(0).Infof("%s.%s masterClient failed to receive from %s: %v", mc.FilerGroup, mc.clientType, master, err)
stats.MasterClientConnectCounter.WithLabelValues(stats.FailedToReceive).Inc()
}
return err
}
// check if it is the leader to determine whether to reset the vidMap
if resp.VolumeLocation != nil {
if resp.VolumeLocation.Leader != "" && string(master) != resp.VolumeLocation.Leader {
glog.V(0).Infof("master %v redirected to leader %v", master, resp.VolumeLocation.Leader)
nextHintedLeader = pb.ServerAddress(resp.VolumeLocation.Leader)
stats.MasterClientConnectCounter.WithLabelValues(stats.RedirectedToLeader).Inc()
return nil
}
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)
for {
resp, err := stream.Recv()
if err != nil {
canceled := isCanceledErr(err) || ctx.Err() != nil
if canceled {
glog.V(1).Infof("%s.%s masterClient stream closed from %s: %v", mc.FilerGroup, mc.clientType, master, err)
} else {
glog.V(0).Infof("%s.%s masterClient failed to receive from %s: %v", mc.FilerGroup, mc.clientType, master, err)
stats.MasterClientConnectCounter.WithLabelValues(stats.FailedToReceive).Inc()
}
return err
}
if resp.VolumeLocation != nil {
// 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)
stats.MasterClientConnectCounter.WithLabelValues(stats.RedirectedToLeader).Inc()
return nil
}
mc.updateVidMap(resp)
}
if resp.ClusterNodeUpdate != nil {
update := resp.ClusterNodeUpdate
mc.OnPeerUpdateLock.RLock()
if mc.OnPeerUpdate != nil {
if update.FilerGroup == mc.FilerGroup {
if update.IsAdd {
glog.V(0).Infof("+ %s@%s noticed %s.%s %s\n", mc.clientType, mc.clientHost, update.FilerGroup, update.NodeType, update.Address)
} else {
glog.V(0).Infof("- %s@%s noticed %s.%s %s\n", mc.clientType, mc.clientHost, update.FilerGroup, update.NodeType, update.Address)
}
stats.MasterClientConnectCounter.WithLabelValues(stats.OnPeerUpdate).Inc()
mc.OnPeerUpdate(update, time.Now())
}
}
mc.OnPeerUpdateLock.RUnlock()
}
if resp.LockRingUpdate != nil {
update := resp.LockRingUpdate
mc.OnLockRingUpdateLock.RLock()
if mc.OnLockRingUpdate != nil {
if update.FilerGroup == mc.FilerGroup {
glog.V(0).Infof("LockRing: %s@%s received ring update v%d: %v", mc.clientType, mc.clientHost, update.Version, update.Servers)
mc.OnLockRingUpdate(update)
}
}
mc.OnLockRingUpdateLock.RUnlock()
}
if err := ctx.Err(); err != nil {
if isCanceledErr(err) {
glog.V(1).Infof("Connection attempt to master stopped: %v", err)
} else {
glog.V(0).Infof("Connection attempt to master stopped: %v", err)
}
return err
}
}
})
if gprcErr != nil {
if isCanceledErr(gprcErr) || ctx.Err() != nil {
glog.V(1).Infof("%s.%s masterClient connection closed to %v: %v", mc.FilerGroup, mc.clientType, master, gprcErr)
return nextHintedLeader
}
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 nextHintedLeader
}
func (mc *MasterClient) updateVidMap(resp *master_pb.KeepConnectedResponse) {
if resp.VolumeLocation.IsEmptyUrl() {
glog.V(0).Infof("updateVidMap ignore short heartbeat: %+v", resp)
return
}
// process new volume location
loc := Location{
Url: resp.VolumeLocation.Url,
PublicUrl: resp.VolumeLocation.PublicUrl,
DataCenter: resp.VolumeLocation.DataCenter,
GrpcPort: int(resp.VolumeLocation.GrpcPort),
}
for _, newVid := range resp.VolumeLocation.NewVids {
glog.V(2).Infof("%s.%s: %s masterClient adds volume %d", mc.FilerGroup, mc.clientType, loc.Url, newVid)
mc.addLocation(newVid, loc)
}
for _, deletedVid := range resp.VolumeLocation.DeletedVids {
glog.V(2).Infof("%s.%s: %s masterClient removes volume %d", mc.FilerGroup, mc.clientType, loc.Url, deletedVid)
mc.deleteLocation(deletedVid, loc)
}
for _, newEcVid := range resp.VolumeLocation.NewEcVids {
glog.V(2).Infof("%s.%s: %s masterClient adds ec volume %d", mc.FilerGroup, mc.clientType, loc.Url, newEcVid)
mc.addEcLocation(newEcVid, loc)
}
for _, deletedEcVid := range resp.VolumeLocation.DeletedEcVids {
glog.V(2).Infof("%s.%s: %s masterClient removes ec volume %d", mc.FilerGroup, mc.clientType, loc.Url, deletedEcVid)
mc.deleteEcLocation(deletedEcVid, loc)
}
glog.V(1).Infof("updateVidMap(%s) %s.%s: %s volume add: %d, del: %d, add ec: %d del ec: %d",
resp.VolumeLocation.DataCenter, mc.FilerGroup, mc.clientType, loc.Url,
len(resp.VolumeLocation.NewVids), len(resp.VolumeLocation.DeletedVids),
len(resp.VolumeLocation.NewEcVids), len(resp.VolumeLocation.DeletedEcVids))
}
func (mc *MasterClient) WithClient(streamingMode bool, fn func(client master_pb.SeaweedClient) error) error {
getMasterF := func() pb.ServerAddress {
return mc.GetMaster(context.Background())
}
return mc.WithClientCustomGetMaster(getMasterF, streamingMode, fn)
}
func (mc *MasterClient) WithClientCustomGetMaster(getMasterF func() pb.ServerAddress, streamingMode bool, fn func(client master_pb.SeaweedClient) error) error {
return util.Retry("master grpc", func() error {
return pb.WithMasterClient(streamingMode, getMasterF(), mc.grpcDialOption, false, func(client master_pb.SeaweedClient) error {
return fn(client)
})
})
}
func (mc *MasterClient) getCurrentMaster() pb.ServerAddress {
mc.currentMasterLock.RLock()
defer mc.currentMasterLock.RUnlock()
return mc.currentMaster
}
func (mc *MasterClient) setCurrentMaster(master pb.ServerAddress) {
mc.currentMasterLock.Lock()
mc.currentMaster = master
mc.currentMasterLock.Unlock()
}
// 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()
}
// 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()
}
// 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
}
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)
}
// Use select with time.After to respect context cancellation during sleep
sleepDuration := time.Duration(rand.Int31n(200)) * time.Millisecond
select {
case <-ctx.Done():
return
case <-time.After(sleepDuration):
// continue to next iteration
}
}
}
}
func (mc *MasterClient) KeepConnectedToMaster(ctx context.Context) {
glog.V(0).Infof("%s.%s masterClient bootstraps with masters %v", mc.FilerGroup, mc.clientType, mc.masters)
reconnectCount := 0
for {
select {
case <-ctx.Done():
if isCanceledErr(ctx.Err()) {
glog.V(1).Infof("Connection to masters stopped: %v", ctx.Err())
} else {
glog.V(0).Infof("Connection to masters stopped: %v", ctx.Err())
}
return
default:
reconnectStart := time.Now()
if reconnectCount > 0 {
glog.V(0).Infof("%s.%s masterClient reconnection attempt #%d", mc.FilerGroup, mc.clientType, reconnectCount)
}
mc.tryAllMasters(ctx)
reconnectCount++
glog.V(1).Infof("%s.%s masterClient connection cycle completed in %v, sleeping before retry",
mc.FilerGroup, mc.clientType, time.Since(reconnectStart))
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
}