Files
seaweedFS/weed/cluster/lock_client.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

281 lines
9.0 KiB
Go

package cluster
import (
"context"
"fmt"
"strings"
"sync/atomic"
"time"
"github.com/seaweedfs/seaweedfs/weed/cluster/lock_manager"
"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/util"
"google.golang.org/grpc"
)
type LockClient struct {
grpcDialOption grpc.DialOption
maxLockDuration time.Duration
sleepDuration time.Duration
seedFiler pb.ServerAddress
}
func NewLockClient(grpcDialOption grpc.DialOption, seedFiler pb.ServerAddress) *LockClient {
return &LockClient{
grpcDialOption: grpcDialOption,
maxLockDuration: 5 * time.Second,
sleepDuration: 2473 * time.Millisecond,
seedFiler: seedFiler,
}
}
type LiveLock struct {
key string
renewToken string
expireAtNs int64
hostFiler pb.ServerAddress
cancelCh chan struct{}
grpcDialOption grpc.DialOption
isLocked int32 // 0 = unlocked, 1 = locked; use atomic operations
self string
lc *LockClient
owner string
lockTTL time.Duration
consecutiveFailures int // Track connection failures to trigger fallback
generation int64 // fencing token from the lock server
}
// NewShortLivedLock creates a lock with a 5-second duration
func (lc *LockClient) NewShortLivedLock(key string, owner string) (lock *LiveLock) {
lock = &LiveLock{
key: key,
hostFiler: lc.seedFiler,
cancelCh: make(chan struct{}),
expireAtNs: time.Now().Add(5 * time.Second).UnixNano(),
grpcDialOption: lc.grpcDialOption,
self: owner,
lc: lc,
}
lock.retryUntilLocked(5 * time.Second)
return
}
// StartLongLivedLock starts a goroutine to lock the key and returns immediately.
// lockTTL specifies how long the lock should be held. The renewal interval is
// automatically derived as lockTTL / 2 to ensure timely renewals.
func (lc *LockClient) StartLongLivedLock(key string, owner string, onLockOwnerChange func(newLockOwner string), lockTTL time.Duration) (lock *LiveLock) {
lock = &LiveLock{
key: key,
hostFiler: lc.seedFiler,
cancelCh: make(chan struct{}),
expireAtNs: time.Now().Add(lockTTL).UnixNano(),
grpcDialOption: lc.grpcDialOption,
self: owner,
lc: lc,
lockTTL: lockTTL,
}
if lock.lockTTL == 0 {
lock.lockTTL = lock_manager.LiveLockTTL
}
go func() {
renewInterval := lock.lockTTL / 2
isLocked := false
lockOwner := ""
for {
// Check for cancellation BEFORE attempting to lock to avoid race condition
// where Stop() is called after sleep but before lock attempt
select {
case <-lock.cancelCh:
return
default:
}
if isLocked {
if err := lock.AttemptToLock(lock.lockTTL); err != nil {
glog.V(0).Infof("Lost lock %s: %v", key, err)
isLocked = false
atomic.StoreInt32(&lock.isLocked, 0)
}
} else {
if err := lock.AttemptToLock(lock.lockTTL); err == nil {
isLocked = true
// Note: AttemptToLock already sets lock.isLocked atomically on success
}
}
if lockOwner != lock.LockOwner() && lock.LockOwner() != "" {
glog.V(0).Infof("Lock owner changed from %s to %s", lockOwner, lock.LockOwner())
onLockOwnerChange(lock.LockOwner())
lockOwner = lock.LockOwner()
}
select {
case <-lock.cancelCh:
return
default:
if isLocked {
time.Sleep(renewInterval)
} else {
time.Sleep(5 * renewInterval)
}
}
}
}()
return
}
func (lock *LiveLock) retryUntilLocked(lockDuration time.Duration) {
util.RetryUntil("create lock:"+lock.key, func() error {
return lock.AttemptToLock(lockDuration)
}, func(err error) (shouldContinue bool) {
if err != nil {
glog.Warningf("create lock %s: %s", lock.key, err)
}
return lock.renewToken == ""
})
}
func (lock *LiveLock) AttemptToLock(lockDuration time.Duration) error {
glog.V(4).Infof("LOCK: AttemptToLock key=%s owner=%s", lock.key, lock.self)
errorMessage, err := lock.doLock(lockDuration)
if err != nil {
glog.V(1).Infof("LOCK: doLock failed for key=%s: %v", lock.key, err)
time.Sleep(time.Second)
return err
}
if errorMessage != "" {
if strings.Contains(errorMessage, "lock already owned") {
glog.V(3).Infof("LOCK: doLock returned error message for key=%s: %s", lock.key, errorMessage)
} else {
glog.V(2).Infof("LOCK: doLock returned error message for key=%s: %s", lock.key, errorMessage)
}
time.Sleep(time.Second)
return fmt.Errorf("%v", errorMessage)
}
if atomic.LoadInt32(&lock.isLocked) == 0 {
// Only log when transitioning from unlocked to locked
glog.V(1).Infof("LOCK: Successfully acquired key=%s owner=%s", lock.key, lock.self)
}
atomic.StoreInt32(&lock.isLocked, 1)
return nil
}
func (lock *LiveLock) StopShortLivedLock() error {
if atomic.LoadInt32(&lock.isLocked) == 0 {
return nil
}
defer func() {
atomic.StoreInt32(&lock.isLocked, 0)
}()
return pb.WithFilerClient(false, 0, lock.hostFiler, lock.grpcDialOption, func(client filer_pb.SeaweedFilerClient) error {
_, err := client.DistributedUnlock(context.Background(), &filer_pb.UnlockRequest{
Name: lock.key,
RenewToken: lock.renewToken,
})
return err
})
}
// Stop stops a long-lived lock by closing the cancel channel and releasing the lock
func (lock *LiveLock) Stop() error {
// Close the cancel channel to stop the long-lived lock goroutine
select {
case <-lock.cancelCh:
// Already closed
default:
close(lock.cancelCh)
}
// Wait a brief moment for the goroutine to see the closed channel
// This reduces the race condition window where the goroutine might
// attempt one more lock operation after we've released the lock
time.Sleep(10 * time.Millisecond)
// Also release the lock if held
// Note: We intentionally don't clear renewToken here because
// StopShortLivedLock needs it to properly unlock
return lock.StopShortLivedLock()
}
func (lock *LiveLock) doLock(lockDuration time.Duration) (errorMessage string, err error) {
glog.V(4).Infof("LOCK: doLock calling DistributedLock - key=%s filer=%s owner=%s",
lock.key, lock.hostFiler, lock.self)
previousHostFiler := lock.hostFiler
previousOwner := lock.owner
err = pb.WithFilerClient(false, 0, lock.hostFiler, lock.grpcDialOption, func(client filer_pb.SeaweedFilerClient) error {
resp, err := client.DistributedLock(context.Background(), &filer_pb.LockRequest{
Name: lock.key,
SecondsToLock: int64(lockDuration.Seconds()),
RenewToken: lock.renewToken,
IsMoved: false,
Owner: lock.self,
})
glog.V(4).Infof("LOCK: DistributedLock response - key=%s err=%v", lock.key, err)
if err == nil && resp != nil {
lock.renewToken = resp.RenewToken
if resp.Generation > 0 {
atomic.StoreInt64(&lock.generation, resp.Generation)
}
lock.consecutiveFailures = 0 // Reset failure counter on success
glog.V(4).Infof("LOCK: Got renewToken for key=%s", lock.key)
} else {
//this can be retried. Need to remember the last valid renewToken
lock.renewToken = ""
glog.V(1).Infof("LOCK: Cleared renewToken for key=%s (err=%v)", lock.key, err)
}
if resp != nil {
errorMessage = resp.Error
if resp.LockHostMovedTo != "" && resp.LockHostMovedTo != string(previousHostFiler) {
// Only log if the host actually changed
glog.V(2).Infof("LOCK: Host changed from %s to %s for key=%s", previousHostFiler, resp.LockHostMovedTo, lock.key)
lock.hostFiler = pb.ServerAddress(resp.LockHostMovedTo)
// Don't update seedFiler - keep original for fallback
} else if resp.LockHostMovedTo != "" {
lock.hostFiler = pb.ServerAddress(resp.LockHostMovedTo)
}
if resp.LockOwner != "" && resp.LockOwner != previousOwner {
// Only log if the owner actually changed
glog.V(2).Infof("LOCK: Owner changed from %s to %s for key=%s", previousOwner, resp.LockOwner, lock.key)
lock.owner = resp.LockOwner
} else if resp.LockOwner != "" {
lock.owner = resp.LockOwner
} else if previousOwner != "" {
glog.V(2).Infof("LOCK: Owner cleared for key=%s", lock.key)
lock.owner = ""
}
}
return err
})
if err != nil && lock.hostFiler != lock.lc.seedFiler {
lock.consecutiveFailures++
// Fall back to seed filer after 3 consecutive connection failures
if lock.consecutiveFailures >= 3 {
glog.V(0).Infof("LOCK: Connection failed %d times for key=%s filer=%s, falling back to seed filer=%s",
lock.consecutiveFailures, lock.key, lock.hostFiler, lock.lc.seedFiler)
lock.hostFiler = lock.lc.seedFiler
lock.consecutiveFailures = 0
lock.renewToken = ""
}
}
return
}
func (lock *LiveLock) LockOwner() string {
return lock.owner
}
// Generation returns the fencing token for this lock.
// It increments on each fresh acquisition and stays the same on renewal.
func (lock *LiveLock) Generation() int64 {
return atomic.LoadInt64(&lock.generation)
}
// IsLocked returns true if this instance currently holds the lock
func (lock *LiveLock) IsLocked() bool {
return atomic.LoadInt32(&lock.isLocked) == 1
}