diff --git a/other/java/client/src/main/proto/filer.proto b/other/java/client/src/main/proto/filer.proto index 47f50a385..d7067def0 100644 --- a/other/java/client/src/main/proto/filer.proto +++ b/other/java/client/src/main/proto/filer.proto @@ -84,6 +84,8 @@ service SeaweedFiler { // distributed lock management internal use only rpc TransferLocks(TransferLocksRequest) returns (TransferLocksResponse) { } + rpc ReplicateLock(ReplicateLockRequest) returns (ReplicateLockResponse) { + } } ////////////////////////////////////////////////// @@ -522,6 +524,7 @@ message LockResponse { string lock_owner = 2; string lock_host_moved_to = 3; string error = 4; + int64 generation = 5; } message UnlockRequest { string name = 1; @@ -544,12 +547,26 @@ message Lock { string renew_token = 2; int64 expired_at_ns = 3; string owner = 4; + int64 generation = 5; + bool is_backup = 6; + int64 seq = 7; } message TransferLocksRequest { repeated Lock locks = 1; } message TransferLocksResponse { } +message ReplicateLockRequest { + string name = 1; + string renew_token = 2; + int64 expired_at_ns = 3; + string owner = 4; + int64 generation = 5; + bool is_unlock = 6; + int64 seq = 7; +} +message ReplicateLockResponse { +} ////////////////////////////////////////////////// // StreamMutateEntry: ordered bidirectional streaming for all filer mutations. diff --git a/weed/cluster/lock_client.go b/weed/cluster/lock_client.go index 5e635eeae..dc5a6ac77 100644 --- a/weed/cluster/lock_client.go +++ b/weed/cluster/lock_client.go @@ -43,7 +43,8 @@ type LiveLock struct { lc *LockClient owner string lockTTL time.Duration - consecutiveFailures int // Track connection failures to trigger fallback + 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 @@ -214,6 +215,9 @@ func (lock *LiveLock) doLock(lockDuration time.Duration) (errorMessage string, e 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 { @@ -264,6 +268,12 @@ 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 diff --git a/weed/cluster/lock_manager/distributed_lock_manager.go b/weed/cluster/lock_manager/distributed_lock_manager.go index 31f8b979f..8269ce384 100644 --- a/weed/cluster/lock_manager/distributed_lock_manager.go +++ b/weed/cluster/lock_manager/distributed_lock_manager.go @@ -13,10 +13,17 @@ const LiveLockTTL = time.Second * 7 var NoLockServerError = fmt.Errorf("no lock server found") +// ReplicateFunc is called to replicate a lock operation to a backup server. +// The caller (filer server) provides this to avoid a circular dependency. +// seq is a per-lock monotonic sequence number for causal ordering — the backup +// rejects mutations with seq <= its current seq for that key. +type ReplicateFunc func(server pb.ServerAddress, key string, expiredAtNs int64, token string, owner string, generation int64, seq int64, isUnlock bool) + type DistributedLockManager struct { - lockManager *LockManager - LockRing *LockRing - Host pb.ServerAddress + lockManager *LockManager + LockRing *LockRing + Host pb.ServerAddress + ReplicateFn ReplicateFunc // set by filer server after creation } func NewDistributedLockManager(host pb.ServerAddress) *DistributedLockManager { @@ -27,35 +34,51 @@ func NewDistributedLockManager(host pb.ServerAddress) *DistributedLockManager { } } -func (dlm *DistributedLockManager) LockWithTimeout(key string, expiredAtNs int64, token string, owner string) (lockOwner string, renewToken string, movedTo pb.ServerAddress, err error) { - movedTo, err = dlm.findLockOwningFiler(key) - if err != nil { +func (dlm *DistributedLockManager) LockWithTimeout(key string, expiredAtNs int64, token string, owner string) (lockOwner string, renewToken string, generation int64, movedTo pb.ServerAddress, err error) { + primary, _ := dlm.LockRing.GetPrimaryAndBackup(key) + if primary == "" { + err = NoLockServerError return } - if movedTo != dlm.Host { + if primary != dlm.Host { + // If this is a renewal (non-empty token) and we still hold the lock locally, + // serve it here rather than redirecting. This handles the window between + // ring update and lock transfer completion — the old primary remains + // authoritative for locks it still holds. + if token != "" { + if lock, found := dlm.lockManager.GetLock(key); found && !lock.IsBackup && lock.Token == token { + var seq int64 + lockOwner, renewToken, generation, seq, err = dlm.lockManager.Lock(key, expiredAtNs, token, owner) + if err == nil && renewToken != "" { + dlm.replicateToBackup(key, expiredAtNs, renewToken, owner, generation, seq, false) + } + return + } + } + movedTo = primary return } - lockOwner, renewToken, err = dlm.lockManager.Lock(key, expiredAtNs, token, owner) - return -} - -func (dlm *DistributedLockManager) findLockOwningFiler(key string) (movedTo pb.ServerAddress, err error) { - servers := dlm.LockRing.GetSnapshot() - if servers == nil { - err = NoLockServerError - return + var seq int64 + lockOwner, renewToken, generation, seq, err = dlm.lockManager.Lock(key, expiredAtNs, token, owner) + if err == nil && renewToken != "" { + dlm.replicateToBackup(key, expiredAtNs, renewToken, owner, generation, seq, false) } - - movedTo = hashKeyToServer(key, servers) return } func (dlm *DistributedLockManager) FindLockOwner(key string) (owner string, movedTo pb.ServerAddress, err error) { - movedTo, err = dlm.findLockOwningFiler(key) - if err != nil { + primary, _ := dlm.LockRing.GetPrimaryAndBackup(key) + if primary == "" { + err = NoLockServerError return } - if movedTo != dlm.Host { + if primary != dlm.Host { + // If we still hold this lock locally, serve it here + if lock, found := dlm.lockManager.GetLock(key); found && !lock.IsBackup { + owner = lock.Owner + return + } + movedTo = primary servers := dlm.LockRing.GetSnapshot() glog.V(0).Infof("lock %s not on current %s but on %s from %v", key, dlm.Host, movedTo, servers) return @@ -65,26 +88,60 @@ func (dlm *DistributedLockManager) FindLockOwner(key string) (owner string, move } func (dlm *DistributedLockManager) Unlock(key string, token string) (movedTo pb.ServerAddress, err error) { - servers := dlm.LockRing.GetSnapshot() - if servers == nil { + primary, _ := dlm.LockRing.GetPrimaryAndBackup(key) + if primary == "" { err = NoLockServerError return } - - server := hashKeyToServer(key, servers) - if server != dlm.Host { - movedTo = server + if primary != dlm.Host { + // If we still hold this lock locally, serve the unlock here + if lock, found := dlm.lockManager.GetLock(key); found && !lock.IsBackup && lock.Token == token { + var isUnlocked bool + var generation int64 + var seq int64 + isUnlocked, generation, seq, err = dlm.lockManager.Unlock(key, token) + if isUnlocked { + dlm.replicateToBackup(key, 0, "", "", generation, seq, true) + } + return + } + movedTo = primary return } - _, err = dlm.lockManager.Unlock(key, token) + var isUnlocked bool + var generation int64 + var seq int64 + isUnlocked, generation, seq, err = dlm.lockManager.Unlock(key, token) + if isUnlocked { + dlm.replicateToBackup(key, 0, "", "", generation, seq, true) + } return } -// InsertLock is used to insert a lock to a server unconditionally -// It is used when a server is down and the lock is moved to another server -func (dlm *DistributedLockManager) InsertLock(key string, expiredAtNs int64, token string, owner string) { - dlm.lockManager.InsertLock(key, expiredAtNs, token, owner) +// InsertLock is used to insert a lock to a server unconditionally. +// It is used when a server is down and the lock is moved to another server. +// After inserting, it replicates to the backup for this key. +func (dlm *DistributedLockManager) InsertLock(key string, expiredAtNs int64, token string, owner string, generation int64, seq int64) { + if dlm.lockManager.InsertLock(key, expiredAtNs, token, owner, generation, seq) { + dlm.replicateToBackup(key, expiredAtNs, token, owner, generation, seq, false) + } +} + +// InsertBackupLock inserts a lock as a backup copy, rejecting stale seq +func (dlm *DistributedLockManager) InsertBackupLock(key string, expiredAtNs int64, token string, owner string, generation int64, seq int64) { + dlm.lockManager.InsertBackupLock(key, expiredAtNs, token, owner, generation, seq) +} + +// RemoveBackupLock removes a backup lock unconditionally +func (dlm *DistributedLockManager) RemoveBackupLock(key string) { + dlm.lockManager.RemoveLock(key) } + +// RemoveBackupLockIfSeq removes a local copy only if the incoming mutation is not older. +func (dlm *DistributedLockManager) RemoveBackupLockIfSeq(key string, generation int64, seq int64) { + dlm.lockManager.RemoveBackupLockIfSeq(key, generation, seq) +} + func (dlm *DistributedLockManager) SelectNotOwnedLocks(servers []pb.ServerAddress) (locks []*Lock) { return dlm.lockManager.SelectLocks(func(key string) bool { server := hashKeyToServer(key, servers) @@ -96,9 +153,40 @@ func (dlm *DistributedLockManager) CalculateTargetServer(key string, servers []p } func (dlm *DistributedLockManager) IsLocal(key string) bool { - servers := dlm.LockRing.GetSnapshot() - if len(servers) <= 1 { + primary := dlm.LockRing.GetPrimary(key) + if primary == "" { return true } - return hashKeyToServer(key, servers) == dlm.Host + return primary == dlm.Host +} + +// AllLocks returns all non-expired locks on this node +func (dlm *DistributedLockManager) AllLocks() []*Lock { + return dlm.lockManager.AllLocks() +} + +// PromoteLock promotes a backup lock to primary +func (dlm *DistributedLockManager) PromoteLock(key string) bool { + return dlm.lockManager.PromoteLock(key) +} + +// DemoteLock demotes a primary lock to backup +func (dlm *DistributedLockManager) DemoteLock(key string) bool { + return dlm.lockManager.DemoteLock(key) +} + +// GetLock returns a copy of a lock if it exists +func (dlm *DistributedLockManager) GetLock(key string) (*Lock, bool) { + return dlm.lockManager.GetLock(key) +} + +// replicateToBackup asynchronously replicates a lock operation to the backup server +func (dlm *DistributedLockManager) replicateToBackup(key string, expiredAtNs int64, token string, owner string, generation int64, seq int64, isUnlock bool) { + _, backup := dlm.LockRing.GetPrimaryAndBackup(key) + if backup == "" { + return // single-server deployment, no backup + } + if dlm.ReplicateFn != nil { + go dlm.ReplicateFn(backup, key, expiredAtNs, token, owner, generation, seq, isUnlock) + } } diff --git a/weed/cluster/lock_manager/distributed_lock_manager_test.go b/weed/cluster/lock_manager/distributed_lock_manager_test.go new file mode 100644 index 000000000..652532fba --- /dev/null +++ b/weed/cluster/lock_manager/distributed_lock_manager_test.go @@ -0,0 +1,710 @@ +package lock_manager + +import ( + "fmt" + "sync" + "testing" + "time" + + "github.com/seaweedfs/seaweedfs/weed/pb" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" +) + +// testCluster simulates a cluster of filer nodes with DLMs. +// It wires up ReplicateFn so that replication calls arrive at the +// correct peer's DLM, enabling end-to-end backup testing without gRPC. +type testCluster struct { + mu sync.Mutex + nodes map[pb.ServerAddress]*DistributedLockManager +} + +func newTestCluster(hosts ...pb.ServerAddress) *testCluster { + c := &testCluster{nodes: make(map[pb.ServerAddress]*DistributedLockManager)} + servers := make([]pb.ServerAddress, len(hosts)) + copy(servers, hosts) + + for _, host := range hosts { + dlm := NewDistributedLockManager(host) + dlm.LockRing.SetSnapshot(servers, 0) + c.nodes[host] = dlm + } + + // Wire up replication: each node's ReplicateFn calls the backup's DLM directly + for _, dlm := range c.nodes { + d := dlm // capture + d.ReplicateFn = func(server pb.ServerAddress, key string, expiredAtNs int64, token string, owner string, generation int64, seq int64, isUnlock bool) { + c.mu.Lock() + target, ok := c.nodes[server] + c.mu.Unlock() + if !ok { + return // server is down + } + if isUnlock { + target.RemoveBackupLockIfSeq(key, generation, seq) + } else { + target.InsertBackupLock(key, expiredAtNs, token, owner, generation, seq) + } + } + } + + return c +} + +func (c *testCluster) removeNode(host pb.ServerAddress) { + c.mu.Lock() + delete(c.nodes, host) + c.mu.Unlock() + + // Update all remaining nodes' rings + remaining := c.getServers() + for _, dlm := range c.getNodes() { + dlm.LockRing.SetSnapshot(remaining, 0) + } +} + +func (c *testCluster) addNode(host pb.ServerAddress) { + c.mu.Lock() + dlm := NewDistributedLockManager(host) + c.nodes[host] = dlm + c.mu.Unlock() + + // Wire up replication + dlm.ReplicateFn = func(server pb.ServerAddress, key string, expiredAtNs int64, token string, owner string, generation int64, seq int64, isUnlock bool) { + c.mu.Lock() + target, ok := c.nodes[server] + c.mu.Unlock() + if !ok { + return + } + if isUnlock { + target.RemoveBackupLockIfSeq(key, generation, seq) + } else { + target.InsertBackupLock(key, expiredAtNs, token, owner, generation, seq) + } + } + + servers := c.getServers() + for _, n := range c.getNodes() { + n.LockRing.SetSnapshot(servers, 0) + } +} + +func (c *testCluster) getNodes() map[pb.ServerAddress]*DistributedLockManager { + c.mu.Lock() + defer c.mu.Unlock() + cp := make(map[pb.ServerAddress]*DistributedLockManager, len(c.nodes)) + for k, v := range c.nodes { + cp[k] = v + } + return cp +} + +func (c *testCluster) getServers() []pb.ServerAddress { + c.mu.Lock() + defer c.mu.Unlock() + var servers []pb.ServerAddress + for s := range c.nodes { + servers = append(servers, s) + } + return servers +} + +func (c *testCluster) get(host pb.ServerAddress) *DistributedLockManager { + c.mu.Lock() + defer c.mu.Unlock() + return c.nodes[host] +} + +// acquireLock tries to acquire a lock on the correct primary node. +// It follows redirects (movedTo) like a real client would. +func (c *testCluster) acquireLock(key, owner string, ttl time.Duration) (renewToken string, generation int64, primaryHost pb.ServerAddress, err error) { + // Try any node first (simulates client connecting to seed filer) + for _, dlm := range c.getNodes() { + expiry := time.Now().Add(ttl).UnixNano() + var movedTo pb.ServerAddress + var lockErr error + _, renewToken, generation, movedTo, lockErr = dlm.LockWithTimeout(key, expiry, "", owner) + if movedTo != "" && movedTo != dlm.Host { + // Follow redirect + target := c.get(movedTo) + if target == nil { + err = fmt.Errorf("primary %s is down", movedTo) + return + } + _, renewToken, generation, _, lockErr = target.LockWithTimeout(key, expiry, "", owner) + if lockErr != nil { + err = lockErr + return + } + primaryHost = movedTo + // Wait briefly for async replication to complete + time.Sleep(10 * time.Millisecond) + return + } + if lockErr != nil { + err = lockErr + return + } + primaryHost = dlm.Host + time.Sleep(10 * time.Millisecond) + return + } + err = fmt.Errorf("no nodes available") + return +} + +// renewLock renews a lock on the primary node +func (c *testCluster) renewLock(key, owner, token string, ttl time.Duration, primaryHost pb.ServerAddress) (newToken string, generation int64, err error) { + target := c.get(primaryHost) + if target == nil { + err = fmt.Errorf("primary %s is down", primaryHost) + return + } + expiry := time.Now().Add(ttl).UnixNano() + var movedTo pb.ServerAddress + var lockErr error + _, newToken, generation, movedTo, lockErr = target.LockWithTimeout(key, expiry, token, owner) + if movedTo != "" && movedTo != primaryHost { + target = c.get(movedTo) + if target == nil { + err = fmt.Errorf("new primary %s is down", movedTo) + return + } + // Pass the existing token so the redirected renewal can match + // if the lock was already transferred to the new primary. + _, newToken, generation, _, lockErr = target.LockWithTimeout(key, expiry, token, owner) + } + err = lockErr + time.Sleep(10 * time.Millisecond) + return +} + +// --- Test Cases --- + +func TestDLM_PrimaryCrash_BackupPromotes(t *testing.T) { + // Scenario: Lock is acquired, primary crashes, backup should have the lock + hosts := []pb.ServerAddress{"filer1:8888", "filer2:8888", "filer3:8888"} + cluster := newTestCluster(hosts...) + + key := "test-lock-primary-crash" + renewToken, _, primaryHost, err := cluster.acquireLock(key, "owner1", 30*time.Second) + require.NoError(t, err) + require.NotEmpty(t, renewToken) + + // Find the backup for this key + _, backup := cluster.get(primaryHost).LockRing.GetPrimaryAndBackup(key) + require.NotEmpty(t, backup, "should have a backup server") + + // Verify backup has the lock + backupDlm := cluster.get(backup) + backupLock, found := backupDlm.GetLock(key) + require.True(t, found, "backup should have the lock") + assert.True(t, backupLock.IsBackup, "lock on backup should be marked as backup") + assert.Equal(t, renewToken, backupLock.Token, "backup should have the same token") + + // Crash the primary + cluster.removeNode(primaryHost) + + // Simulate topology change: promote backup locks + for _, dlm := range cluster.getNodes() { + locks := dlm.AllLocks() + for _, lock := range locks { + newPrimary, _ := dlm.LockRing.GetPrimaryAndBackup(lock.Key) + if newPrimary == dlm.Host && lock.IsBackup { + dlm.PromoteLock(lock.Key) + } + } + } + + // The backup should now be the primary + newPrimary := backupDlm.LockRing.GetPrimary(key) + assert.Equal(t, backup, newPrimary, "backup should be the new primary") + + // The promoted lock should work — verify it's no longer a backup + promotedLock, found := backupDlm.GetLock(key) + require.True(t, found, "lock should still exist after promotion") + assert.False(t, promotedLock.IsBackup, "lock should be promoted to primary") + + // Client should be able to renew with the same token on the new primary + newToken, _, err := cluster.renewLock(key, "owner1", renewToken, 30*time.Second, backup) + require.NoError(t, err) + assert.NotEmpty(t, newToken, "renewal on new primary should succeed") +} + +func TestDLM_BackupCrash_PrimaryContinues(t *testing.T) { + hosts := []pb.ServerAddress{"filer1:8888", "filer2:8888", "filer3:8888"} + cluster := newTestCluster(hosts...) + + key := "test-lock-backup-crash" + renewToken, _, primaryHost, err := cluster.acquireLock(key, "owner1", 30*time.Second) + require.NoError(t, err) + + _, backup := cluster.get(primaryHost).LockRing.GetPrimaryAndBackup(key) + + // Crash the backup + cluster.removeNode(backup) + + // Primary should still work — renew the lock + newToken, _, err := cluster.renewLock(key, "owner1", renewToken, 30*time.Second, primaryHost) + require.NoError(t, err) + assert.NotEmpty(t, newToken, "primary should continue working after backup crash") + + // Verify primary is still the primary for this key + newPrimary := cluster.get(primaryHost).LockRing.GetPrimary(key) + assert.Equal(t, primaryHost, newPrimary) +} + +func TestDLM_BothPrimaryAndBackupCrash(t *testing.T) { + hosts := []pb.ServerAddress{"filer1:8888", "filer2:8888", "filer3:8888"} + cluster := newTestCluster(hosts...) + + key := "test-lock-both-crash" + _, _, primaryHost, err := cluster.acquireLock(key, "owner1", 30*time.Second) + require.NoError(t, err) + + _, backup := cluster.get(primaryHost).LockRing.GetPrimaryAndBackup(key) + + // Crash both + cluster.removeNode(primaryHost) + cluster.removeNode(backup) + + // The lock is lost — the surviving node should be able to acquire it fresh + newToken, _, _, err := cluster.acquireLock(key, "owner2", 30*time.Second) + require.NoError(t, err) + assert.NotEmpty(t, newToken, "new owner should acquire lock after both crash") +} + +func TestDLM_RollingRestart(t *testing.T) { + hosts := []pb.ServerAddress{"filer1:8888", "filer2:8888", "filer3:8888"} + cluster := newTestCluster(hosts...) + + // Acquire multiple locks + type lockState struct { + key, owner, token string + generation int64 + primary pb.ServerAddress + } + locks := make([]lockState, 5) + for i := range locks { + key := fmt.Sprintf("rolling-lock-%d", i) + token, gen, primary, err := cluster.acquireLock(key, fmt.Sprintf("owner-%d", i), 30*time.Second) + require.NoError(t, err) + locks[i] = lockState{key: key, owner: fmt.Sprintf("owner-%d", i), token: token, generation: gen, primary: primary} + } + + // Rolling restart: remove and re-add each node one at a time. + // After removing a node, promote backups and re-replicate to new backups + // to maintain the invariant that each lock has a backup copy. + for _, host := range hosts { + cluster.removeNode(host) + + // Simulate full OnDlmChangeSnapshot: promote backups and re-replicate + for _, dlm := range cluster.getNodes() { + for _, lock := range dlm.AllLocks() { + newPrimary, _ := dlm.LockRing.GetPrimaryAndBackup(lock.Key) + if newPrimary == dlm.Host && lock.IsBackup { + dlm.PromoteLock(lock.Key) + } + } + // Re-replicate all primary locks to their new backups + for _, lock := range dlm.AllLocks() { + newPrimary, _ := dlm.LockRing.GetPrimaryAndBackup(lock.Key) + if newPrimary == dlm.Host && !lock.IsBackup { + dlm.replicateToBackup(lock.Key, lock.ExpiredAtNs, lock.Token, lock.Owner, lock.Generation, lock.Seq, false) + } + } + } + + time.Sleep(10 * time.Millisecond) + + // Re-add the node + cluster.addNode(host) + time.Sleep(10 * time.Millisecond) + } + + // After rolling restart, locks should survive via backup promotion + survivedCount := 0 + for _, ls := range locks { + for _, dlm := range cluster.getNodes() { + lock, found := dlm.GetLock(ls.key) + if found && !lock.IsBackup { + survivedCount++ + break + } + } + } + t.Logf("Locks survived rolling restart: %d / %d", survivedCount, len(locks)) + require.Greater(t, survivedCount, 0, "at least some locks should survive a rolling restart via backup promotion") +} + +func TestDLM_GenerationIncrementsOnNewAcquisition(t *testing.T) { + hosts := []pb.ServerAddress{"filer1:8888", "filer2:8888"} + cluster := newTestCluster(hosts...) + + key := "gen-test-lock" + + // Acquire lock — generation should be > 0 + token1, gen1, primary, err := cluster.acquireLock(key, "owner1", 2*time.Second) + require.NoError(t, err) + assert.Greater(t, gen1, int64(0)) + + // Renew — generation should stay the same + token2, gen2, err := cluster.renewLock(key, "owner1", token1, 2*time.Second, primary) + require.NoError(t, err) + assert.Equal(t, gen1, gen2, "generation should not change on renewal") + + // Let lock expire + time.Sleep(3 * time.Second) + + // Re-acquire — generation should increment + _, gen3, _, err := cluster.acquireLock(key, "owner2", 30*time.Second) + require.NoError(t, err) + assert.Greater(t, gen3, gen1, "generation should increment on new acquisition") + _ = token2 +} + +func TestDLM_ReplicationFailure_PrimaryStillWorks(t *testing.T) { + hosts := []pb.ServerAddress{"filer1:8888", "filer2:8888", "filer3:8888"} + cluster := newTestCluster(hosts...) + + // Break replication by setting a no-op ReplicateFn on all nodes + for _, dlm := range cluster.getNodes() { + dlm.ReplicateFn = func(server pb.ServerAddress, key string, expiredAtNs int64, token string, owner string, generation int64, seq int64, isUnlock bool) { + // Simulate replication failure: do nothing + } + } + + key := "repl-fail-lock" + renewToken, _, primaryHost, err := cluster.acquireLock(key, "owner1", 30*time.Second) + require.NoError(t, err) + + // Primary should have the lock + primaryDlm := cluster.get(primaryHost) + lock, found := primaryDlm.GetLock(key) + require.True(t, found, "primary should have the lock") + assert.False(t, lock.IsBackup) + + // Backup should NOT have it (replication failed) + _, backup := primaryDlm.LockRing.GetPrimaryAndBackup(key) + backupDlm := cluster.get(backup) + _, found = backupDlm.GetLock(key) + assert.False(t, found, "backup should not have the lock when replication fails") + + // Primary should still be able to renew + newToken, _, err := cluster.renewLock(key, "owner1", renewToken, 30*time.Second, primaryHost) + require.NoError(t, err) + assert.NotEmpty(t, newToken) +} + +func TestDLM_UnlockReplicatesToBackup(t *testing.T) { + hosts := []pb.ServerAddress{"filer1:8888", "filer2:8888"} + cluster := newTestCluster(hosts...) + + key := "unlock-repl-lock" + renewToken, _, primaryHost, err := cluster.acquireLock(key, "owner1", 30*time.Second) + require.NoError(t, err) + + _, backup := cluster.get(primaryHost).LockRing.GetPrimaryAndBackup(key) + + // Verify backup has the lock + _, found := cluster.get(backup).GetLock(key) + require.True(t, found, "backup should have the lock") + + // Unlock on primary + primaryDlm := cluster.get(primaryHost) + movedTo, err := primaryDlm.Unlock(key, renewToken) + require.NoError(t, err) + assert.Empty(t, movedTo) + + // Wait for async replication + time.Sleep(20 * time.Millisecond) + + // Backup should also have removed the lock + _, found = cluster.get(backup).GetLock(key) + assert.False(t, found, "backup should remove lock after unlock replication") +} + +func TestDLM_TopologyChange_LockSurvivesServerAddition(t *testing.T) { + // Start with 2 servers, acquire lock, add a 3rd server + hosts := []pb.ServerAddress{"filer1:8888", "filer2:8888"} + cluster := newTestCluster(hosts...) + + key := "topo-add-lock" + renewToken, _, primaryHost, err := cluster.acquireLock(key, "owner1", 30*time.Second) + require.NoError(t, err) + + // Add a new server + cluster.addNode("filer3:8888") + time.Sleep(20 * time.Millisecond) + + // The lock should still be accessible — either the same primary or on a new one + // Try to renew on the original primary first + newPrimary := cluster.get(primaryHost).LockRing.GetPrimary(key) + if newPrimary == primaryHost { + // Still on same primary + newToken, _, err := cluster.renewLock(key, "owner1", renewToken, 30*time.Second, primaryHost) + require.NoError(t, err) + assert.NotEmpty(t, newToken) + } + // If primary changed, the lock may need transfer — that's handled by OnDlmChangeSnapshot + // which is tested at the server level +} + +func TestDLM_ConsistentHashing_MinimalDisruption(t *testing.T) { + // Verify that removing a server only affects locks on that server + hosts := []pb.ServerAddress{"filer1:8888", "filer2:8888", "filer3:8888"} + cluster := newTestCluster(hosts...) + + // Acquire 50 locks + type lockInfo struct { + key, token string + primary pb.ServerAddress + } + locks := make([]lockInfo, 50) + for i := range locks { + key := fmt.Sprintf("min-disrupt-%d", i) + token, _, primary, err := cluster.acquireLock(key, "owner", 30*time.Second) + require.NoError(t, err) + locks[i] = lockInfo{key: key, token: token, primary: primary} + } + + // Count locks per server before removal + countBefore := make(map[pb.ServerAddress]int) + for _, l := range locks { + countBefore[l.primary]++ + } + t.Logf("Lock distribution before: %v", countBefore) + + // Remove filer2 + cluster.removeNode("filer2:8888") + + // Count how many locks changed primary + changed := 0 + for _, l := range locks { + // Check where the lock should be now + for _, dlm := range cluster.getNodes() { + newPrimary := dlm.LockRing.GetPrimary(l.key) + if newPrimary != l.primary { + changed++ + } + break + } + } + + // Only locks from filer2 should have changed + assert.Equal(t, countBefore["filer2:8888"], changed, + "only locks from removed server should change primary") +} + +func TestDLM_NodeDropAndJoin_OwnershipDisruption(t *testing.T) { + // Scenario: 3 nodes, acquire locks, one drops and a NEW node joins quickly. + // The new node steals hash ranges from surviving nodes, not just from the + // departed node. This test measures the disruption. + hosts := []pb.ServerAddress{"filer1:8888", "filer2:8888", "filer3:8888"} + cluster := newTestCluster(hosts...) + + // Acquire many locks + numLocks := 100 + type lockInfo struct { + key, token string + primary pb.ServerAddress + } + locks := make([]lockInfo, numLocks) + for i := range locks { + key := fmt.Sprintf("churn-lock-%d", i) + token, _, primary, err := cluster.acquireLock(key, "owner", 30*time.Second) + require.NoError(t, err) + locks[i] = lockInfo{key: key, token: token, primary: primary} + } + + // Record primary for each lock before the change + beforePrimary := make(map[string]pb.ServerAddress) + for _, l := range locks { + beforePrimary[l.key] = l.primary + } + + // Drop filer3 and immediately add filer4 + cluster.removeNode("filer3:8888") + + // Promote backups on remaining nodes (simulates OnDlmChangeSnapshot) + for _, dlm := range cluster.getNodes() { + for _, lock := range dlm.AllLocks() { + p, _ := dlm.LockRing.GetPrimaryAndBackup(lock.Key) + if p == dlm.Host && lock.IsBackup { + dlm.PromoteLock(lock.Key) + } + } + // Re-replicate primary locks to new backups + for _, lock := range dlm.AllLocks() { + p, _ := dlm.LockRing.GetPrimaryAndBackup(lock.Key) + if p == dlm.Host && !lock.IsBackup { + dlm.replicateToBackup(lock.Key, lock.ExpiredAtNs, lock.Token, lock.Owner, lock.Generation, lock.Seq, false) + } + } + } + time.Sleep(10 * time.Millisecond) + + // Now add filer4 (new node, empty) + cluster.addNode("filer4:8888") + time.Sleep(10 * time.Millisecond) + + // Simulate OnDlmChangeSnapshot on all nodes after filer4 joins: + // transfer locks that now belong to filer4 + for host, dlm := range cluster.getNodes() { + for _, lock := range dlm.AllLocks() { + p, _ := dlm.LockRing.GetPrimaryAndBackup(lock.Key) + if p != host && !lock.IsBackup { + // This lock should move to the new primary + target := cluster.get(p) + if target != nil { + target.InsertLock(lock.Key, lock.ExpiredAtNs, lock.Token, lock.Owner, lock.Generation, lock.Seq) + dlm.DemoteLock(lock.Key) + } + } + } + } + time.Sleep(10 * time.Millisecond) + + // Count disruptions: locks whose primary changed to a node other than filer3's successor + disruptedFromSurvivors := 0 + disruptedFromDeparted := 0 + movedToFiler4 := 0 + for _, l := range locks { + // What's the new primary? + var newPrimary pb.ServerAddress + for _, dlm := range cluster.getNodes() { + newPrimary = dlm.LockRing.GetPrimary(l.key) + break + } + oldPrimary := beforePrimary[l.key] + if newPrimary != oldPrimary { + if oldPrimary == "filer3:8888" { + disruptedFromDeparted++ + } else { + disruptedFromSurvivors++ + } + } + if newPrimary == "filer4:8888" { + movedToFiler4++ + } + } + + t.Logf("Locks disrupted from departed filer3: %d / %d", disruptedFromDeparted, numLocks) + t.Logf("Locks disrupted from surviving filer1/filer2: %d / %d", disruptedFromSurvivors, numLocks) + t.Logf("Locks now on new filer4: %d / %d", movedToFiler4, numLocks) + + // The key concern: filer4 joining disrupts locks on surviving nodes + // With consistent hashing, new node steals ~1/N of each surviving node's keys + // Verify that the transfer logic above moved those locks to filer4 + for _, l := range locks { + var newPrimary pb.ServerAddress + for _, dlm := range cluster.getNodes() { + newPrimary = dlm.LockRing.GetPrimary(l.key) + break + } + target := cluster.get(newPrimary) + require.NotNil(t, target, "primary %s should exist", newPrimary) + + lock, found := target.GetLock(l.key) + if !found { + // Lock may have only a backup copy if transfer happened but + // the lock was on the departed node and wasn't re-replicated. + // Check all nodes for any copy. + anyFound := false + for _, dlm := range cluster.getNodes() { + if _, f := dlm.GetLock(l.key); f { + anyFound = true + break + } + } + if !anyFound { + t.Errorf("lock %s completely lost (primary should be %s)", l.key, newPrimary) + } + continue + } + assert.False(t, lock.IsBackup, "lock %s on primary %s should not be a backup", l.key, newPrimary) + } +} + +func TestDLM_RenewalDuringTransferWindow(t *testing.T) { + // When a new node joins and steals a key range from a surviving node, + // there's a window between ring update and lock transfer. During this + // window, a client renewal should still succeed on the old primary + // (because it still holds the lock locally). + hosts := []pb.ServerAddress{"filer1:8888", "filer2:8888", "filer3:8888"} + cluster := newTestCluster(hosts...) + + // Find a key that will move primary when filer4 is added. + // Try candidate keys until we find one whose primary changes. + var key, renewToken string + var primaryHost pb.ServerAddress + for i := 0; i < 1000; i++ { + candidate := fmt.Sprintf("transfer-window-lock-%d", i) + token, _, primary, err := cluster.acquireLock(candidate, "owner1", 30*time.Second) + require.NoError(t, err) + + // Check if adding filer4 would move this key's primary + tmpRing := NewHashRing(DefaultVnodeCount) + tmpRing.SetServers([]pb.ServerAddress{"filer1:8888", "filer2:8888", "filer3:8888", "filer4:8888"}) + newPrimary := tmpRing.GetPrimary(candidate) + if newPrimary != primary { + key = candidate + renewToken = token + primaryHost = primary + break + } + } + require.NotEmpty(t, key, "should find a key that moves primary when filer4 joins") + + // Add filer4 — this changes the primary for our key per the ring + cluster.addNode("filer4:8888") + time.Sleep(10 * time.Millisecond) + + newPrimary := cluster.get(primaryHost).LockRing.GetPrimary(key) + require.NotEqual(t, primaryHost, newPrimary, "key should have moved to a different primary") + + // Renewal on the OLD primary should still succeed because it holds the lock locally + newToken, _, err := cluster.renewLock(key, "owner1", renewToken, 30*time.Second, primaryHost) + require.NoError(t, err, "renewal on old primary should succeed during transfer window") + assert.NotEmpty(t, newToken, "should get a new token from old primary") + t.Logf("Key %s: primary changed from %s to %s, but renewal on old primary succeeded", key, primaryHost, newPrimary) +} + +func TestDLM_StaleReplicationRejected(t *testing.T) { + // Verify that a stale replication (lower seq) does not overwrite a newer one + lm := NewLockManager() + + // Insert backup with seq=3 + lm.InsertBackupLock("key1", time.Now().Add(30*time.Second).UnixNano(), "token-new", "owner1", 1, 3) + lock, found := lm.GetLock("key1") + require.True(t, found) + assert.Equal(t, "token-new", lock.Token) + assert.Equal(t, int64(3), lock.Seq) + + // Try to overwrite with stale seq=2 — should be rejected + lm.InsertBackupLock("key1", time.Now().Add(30*time.Second).UnixNano(), "token-old", "owner1", 1, 2) + lock, found = lm.GetLock("key1") + require.True(t, found) + assert.Equal(t, "token-new", lock.Token, "stale replication should be rejected") + assert.Equal(t, int64(3), lock.Seq) + + // Update with higher seq=4 — should succeed + lm.InsertBackupLock("key1", time.Now().Add(30*time.Second).UnixNano(), "token-newer", "owner1", 1, 4) + lock, found = lm.GetLock("key1") + require.True(t, found) + assert.Equal(t, "token-newer", lock.Token, "newer replication should be accepted") + assert.Equal(t, int64(4), lock.Seq) + + // Stale unlock (seq=2) should not delete the lock + removed := lm.RemoveBackupLockIfSeq("key1", 1, 2) + assert.False(t, removed, "stale unlock should be rejected") + _, found = lm.GetLock("key1") + assert.True(t, found, "lock should still exist after stale unlock") + + // Valid unlock (seq=5) should delete + removed = lm.RemoveBackupLockIfSeq("key1", 1, 5) + assert.True(t, removed, "valid unlock should be accepted") + _, found = lm.GetLock("key1") + assert.False(t, found, "lock should be removed after valid unlock") +} diff --git a/weed/cluster/lock_manager/hash_ring.go b/weed/cluster/lock_manager/hash_ring.go new file mode 100644 index 000000000..770a58eda --- /dev/null +++ b/weed/cluster/lock_manager/hash_ring.go @@ -0,0 +1,198 @@ +package lock_manager + +import ( + "hash/crc32" + "sort" + "sync" + + "github.com/seaweedfs/seaweedfs/weed/pb" +) + +const DefaultVnodeCount = 50 + +// HashRing implements consistent hashing with virtual nodes. +// When a server is removed, only the keys that hashed to that server +// are remapped (to the next server on the ring), leaving all other +// key-to-server mappings stable. +// +// UPGRADE NOTE: This replaces the previous modulo-based hashing +// (hash % len(servers)). The two schemes compute different primaries +// for the same key, so all filer nodes in the cluster must be upgraded +// together (or via a rolling restart that completes within the lock TTL +// window of ~7 seconds) to avoid routing disagreements. +type HashRing struct { + mu sync.RWMutex + vnodeCount int + sortedHashes []uint32 // sorted ring positions + vnodeToServer map[uint32]pb.ServerAddress // ring position → server + servers map[pb.ServerAddress]struct{} // set of all servers +} + +func NewHashRing(vnodeCount int) *HashRing { + if vnodeCount <= 0 { + vnodeCount = DefaultVnodeCount + } + return &HashRing{ + vnodeCount: vnodeCount, + vnodeToServer: make(map[uint32]pb.ServerAddress), + servers: make(map[pb.ServerAddress]struct{}), + } +} + +// AddServer adds a server with virtual nodes to the ring. +func (hr *HashRing) AddServer(server pb.ServerAddress) { + hr.mu.Lock() + defer hr.mu.Unlock() + + if _, exists := hr.servers[server]; exists { + return + } + hr.servers[server] = struct{}{} + hr.rebuildRing() +} + +// RemoveServer removes a server and its virtual nodes from the ring. +func (hr *HashRing) RemoveServer(server pb.ServerAddress) { + hr.mu.Lock() + defer hr.mu.Unlock() + + if _, exists := hr.servers[server]; !exists { + return + } + delete(hr.servers, server) + hr.rebuildRing() +} + +// SetServers replaces the entire server set. +func (hr *HashRing) SetServers(servers []pb.ServerAddress) { + hr.mu.Lock() + defer hr.mu.Unlock() + + hr.servers = make(map[pb.ServerAddress]struct{}, len(servers)) + for _, s := range servers { + hr.servers[s] = struct{}{} + } + hr.rebuildRing() +} + +// GetPrimaryAndBackup returns the primary server for a key and its backup +// (the next distinct server clockwise on the ring). +// If there is only one server, backup is empty. +func (hr *HashRing) GetPrimaryAndBackup(key string) (primary, backup pb.ServerAddress) { + hr.mu.RLock() + defer hr.mu.RUnlock() + + if len(hr.sortedHashes) == 0 { + return "", "" + } + + hash := hashKey(key) + idx := hr.search(hash) + primary = hr.vnodeToServer[hr.sortedHashes[idx]] + + // Walk clockwise to find a different server for backup + ringLen := len(hr.sortedHashes) + for i := 1; i < ringLen; i++ { + candidate := hr.vnodeToServer[hr.sortedHashes[(idx+i)%ringLen]] + if candidate != primary { + backup = candidate + return + } + } + // Only one server — no backup + return primary, "" +} + +// GetPrimary returns just the primary server for a key. +func (hr *HashRing) GetPrimary(key string) pb.ServerAddress { + hr.mu.RLock() + defer hr.mu.RUnlock() + + if len(hr.sortedHashes) == 0 { + return "" + } + + hash := hashKey(key) + idx := hr.search(hash) + return hr.vnodeToServer[hr.sortedHashes[idx]] +} + +// GetServers returns a sorted copy of all servers in the ring. +func (hr *HashRing) GetServers() []pb.ServerAddress { + hr.mu.RLock() + defer hr.mu.RUnlock() + + servers := make([]pb.ServerAddress, 0, len(hr.servers)) + for s := range hr.servers { + servers = append(servers, s) + } + sort.Slice(servers, func(i, j int) bool { + return servers[i] < servers[j] + }) + return servers +} + +// ServerCount returns the number of servers in the ring. +func (hr *HashRing) ServerCount() int { + hr.mu.RLock() + defer hr.mu.RUnlock() + return len(hr.servers) +} + +// rebuildRing rebuilds the sorted hash ring from the current server set. +// Caller must hold hr.mu write lock. +func (hr *HashRing) rebuildRing() { + hr.vnodeToServer = make(map[uint32]pb.ServerAddress, len(hr.servers)*hr.vnodeCount) + hr.sortedHashes = make([]uint32, 0, len(hr.servers)*hr.vnodeCount) + + for server := range hr.servers { + for i := 0; i < hr.vnodeCount; i++ { + vnodeKey := vnodeKeyFor(server, i) + hash := hashKey(vnodeKey) + hr.vnodeToServer[hash] = server + hr.sortedHashes = append(hr.sortedHashes, hash) + } + } + sort.Slice(hr.sortedHashes, func(i, j int) bool { + return hr.sortedHashes[i] < hr.sortedHashes[j] + }) +} + +// search finds the first ring position >= hash. +func (hr *HashRing) search(hash uint32) int { + idx := sort.Search(len(hr.sortedHashes), func(i int) bool { + return hr.sortedHashes[i] >= hash + }) + if idx >= len(hr.sortedHashes) { + idx = 0 // wrap around + } + return idx +} + +func hashKey(key string) uint32 { + return crc32.ChecksumIEEE([]byte(key)) +} + +func vnodeKeyFor(server pb.ServerAddress, index int) string { + // Use a format that distributes well across the ring + buf := make([]byte, 0, len(server)+10) + buf = append(buf, []byte(server)...) + buf = append(buf, '#') + buf = appendInt(buf, index) + return string(buf) +} + +func appendInt(buf []byte, n int) []byte { + if n == 0 { + return append(buf, '0') + } + // Simple int-to-string without importing strconv + digits := [20]byte{} + pos := len(digits) + for n > 0 { + pos-- + digits[pos] = byte('0' + n%10) + n /= 10 + } + return append(buf, digits[pos:]...) +} diff --git a/weed/cluster/lock_manager/hash_ring_test.go b/weed/cluster/lock_manager/hash_ring_test.go new file mode 100644 index 000000000..6a7dabe2e --- /dev/null +++ b/weed/cluster/lock_manager/hash_ring_test.go @@ -0,0 +1,173 @@ +package lock_manager + +import ( + "fmt" + "math" + "testing" + + "github.com/seaweedfs/seaweedfs/weed/pb" + "github.com/stretchr/testify/assert" +) + +func TestHashRing_BasicOperations(t *testing.T) { + hr := NewHashRing(50) + + // Empty ring + p, b := hr.GetPrimaryAndBackup("key1") + assert.Equal(t, pb.ServerAddress(""), p) + assert.Equal(t, pb.ServerAddress(""), b) + + // Single server — no backup + hr.AddServer("filer1:8888") + p, b = hr.GetPrimaryAndBackup("key1") + assert.Equal(t, pb.ServerAddress("filer1:8888"), p) + assert.Equal(t, pb.ServerAddress(""), b) + + // Two servers — backup is the other server + hr.AddServer("filer2:8888") + p, b = hr.GetPrimaryAndBackup("key1") + assert.NotEqual(t, p, b) + assert.NotEmpty(t, b) + + // Three servers + hr.AddServer("filer3:8888") + p, b = hr.GetPrimaryAndBackup("key1") + assert.NotEqual(t, p, b) + assert.NotEmpty(t, b) + + // Remove server + hr.RemoveServer("filer2:8888") + assert.Equal(t, 2, hr.ServerCount()) +} + +func TestHashRing_DuplicateAddRemove(t *testing.T) { + hr := NewHashRing(50) + + hr.AddServer("filer1:8888") + hr.AddServer("filer1:8888") // duplicate + assert.Equal(t, 1, hr.ServerCount()) + + hr.RemoveServer("filer1:8888") + assert.Equal(t, 0, hr.ServerCount()) + + hr.RemoveServer("filer1:8888") // remove non-existent + assert.Equal(t, 0, hr.ServerCount()) +} + +func TestHashRing_SetServers(t *testing.T) { + hr := NewHashRing(50) + + hr.SetServers([]pb.ServerAddress{"a:1", "b:2", "c:3"}) + assert.Equal(t, 3, hr.ServerCount()) + + servers := hr.GetServers() + assert.Equal(t, 3, len(servers)) + + // SetServers replaces + hr.SetServers([]pb.ServerAddress{"x:1", "y:2"}) + assert.Equal(t, 2, hr.ServerCount()) +} + +func TestHashRing_ConsistencyOnRemoval(t *testing.T) { + // The key property of consistent hashing: when a server is removed, + // only keys that mapped to the removed server change. + hr := NewHashRing(50) + servers := []pb.ServerAddress{"filer1:8888", "filer2:8888", "filer3:8888"} + hr.SetServers(servers) + + numKeys := 1000 + // Record where each key maps before removal + before := make(map[string]pb.ServerAddress, numKeys) + for i := 0; i < numKeys; i++ { + key := fmt.Sprintf("lock-key-%d", i) + before[key] = hr.GetPrimary(key) + } + + // Remove filer2 + hr.RemoveServer("filer2:8888") + + moved := 0 + for i := 0; i < numKeys; i++ { + key := fmt.Sprintf("lock-key-%d", i) + after := hr.GetPrimary(key) + if before[key] != after { + // Only keys from filer2 should move + assert.Equal(t, pb.ServerAddress("filer2:8888"), before[key], + "key %s moved from %s to %s, but it wasn't on the removed server", key, before[key], after) + moved++ + } + } + // Roughly 1/3 of keys should move (those that were on filer2) + t.Logf("Keys that moved: %d / %d", moved, numKeys) + assert.Greater(t, moved, 0, "some keys should have moved") + assert.Less(t, moved, numKeys, "not all keys should move") +} + +func TestHashRing_BackupIsSuccessor(t *testing.T) { + // After removing primary, the backup should become the new primary + hr := NewHashRing(50) + servers := []pb.ServerAddress{"filer1:8888", "filer2:8888", "filer3:8888"} + hr.SetServers(servers) + + // For each key, verify that removing the primary makes the backup the new primary + promoted := 0 + total := 500 + for i := 0; i < total; i++ { + key := fmt.Sprintf("test-lock-%d", i) + primary, backup := hr.GetPrimaryAndBackup(key) + assert.NotEqual(t, primary, backup) + + // Temporarily remove primary + hr.RemoveServer(primary) + newPrimary := hr.GetPrimary(key) + if newPrimary == backup { + promoted++ + } + // Restore + hr.AddServer(primary) + } + // The backup should become new primary for all keys + assert.Equal(t, total, promoted, + "backup should become new primary for all keys when primary is removed") +} + +func TestHashRing_Distribution(t *testing.T) { + hr := NewHashRing(50) + servers := []pb.ServerAddress{"filer1:8888", "filer2:8888", "filer3:8888"} + hr.SetServers(servers) + + counts := make(map[pb.ServerAddress]int) + numKeys := 3000 + for i := 0; i < numKeys; i++ { + key := fmt.Sprintf("dist-key-%d", i) + p := hr.GetPrimary(key) + counts[p]++ + } + + expected := float64(numKeys) / float64(len(servers)) + for server, count := range counts { + deviation := math.Abs(float64(count)-expected) / expected + t.Logf("Server %s: %d keys (%.1f%% deviation)", server, count, deviation*100) + // Allow up to 40% deviation with 50 vnodes and 3 servers + assert.Less(t, deviation, 0.40, + "server %s has too many or too few keys: %d (expected ~%d)", server, count, int(expected)) + } +} + +func TestHashRing_GetPrimary(t *testing.T) { + hr := NewHashRing(50) + + // Empty ring + assert.Equal(t, pb.ServerAddress(""), hr.GetPrimary("key")) + + hr.SetServers([]pb.ServerAddress{"a:1", "b:2"}) + + // Deterministic: same key always maps to same server + p1 := hr.GetPrimary("mykey") + p2 := hr.GetPrimary("mykey") + assert.Equal(t, p1, p2) + + // GetPrimary matches the primary from GetPrimaryAndBackup + primary, _ := hr.GetPrimaryAndBackup("mykey") + assert.Equal(t, primary, hr.GetPrimary("mykey")) +} diff --git a/weed/cluster/lock_manager/lock_manager.go b/weed/cluster/lock_manager/lock_manager.go index 4f7c6f9c6..b86a1f36d 100644 --- a/weed/cluster/lock_manager/lock_manager.go +++ b/weed/cluster/lock_manager/lock_manager.go @@ -3,6 +3,7 @@ package lock_manager import ( "fmt" "sync" + "sync/atomic" "time" "github.com/google/uuid" @@ -17,14 +18,18 @@ var LockNotFound = fmt.Errorf("lock not found") // LockManager local lock manager, used by distributed lock manager type LockManager struct { - locks map[string]*Lock - accessLock sync.RWMutex + locks map[string]*Lock + accessLock sync.RWMutex + nextGeneration atomic.Int64 } type Lock struct { Token string ExpiredAtNs int64 Key string // only used for moving locks Owner string + IsBackup bool // true if this node holds the lock as a backup + Generation int64 // monotonic fencing token, increments on fresh acquisition + Seq int64 // per-lock sequence number, increments on every mutation (acquire/renew/unlock) } func NewLockManager() *LockManager { @@ -35,7 +40,38 @@ func NewLockManager() *LockManager { return t } -func (lm *LockManager) Lock(path string, expiredAtNs int64, token string, owner string) (lockOwner, renewToken string, err error) { +func (lm *LockManager) NextGeneration() int64 { + return lm.nextGeneration.Add(1) +} + +func compareMutationVersion(generation, seq int64, existingGeneration, existingSeq int64) int { + switch { + case generation < existingGeneration: + return -1 + case generation > existingGeneration: + return 1 + case seq < existingSeq: + return -1 + case seq > existingSeq: + return 1 + default: + return 0 + } +} + +func (lm *LockManager) advanceGenerationFloor(generation int64) { + for { + current := lm.nextGeneration.Load() + if generation < current { + return + } + if lm.nextGeneration.CompareAndSwap(current, generation) { + return + } + } +} + +func (lm *LockManager) Lock(path string, expiredAtNs int64, token string, owner string) (lockOwner, renewToken string, generation int64, seq int64, err error) { lm.accessLock.Lock() defer lm.accessLock.Unlock() @@ -51,8 +87,10 @@ func (lm *LockManager) Lock(path string, expiredAtNs int64, token string, owner } else { // new lock renewToken = uuid.New().String() - glog.V(4).Infof("key %s new token %v owner %v", path, renewToken, owner) - lm.locks[path] = &Lock{Token: renewToken, ExpiredAtNs: expiredAtNs, Owner: owner} + generation = lm.NextGeneration() + seq = 1 + glog.V(4).Infof("key %s new token %v owner %v generation %d", path, renewToken, owner, generation) + lm.locks[path] = &Lock{Token: renewToken, ExpiredAtNs: expiredAtNs, Owner: owner, Generation: generation, Seq: seq} return } } @@ -61,8 +99,10 @@ func (lm *LockManager) Lock(path string, expiredAtNs int64, token string, owner if oldValue.Token == token { // token matches, renew the lock renewToken = uuid.New().String() + generation = oldValue.Generation // keep same generation on renewal + seq = oldValue.Seq + 1 glog.V(4).Infof("key %s old token %v owner %v => %v owner %v", path, oldValue.Token, oldValue.Owner, renewToken, owner) - lm.locks[path] = &Lock{Token: renewToken, ExpiredAtNs: expiredAtNs, Owner: owner} + lm.locks[path] = &Lock{Token: renewToken, ExpiredAtNs: expiredAtNs, Owner: owner, Generation: generation, Seq: seq} return } else { if token == "" { @@ -79,9 +119,11 @@ func (lm *LockManager) Lock(path string, expiredAtNs int64, token string, owner glog.V(4).Infof("key %s no lock owner %v", path, owner) if token == "" { // new lock - glog.V(4).Infof("key %s new token %v owner %v", path, token, owner) renewToken = uuid.New().String() - lm.locks[path] = &Lock{Token: renewToken, ExpiredAtNs: expiredAtNs, Owner: owner} + generation = lm.NextGeneration() + seq = 1 + glog.V(4).Infof("key %s new token %v owner %v generation %d", path, renewToken, owner, generation) + lm.locks[path] = &Lock{Token: renewToken, ExpiredAtNs: expiredAtNs, Owner: owner, Generation: generation, Seq: seq} return } else { glog.V(4).Infof("key %s non-empty token %v owner %v", path, token, owner) @@ -91,7 +133,7 @@ func (lm *LockManager) Lock(path string, expiredAtNs int64, token string, owner } } -func (lm *LockManager) Unlock(path string, token string) (isUnlocked bool, err error) { +func (lm *LockManager) Unlock(path string, token string) (isUnlocked bool, generation int64, seq int64, err error) { lm.accessLock.Lock() defer lm.accessLock.Unlock() @@ -100,12 +142,16 @@ func (lm *LockManager) Unlock(path string, token string) (isUnlocked bool, err e if oldValue.ExpiredAtNs > 0 && oldValue.ExpiredAtNs < now.UnixNano() { // lock is expired, delete it isUnlocked = true + generation = oldValue.Generation + seq = oldValue.Seq + 1 glog.V(4).Infof("key %s expired at %v", path, time.Unix(0, oldValue.ExpiredAtNs)) delete(lm.locks, path) return } if oldValue.Token == token { isUnlocked = true + generation = oldValue.Generation + seq = oldValue.Seq + 1 glog.V(4).Infof("key %s unlocked with %v", path, token) delete(lm.locks, path) return @@ -140,10 +186,10 @@ func (lm *LockManager) CleanUp() { } // SelectLocks takes out locks by key -// if keyFn return true, the lock will be taken out +// if selectFn returns true, the lock will be removed and returned func (lm *LockManager) SelectLocks(selectFn func(key string) bool) (locks []*Lock) { - lm.accessLock.RLock() - defer lm.accessLock.RUnlock() + lm.accessLock.Lock() + defer lm.accessLock.Unlock() now := time.Now().UnixNano() @@ -163,12 +209,90 @@ func (lm *LockManager) SelectLocks(selectFn func(key string) bool) (locks []*Loc return } -// InsertLock inserts a lock unconditionally -func (lm *LockManager) InsertLock(path string, expiredAtNs int64, token string, owner string) { +// InsertLock inserts a lock from a transfer if it is newer than the current state. +func (lm *LockManager) InsertLock(path string, expiredAtNs int64, token string, owner string, generation int64, seq int64) bool { + lm.accessLock.Lock() + defer lm.accessLock.Unlock() + + if existing, found := lm.locks[path]; found { + if compareMutationVersion(generation, seq, existing.Generation, existing.Seq) <= 0 { + glog.V(4).Infof("lock %s: rejecting stale transfer gen=%d seq=%d (current gen=%d seq=%d)", path, generation, seq, existing.Generation, existing.Seq) + return false + } + } + lm.locks[path] = &Lock{Token: token, ExpiredAtNs: expiredAtNs, Owner: owner, Generation: generation, Seq: seq} + lm.advanceGenerationFloor(generation) + return true +} + +// InsertBackupLock inserts or updates a lock as a backup copy. +// It rejects stale mutations by comparing (generation, seq). If a current primary +// already exists on this node, newer replicated state refreshes that primary copy +// without demoting it back to a backup. +func (lm *LockManager) InsertBackupLock(path string, expiredAtNs int64, token string, owner string, generation int64, seq int64) { + lm.accessLock.Lock() + defer lm.accessLock.Unlock() + + if existing, found := lm.locks[path]; found { + if compareMutationVersion(generation, seq, existing.Generation, existing.Seq) <= 0 { + glog.V(4).Infof("backup lock %s: rejecting stale gen=%d seq=%d (current gen=%d seq=%d)", path, generation, seq, existing.Generation, existing.Seq) + return + } + lm.locks[path] = &Lock{ + Token: token, + ExpiredAtNs: expiredAtNs, + Owner: owner, + IsBackup: existing.IsBackup, + Generation: generation, + Seq: seq, + } + lm.advanceGenerationFloor(generation) + return + } + lm.locks[path] = &Lock{Token: token, ExpiredAtNs: expiredAtNs, Owner: owner, IsBackup: true, Generation: generation, Seq: seq} + lm.advanceGenerationFloor(generation) +} + +// RemoveLock removes a lock by key +func (lm *LockManager) RemoveLock(path string) { + lm.accessLock.Lock() + defer lm.accessLock.Unlock() + delete(lm.locks, path) +} + +// RemoveBackupLockIfSeq removes the local copy only if the incoming mutation is +// not older than the current (generation, seq). This prevents a late unlock from +// deleting a newer reacquired lock whose seq has reset. +func (lm *LockManager) RemoveBackupLockIfSeq(path string, generation int64, seq int64) bool { lm.accessLock.Lock() defer lm.accessLock.Unlock() - lm.locks[path] = &Lock{Token: token, ExpiredAtNs: expiredAtNs, Owner: owner} + if existing, found := lm.locks[path]; found { + if compareMutationVersion(generation, seq, existing.Generation, existing.Seq) >= 0 { + delete(lm.locks, path) + return true + } + glog.V(4).Infof("backup lock %s: rejecting stale unlock gen=%d seq=%d (current gen=%d seq=%d)", path, generation, seq, existing.Generation, existing.Seq) + return false + } + return true +} + +// GetLock returns a copy of the lock for a key, if it exists and is not expired +func (lm *LockManager) GetLock(key string) (*Lock, bool) { + lm.accessLock.RLock() + defer lm.accessLock.RUnlock() + + lock, found := lm.locks[key] + if !found { + return nil, false + } + if time.Now().UnixNano() > lock.ExpiredAtNs { + return nil, false + } + // Return a copy + cp := *lock + return &cp, true } func (lm *LockManager) GetLockOwner(key string) (owner string, err error) { @@ -176,8 +300,54 @@ func (lm *LockManager) GetLockOwner(key string) (owner string, err error) { defer lm.accessLock.RUnlock() if lock, found := lm.locks[key]; found { + if time.Now().UnixNano() > lock.ExpiredAtNs { + err = LockNotFound + return + } return lock.Owner, nil } err = LockNotFound return } + +// AllLocks returns a copy of all non-expired locks +func (lm *LockManager) AllLocks() []*Lock { + lm.accessLock.RLock() + defer lm.accessLock.RUnlock() + + now := time.Now().UnixNano() + var result []*Lock + for key, lock := range lm.locks { + if now > lock.ExpiredAtNs { + continue + } + cp := *lock + cp.Key = key + result = append(result, &cp) + } + return result +} + +// PromoteLock changes a backup lock to a primary lock +func (lm *LockManager) PromoteLock(key string) bool { + lm.accessLock.Lock() + defer lm.accessLock.Unlock() + + if lock, found := lm.locks[key]; found && lock.IsBackup { + lock.IsBackup = false + return true + } + return false +} + +// DemoteLock changes a primary lock to a backup lock +func (lm *LockManager) DemoteLock(key string) bool { + lm.accessLock.Lock() + defer lm.accessLock.Unlock() + + if lock, found := lm.locks[key]; found && !lock.IsBackup { + lock.IsBackup = true + return true + } + return false +} diff --git a/weed/cluster/lock_manager/lock_manager_test.go b/weed/cluster/lock_manager/lock_manager_test.go new file mode 100644 index 000000000..5caf31da2 --- /dev/null +++ b/weed/cluster/lock_manager/lock_manager_test.go @@ -0,0 +1,86 @@ +package lock_manager + +import ( + "testing" + "time" + + "github.com/stretchr/testify/assert" +) + +func TestLockManager_GetLockOwnerIgnoresExpiredLock(t *testing.T) { + lm := NewLockManager() + + lm.InsertLock("key1", time.Now().Add(-time.Second).UnixNano(), "token1", "owner1", 7, 3) + + owner, err := lm.GetLockOwner("key1") + assert.Empty(t, owner) + assert.ErrorIs(t, err, LockNotFound) +} + +func TestLockManager_InsertLockRejectsStaleTransfer(t *testing.T) { + lm := NewLockManager() + + lm.InsertLock("key1", time.Now().Add(30*time.Second).UnixNano(), "token-new", "owner1", 8, 4) + lm.InsertLock("key1", time.Now().Add(30*time.Second).UnixNano(), "token-old", "owner1", 7, 3) + + lock, found := lm.GetLock("key1") + assert.True(t, found) + assert.Equal(t, "token-new", lock.Token) + assert.Equal(t, int64(8), lock.Generation) + assert.Equal(t, int64(4), lock.Seq) +} + +func TestLockManager_InsertLockAdvancesGenerationCounter(t *testing.T) { + lm := NewLockManager() + + lm.InsertLock("key1", time.Now().Add(30*time.Second).UnixNano(), "token1", "owner1", 12, 1) + + _, renewToken, generation, _, err := lm.Lock("key2", time.Now().Add(30*time.Second).UnixNano(), "", "owner2") + assert.NoError(t, err) + assert.NotEmpty(t, renewToken) + assert.Greater(t, generation, int64(12)) +} + +func TestLockManager_InsertBackupLockRejectsOlderGeneration(t *testing.T) { + lm := NewLockManager() + + lm.InsertBackupLock("key1", time.Now().Add(30*time.Second).UnixNano(), "token-new", "owner1", 8, 1) + lm.InsertBackupLock("key1", time.Now().Add(30*time.Second).UnixNano(), "token-old", "owner1", 7, 9) + + lock, found := lm.GetLock("key1") + assert.True(t, found) + assert.Equal(t, "token-new", lock.Token) + assert.Equal(t, int64(8), lock.Generation) + assert.Equal(t, int64(1), lock.Seq) +} + +func TestLockManager_InsertBackupLockKeepsPrimaryRole(t *testing.T) { + lm := NewLockManager() + + ok := lm.InsertLock("key1", time.Now().Add(30*time.Second).UnixNano(), "token-old", "owner1", 8, 1) + assert.True(t, ok) + + lm.InsertBackupLock("key1", time.Now().Add(30*time.Second).UnixNano(), "token-new", "owner1", 8, 2) + + lock, found := lm.GetLock("key1") + assert.True(t, found) + assert.False(t, lock.IsBackup) + assert.Equal(t, "token-new", lock.Token) + assert.Equal(t, int64(8), lock.Generation) + assert.Equal(t, int64(2), lock.Seq) +} + +func TestLockManager_RemoveBackupLockRejectsOlderGeneration(t *testing.T) { + lm := NewLockManager() + + lm.InsertBackupLock("key1", time.Now().Add(30*time.Second).UnixNano(), "token-new", "owner1", 8, 1) + + removed := lm.RemoveBackupLockIfSeq("key1", 7, 9) + assert.False(t, removed) + + lock, found := lm.GetLock("key1") + assert.True(t, found) + assert.Equal(t, "token-new", lock.Token) + assert.Equal(t, int64(8), lock.Generation) + assert.Equal(t, int64(1), lock.Seq) +} diff --git a/weed/cluster/lock_manager/lock_ring.go b/weed/cluster/lock_manager/lock_ring.go index 398f26153..932778240 100644 --- a/weed/cluster/lock_manager/lock_ring.go +++ b/weed/cluster/lock_manager/lock_ring.go @@ -7,7 +7,6 @@ import ( "github.com/seaweedfs/seaweedfs/weed/glog" "github.com/seaweedfs/seaweedfs/weed/pb" - "github.com/seaweedfs/seaweedfs/weed/util" ) type LockRingSnapshot struct { @@ -18,18 +17,18 @@ type LockRingSnapshot struct { type LockRing struct { sync.RWMutex snapshots []*LockRingSnapshot - candidateServers map[pb.ServerAddress]struct{} - lastUpdateTime time.Time lastCompactTime time.Time snapshotInterval time.Duration onTakeSnapshot func(snapshot []pb.ServerAddress) cleanupWg sync.WaitGroup + Ring *HashRing // consistent hash ring + version int64 // monotonic version from master, rejects stale updates } func NewLockRing(snapshotInterval time.Duration) *LockRing { return &LockRing{ snapshotInterval: snapshotInterval, - candidateServers: make(map[pb.ServerAddress]struct{}), + Ring: NewHashRing(DefaultVnodeCount), } } @@ -39,52 +38,26 @@ func (r *LockRing) SetTakeSnapshotCallback(onTakeSnapshot func(snapshot []pb.Ser r.onTakeSnapshot = onTakeSnapshot } -// AddServer adds a server to the ring -// if the previous snapshot passed the snapshot interval, create a new snapshot -func (r *LockRing) AddServer(server pb.ServerAddress) { - glog.V(0).Infof("add server %v", server) - r.Lock() - - if _, found := r.candidateServers[server]; found { - glog.V(0).Infof("add server: already exists %v", server) - r.Unlock() - return - } - r.lastUpdateTime = time.Now() - r.candidateServers[server] = struct{}{} - r.Unlock() - - r.takeSnapshotWithDelayedCompaction() -} - -func (r *LockRing) RemoveServer(server pb.ServerAddress) { - glog.V(0).Infof("remove server %v", server) - - r.Lock() - - if _, found := r.candidateServers[server]; !found { - r.Unlock() - return - } - r.lastUpdateTime = time.Now() - delete(r.candidateServers, server) - r.Unlock() - - r.takeSnapshotWithDelayedCompaction() -} - -func (r *LockRing) SetSnapshot(servers []pb.ServerAddress) { +// SetSnapshot replaces the ring with a new server list from the master. +// The version must be >= the current version, otherwise the update is rejected +// (protects against reordered messages). Version 0 is always accepted (bootstrap). +func (r *LockRing) SetSnapshot(servers []pb.ServerAddress, version int64) bool { sort.Slice(servers, func(i, j int) bool { return servers[i] < servers[j] }) r.Lock() - r.lastUpdateTime = time.Now() - // init candidateServers - for _, server := range servers { - r.candidateServers[server] = struct{}{} + if version > 0 && version < r.version { + glog.V(0).Infof("LockRing: rejecting stale update v%d (current v%d)", version, r.version) + r.Unlock() + return false } + r.version = version + // Update the ring while holding the lock so version and ring state + // are always consistent — prevents a concurrent SetSnapshot from + // seeing the new version but applying its servers to the old ring. + r.Ring.SetServers(servers) r.Unlock() r.addOneSnapshot(servers) @@ -95,23 +68,14 @@ func (r *LockRing) SetSnapshot(servers []pb.ServerAddress) { <-time.After(r.snapshotInterval) r.compactSnapshots() }() + return true } -func (r *LockRing) takeSnapshotWithDelayedCompaction() { - r.doTakeSnapshot() - - r.cleanupWg.Add(1) - go func() { - defer r.cleanupWg.Done() - <-time.After(r.snapshotInterval) - r.compactSnapshots() - }() -} - -func (r *LockRing) doTakeSnapshot() { - servers := r.getSortedServers() - - r.addOneSnapshot(servers) +// Version returns the current ring version. +func (r *LockRing) Version() int64 { + r.RLock() + defer r.RUnlock() + return r.version } func (r *LockRing) addOneSnapshot(servers []pb.ServerAddress) { @@ -138,35 +102,19 @@ func (r *LockRing) compactSnapshots() { r.Lock() defer r.Unlock() - // Always attempt compaction when called, regardless of lastCompactTime - // This ensures proper cleanup even with multiple concurrent compaction requests - ts := time.Now() - // remove old snapshots recentSnapshotIndex := 1 for ; recentSnapshotIndex < len(r.snapshots); recentSnapshotIndex++ { if ts.Sub(r.snapshots[recentSnapshotIndex].ts) > r.snapshotInterval { break } } - // keep the one that has been running for a while if recentSnapshotIndex+1 <= len(r.snapshots) { r.snapshots = r.snapshots[:recentSnapshotIndex+1] } r.lastCompactTime = ts } -func (r *LockRing) getSortedServers() []pb.ServerAddress { - sortedServers := make([]pb.ServerAddress, 0, len(r.candidateServers)) - for server := range r.candidateServers { - sortedServers = append(sortedServers, server) - } - sort.Slice(sortedServers, func(i, j int) bool { - return sortedServers[i] < sortedServers[j] - }) - return sortedServers -} - func (r *LockRing) GetSnapshot() (servers []pb.ServerAddress) { r.RLock() defer r.RUnlock() @@ -178,7 +126,6 @@ func (r *LockRing) GetSnapshot() (servers []pb.ServerAddress) { } // WaitForCleanup waits for all pending cleanup operations to complete -// This is useful for testing to ensure deterministic behavior func (r *LockRing) WaitForCleanup() { r.cleanupWg.Wait() } @@ -190,14 +137,23 @@ func (r *LockRing) GetSnapshotCount() int { return len(r.snapshots) } +// GetPrimaryAndBackup returns the primary and backup servers for a key +// using the consistent hash ring. +func (r *LockRing) GetPrimaryAndBackup(key string) (primary, backup pb.ServerAddress) { + return r.Ring.GetPrimaryAndBackup(key) +} + +// GetPrimary returns the primary server for a key using the consistent hash ring. +func (r *LockRing) GetPrimary(key string) pb.ServerAddress { + return r.Ring.GetPrimary(key) +} + +// hashKeyToServer uses a temporary consistent hash ring for the given server list. func hashKeyToServer(key string, servers []pb.ServerAddress) pb.ServerAddress { if len(servers) == 0 { return "" } - x := util.HashStringToLong(key) - if x < 0 { - x = -x - } - x = x % int64(len(servers)) - return servers[x] + ring := NewHashRing(DefaultVnodeCount) + ring.SetServers(servers) + return ring.GetPrimary(key) } diff --git a/weed/cluster/lock_manager/lock_ring_test.go b/weed/cluster/lock_manager/lock_ring_test.go index f82a5ffe4..39c65c9da 100644 --- a/weed/cluster/lock_manager/lock_ring_test.go +++ b/weed/cluster/lock_manager/lock_ring_test.go @@ -8,84 +8,91 @@ import ( "github.com/stretchr/testify/assert" ) -func TestAddServer(t *testing.T) { +func TestLockRing_SetSnapshot(t *testing.T) { r := NewLockRing(100 * time.Millisecond) - // Add servers - r.AddServer("localhost:8080") - r.AddServer("localhost:8081") - r.AddServer("localhost:8082") - r.AddServer("localhost:8083") - r.AddServer("localhost:8084") + // Set 5 servers + r.SetSnapshot([]pb.ServerAddress{ + "localhost:8080", "localhost:8081", "localhost:8082", + "localhost:8083", "localhost:8084", + }, 1) - // Verify all servers are present servers := r.GetSnapshot() assert.Equal(t, 5, len(servers)) - assert.Contains(t, servers, pb.ServerAddress("localhost:8080")) - assert.Contains(t, servers, pb.ServerAddress("localhost:8081")) - assert.Contains(t, servers, pb.ServerAddress("localhost:8082")) - assert.Contains(t, servers, pb.ServerAddress("localhost:8083")) - assert.Contains(t, servers, pb.ServerAddress("localhost:8084")) - // Remove servers - r.RemoveServer("localhost:8084") - r.RemoveServer("localhost:8082") - r.RemoveServer("localhost:8080") + // Replace with 2 servers + r.SetSnapshot([]pb.ServerAddress{"localhost:8081", "localhost:8083"}, 2) - // Wait for all cleanup operations to complete r.WaitForCleanup() - - // Verify only 2 servers remain (localhost:8081 and localhost:8083) servers = r.GetSnapshot() assert.Equal(t, 2, len(servers)) assert.Contains(t, servers, pb.ServerAddress("localhost:8081")) assert.Contains(t, servers, pb.ServerAddress("localhost:8083")) - // Verify cleanup has happened - wait for snapshot interval and check snapshots are compacted + // Verify compaction time.Sleep(110 * time.Millisecond) r.WaitForCleanup() - - // Verify snapshot history is cleaned up properly (should have at most 2 snapshots after compaction) - snapshotCount := r.GetSnapshotCount() - assert.LessOrEqual(t, snapshotCount, 2, "Snapshot history should be compacted") + assert.LessOrEqual(t, r.GetSnapshotCount(), 2) } -func TestLockRing(t *testing.T) { +func TestLockRing_SnapshotCompaction(t *testing.T) { r := NewLockRing(100 * time.Millisecond) - // Test initial snapshot - r.SetSnapshot([]pb.ServerAddress{"localhost:8080", "localhost:8081"}) + r.SetSnapshot([]pb.ServerAddress{"localhost:8080", "localhost:8081"}, 1) assert.Equal(t, 1, r.GetSnapshotCount()) - servers := r.GetSnapshot() - assert.Equal(t, 2, len(servers)) - assert.Contains(t, servers, pb.ServerAddress("localhost:8080")) - assert.Contains(t, servers, pb.ServerAddress("localhost:8081")) - // Add another server - r.SetSnapshot([]pb.ServerAddress{"localhost:8080", "localhost:8081", "localhost:8082"}) + r.SetSnapshot([]pb.ServerAddress{"localhost:8080", "localhost:8081", "localhost:8082"}, 2) assert.Equal(t, 2, r.GetSnapshotCount()) - servers = r.GetSnapshot() - assert.Equal(t, 3, len(servers)) - assert.Contains(t, servers, pb.ServerAddress("localhost:8082")) - // Wait for cleanup interval and add another server + // Wait for compaction time.Sleep(110 * time.Millisecond) r.WaitForCleanup() - r.SetSnapshot([]pb.ServerAddress{"localhost:8080", "localhost:8081", "localhost:8082", "localhost:8083"}) + + r.SetSnapshot([]pb.ServerAddress{"localhost:8080", "localhost:8081", "localhost:8082", "localhost:8083"}, 3) assert.LessOrEqual(t, r.GetSnapshotCount(), 3) - servers = r.GetSnapshot() + servers := r.GetSnapshot() assert.Equal(t, 4, len(servers)) - assert.Contains(t, servers, pb.ServerAddress("localhost:8083")) - // Wait for cleanup and verify compaction time.Sleep(110 * time.Millisecond) r.WaitForCleanup() assert.LessOrEqual(t, r.GetSnapshotCount(), 2, "Snapshots should be compacted") - // Add final server - r.SetSnapshot([]pb.ServerAddress{"localhost:8080", "localhost:8081", "localhost:8082", "localhost:8083", "localhost:8084"}) + r.SetSnapshot([]pb.ServerAddress{ + "localhost:8080", "localhost:8081", "localhost:8082", + "localhost:8083", "localhost:8084", + }, 4) servers = r.GetSnapshot() assert.Equal(t, 5, len(servers)) - assert.Contains(t, servers, pb.ServerAddress("localhost:8084")) - assert.LessOrEqual(t, r.GetSnapshotCount(), 3) +} + +func TestLockRing_VersionRejectsStale(t *testing.T) { + r := NewLockRing(100 * time.Millisecond) + + // Apply version 3 + ok := r.SetSnapshot([]pb.ServerAddress{"a:1", "b:2", "c:3"}, 3) + assert.True(t, ok) + assert.Equal(t, int64(3), r.Version()) + assert.Equal(t, 3, len(r.GetSnapshot())) + + // Stale version 2 — should be rejected + ok = r.SetSnapshot([]pb.ServerAddress{"x:1"}, 2) + assert.False(t, ok) + assert.Equal(t, int64(3), r.Version()) + assert.Equal(t, 3, len(r.GetSnapshot()), "stale update should not change the ring") + + // Same version 3 — accepted (SetSnapshot accepts version >= current, state-changing) + ok = r.SetSnapshot([]pb.ServerAddress{"a:1", "b:2"}, 3) + assert.True(t, ok) + assert.Equal(t, 2, len(r.GetSnapshot())) + + // Newer version 5 — should be accepted + ok = r.SetSnapshot([]pb.ServerAddress{"d:1", "e:2", "f:3", "g:4"}, 5) + assert.True(t, ok) + assert.Equal(t, int64(5), r.Version()) + assert.Equal(t, 4, len(r.GetSnapshot())) + + // Version 0 always accepted (bootstrap) + ok = r.SetSnapshot([]pb.ServerAddress{"z:1"}, 0) + assert.True(t, ok) + assert.Equal(t, 1, len(r.GetSnapshot())) } diff --git a/weed/cluster/lock_ring_manager.go b/weed/cluster/lock_ring_manager.go new file mode 100644 index 000000000..0113a3aeb --- /dev/null +++ b/weed/cluster/lock_ring_manager.go @@ -0,0 +1,164 @@ +package cluster + +import ( + "sync" + "time" + + "github.com/seaweedfs/seaweedfs/weed/glog" + "github.com/seaweedfs/seaweedfs/weed/pb" + "github.com/seaweedfs/seaweedfs/weed/pb/master_pb" +) + +const LockRingStabilizationInterval = 1 * time.Second + +// LockRingManager tracks filer membership for the distributed lock ring. +// It batches rapid topology changes (e.g., node drop + join) with a +// stabilization timer, then broadcasts the complete member list atomically +// so filers receive a single consistent ring update instead of multiple +// intermediate states. +type LockRingManager struct { + mu sync.Mutex + members map[FilerGroupName]map[pb.ServerAddress]struct{} + version map[FilerGroupName]int64 + lastBroadcast map[FilerGroupName]*master_pb.LockRingUpdate + pendingTimer map[FilerGroupName]*time.Timer + broadcastFn func(resp *master_pb.KeepConnectedResponse) + stabilizeDelay time.Duration +} + +func NewLockRingManager(broadcastFn func(resp *master_pb.KeepConnectedResponse)) *LockRingManager { + return &LockRingManager{ + members: make(map[FilerGroupName]map[pb.ServerAddress]struct{}), + version: make(map[FilerGroupName]int64), + lastBroadcast: make(map[FilerGroupName]*master_pb.LockRingUpdate), + pendingTimer: make(map[FilerGroupName]*time.Timer), + broadcastFn: broadcastFn, + stabilizeDelay: LockRingStabilizationInterval, + } +} + +// AddServer records a filer joining and schedules a batched broadcast. +func (lrm *LockRingManager) AddServer(filerGroup FilerGroupName, address pb.ServerAddress) { + lrm.mu.Lock() + defer lrm.mu.Unlock() + + if _, ok := lrm.members[filerGroup]; !ok { + lrm.members[filerGroup] = make(map[pb.ServerAddress]struct{}) + } + lrm.members[filerGroup][address] = struct{}{} + lrm.scheduleBroadcast(filerGroup) +} + +// RemoveServer records a filer leaving and schedules a batched broadcast. +func (lrm *LockRingManager) RemoveServer(filerGroup FilerGroupName, address pb.ServerAddress) { + lrm.mu.Lock() + defer lrm.mu.Unlock() + + if members, ok := lrm.members[filerGroup]; ok { + delete(members, address) + } + lrm.scheduleBroadcast(filerGroup) +} + +// GetServers returns the current member list for a filer group. +func (lrm *LockRingManager) GetServers(filerGroup FilerGroupName) []string { + lrm.mu.Lock() + defer lrm.mu.Unlock() + + members, ok := lrm.members[filerGroup] + if !ok { + return nil + } + servers := make([]string, 0, len(members)) + for addr := range members { + servers = append(servers, string(addr)) + } + return servers +} + +// GetVersion returns the current version for a filer group. +func (lrm *LockRingManager) GetVersion(filerGroup FilerGroupName) int64 { + lrm.mu.Lock() + defer lrm.mu.Unlock() + return lrm.version[filerGroup] +} + +// GetLastUpdate returns a copy of the most recently broadcast lock-ring snapshot +// for the filer group. It intentionally does not expose pending, unstabilized changes. +func (lrm *LockRingManager) GetLastUpdate(filerGroup FilerGroupName) *master_pb.LockRingUpdate { + lrm.mu.Lock() + defer lrm.mu.Unlock() + + update, ok := lrm.lastBroadcast[filerGroup] + if !ok || update == nil { + return nil + } + cp := *update + cp.Servers = append([]string(nil), update.Servers...) + return &cp +} + +// scheduleBroadcast resets the stabilization timer. If another change arrives +// before the timer fires, the timer resets, batching the changes. +// Caller must hold lrm.mu. +func (lrm *LockRingManager) scheduleBroadcast(filerGroup FilerGroupName) { + if timer, ok := lrm.pendingTimer[filerGroup]; ok { + if !timer.Stop() { + // Timer already fired, callback is running or queued. + // It will pick up the latest state from lrm.members, so + // just schedule a new one for any further changes. + } + } + lrm.pendingTimer[filerGroup] = time.AfterFunc(lrm.stabilizeDelay, func() { + lrm.doBroadcast(filerGroup) + }) +} + +func (lrm *LockRingManager) doBroadcast(filerGroup FilerGroupName) { + lrm.mu.Lock() + // Use wall-clock nanoseconds so the version survives master restarts + // without persistence — a restarted master produces a version greater + // than any pre-restart value (assuming clocks don't jump backward). + version := time.Now().UnixNano() + lrm.version[filerGroup] = version + servers := make([]string, 0) + if members, ok := lrm.members[filerGroup]; ok { + for addr := range members { + servers = append(servers, string(addr)) + } + } + update := &master_pb.LockRingUpdate{ + FilerGroup: string(filerGroup), + Servers: append([]string(nil), servers...), + Version: version, + } + lrm.lastBroadcast[filerGroup] = update + delete(lrm.pendingTimer, filerGroup) + lrm.mu.Unlock() + + glog.V(0).Infof("LockRing: broadcasting ring update for group %q version %d: %v", filerGroup, version, servers) + + if lrm.broadcastFn != nil { + lrm.broadcastFn(&master_pb.KeepConnectedResponse{ + LockRingUpdate: update, + }) + } +} + +// FlushPending fires any pending timer immediately (for testing or shutdown). +func (lrm *LockRingManager) FlushPending(filerGroup FilerGroupName) { + lrm.mu.Lock() + if timer, ok := lrm.pendingTimer[filerGroup]; ok { + if timer.Stop() { + // Timer was pending — we stopped it, so we broadcast now + delete(lrm.pendingTimer, filerGroup) + lrm.mu.Unlock() + lrm.doBroadcast(filerGroup) + } else { + // Timer already fired, callback is running — let it finish + lrm.mu.Unlock() + } + } else { + lrm.mu.Unlock() + } +} diff --git a/weed/cluster/lock_ring_manager_test.go b/weed/cluster/lock_ring_manager_test.go new file mode 100644 index 000000000..981d13d9c --- /dev/null +++ b/weed/cluster/lock_ring_manager_test.go @@ -0,0 +1,229 @@ +package cluster + +import ( + "sync" + "testing" + "time" + + "github.com/seaweedfs/seaweedfs/weed/pb" + "github.com/seaweedfs/seaweedfs/weed/pb/master_pb" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" +) + +func TestLockRingManager_BatchesRapidChanges(t *testing.T) { + var mu sync.Mutex + var broadcasts []*master_pb.LockRingUpdate + + lrm := NewLockRingManager(func(resp *master_pb.KeepConnectedResponse) { + mu.Lock() + if resp.LockRingUpdate != nil { + broadcasts = append(broadcasts, resp.LockRingUpdate) + } + mu.Unlock() + }) + lrm.stabilizeDelay = 100 * time.Millisecond + + group := FilerGroupName("default") + + // Add 3 servers in rapid succession + lrm.AddServer(group, "filer1:8888") + lrm.AddServer(group, "filer2:8888") + lrm.AddServer(group, "filer3:8888") + + // No broadcast should have happened yet (timer hasn't fired) + mu.Lock() + assert.Equal(t, 0, len(broadcasts), "should not broadcast before stabilization delay") + mu.Unlock() + + // Wait for stabilization + time.Sleep(200 * time.Millisecond) + + mu.Lock() + require.Equal(t, 1, len(broadcasts), "should batch into a single broadcast") + assert.Equal(t, 3, len(broadcasts[0].Servers), "should include all 3 servers") + assert.Greater(t, broadcasts[0].Version, int64(0)) + mu.Unlock() +} + +func TestLockRingManager_DropAndJoinBatched(t *testing.T) { + var mu sync.Mutex + var broadcasts []*master_pb.LockRingUpdate + + lrm := NewLockRingManager(func(resp *master_pb.KeepConnectedResponse) { + mu.Lock() + if resp.LockRingUpdate != nil { + broadcasts = append(broadcasts, resp.LockRingUpdate) + } + mu.Unlock() + }) + lrm.stabilizeDelay = 100 * time.Millisecond + + group := FilerGroupName("default") + + // Set up initial state + lrm.AddServer(group, "filer1:8888") + lrm.AddServer(group, "filer2:8888") + lrm.AddServer(group, "filer3:8888") + lrm.FlushPending(group) + + mu.Lock() + broadcasts = nil // reset + mu.Unlock() + + // Simulate drop + join in rapid succession + lrm.RemoveServer(group, "filer3:8888") + lrm.AddServer(group, "filer4:8888") + + // Should not have broadcast yet + mu.Lock() + assert.Equal(t, 0, len(broadcasts)) + mu.Unlock() + + // Wait for stabilization + time.Sleep(200 * time.Millisecond) + + mu.Lock() + require.Equal(t, 1, len(broadcasts), "drop+join should be batched into single broadcast") + servers := broadcasts[0].Servers + assert.Equal(t, 3, len(servers), "should have filer1, filer2, filer4") + // filer3 should be gone, filer4 should be present + serverSet := make(map[string]bool) + for _, s := range servers { + serverSet[s] = true + } + assert.False(t, serverSet["filer3:8888"], "filer3 should be removed") + assert.True(t, serverSet["filer4:8888"], "filer4 should be added") + mu.Unlock() +} + +func TestLockRingManager_VersionIncrements(t *testing.T) { + var mu sync.Mutex + var broadcasts []*master_pb.LockRingUpdate + + lrm := NewLockRingManager(func(resp *master_pb.KeepConnectedResponse) { + mu.Lock() + if resp.LockRingUpdate != nil { + broadcasts = append(broadcasts, resp.LockRingUpdate) + } + mu.Unlock() + }) + lrm.stabilizeDelay = 50 * time.Millisecond + + group := FilerGroupName("default") + + lrm.AddServer(group, "filer1:8888") + time.Sleep(100 * time.Millisecond) + + lrm.AddServer(group, "filer2:8888") + time.Sleep(100 * time.Millisecond) + + mu.Lock() + require.Equal(t, 2, len(broadcasts)) + assert.Greater(t, broadcasts[0].Version, int64(0), "version should be positive") + assert.Greater(t, broadcasts[1].Version, broadcasts[0].Version, "versions should be monotonically increasing") + mu.Unlock() +} + +func TestLockRingManager_FlushPending(t *testing.T) { + var mu sync.Mutex + var broadcasts []*master_pb.LockRingUpdate + + lrm := NewLockRingManager(func(resp *master_pb.KeepConnectedResponse) { + mu.Lock() + if resp.LockRingUpdate != nil { + broadcasts = append(broadcasts, resp.LockRingUpdate) + } + mu.Unlock() + }) + lrm.stabilizeDelay = 10 * time.Second // long delay + + group := FilerGroupName("default") + + lrm.AddServer(group, "filer1:8888") + lrm.AddServer(group, "filer2:8888") + + // Flush immediately + lrm.FlushPending(group) + + mu.Lock() + require.Equal(t, 1, len(broadcasts)) + assert.Equal(t, 2, len(broadcasts[0].Servers)) + mu.Unlock() +} + +func TestLockRingManager_MultipleGroups(t *testing.T) { + var mu sync.Mutex + broadcastsByGroup := make(map[string][]*master_pb.LockRingUpdate) + + lrm := NewLockRingManager(func(resp *master_pb.KeepConnectedResponse) { + mu.Lock() + if resp.LockRingUpdate != nil { + broadcastsByGroup[resp.LockRingUpdate.FilerGroup] = append( + broadcastsByGroup[resp.LockRingUpdate.FilerGroup], resp.LockRingUpdate) + } + mu.Unlock() + }) + lrm.stabilizeDelay = 50 * time.Millisecond + + lrm.AddServer("group1", "filer1:8888") + lrm.AddServer("group2", "filer2:8888") + + time.Sleep(100 * time.Millisecond) + + mu.Lock() + assert.Equal(t, 1, len(broadcastsByGroup["group1"])) + assert.Equal(t, 1, len(broadcastsByGroup["group2"])) + assert.Equal(t, []string{"filer1:8888"}, broadcastsByGroup["group1"][0].Servers) + assert.Equal(t, []string{"filer2:8888"}, broadcastsByGroup["group2"][0].Servers) + mu.Unlock() +} + +func TestLockRingManager_GetServers(t *testing.T) { + lrm := NewLockRingManager(nil) + + group := FilerGroupName("default") + lrm.AddServer(group, "filer1:8888") + lrm.AddServer(group, "filer2:8888") + + servers := lrm.GetServers(group) + assert.Equal(t, 2, len(servers)) + + // Contains both + serverSet := make(map[string]bool) + for _, s := range servers { + serverSet[s] = true + } + assert.True(t, serverSet["filer1:8888"]) + assert.True(t, serverSet["filer2:8888"]) + + // Remove one + lrm.RemoveServer(group, "filer1:8888") + servers = lrm.GetServers(group) + assert.Equal(t, 1, len(servers)) + assert.Equal(t, "filer2:8888", servers[0]) +} + +func TestLockRingManager_NoBroadcastWithoutFn(t *testing.T) { + // No panic when broadcastFn is nil + lrm := NewLockRingManager(nil) + lrm.stabilizeDelay = 10 * time.Millisecond + + lrm.AddServer("default", pb.ServerAddress("filer1:8888")) + time.Sleep(50 * time.Millisecond) // should not panic +} + +func TestLockRingManager_GetLastUpdateReturnsBroadcastState(t *testing.T) { + lrm := NewLockRingManager(nil) + + group := FilerGroupName("default") + lrm.AddServer(group, "filer1:8888") + lrm.AddServer(group, "filer2:8888") + lrm.FlushPending(group) + + update := lrm.GetLastUpdate(group) + require.NotNil(t, update) + assert.Equal(t, "default", update.FilerGroup) + assert.ElementsMatch(t, []string{"filer1:8888", "filer2:8888"}, update.Servers) + assert.Greater(t, update.Version, int64(0)) +} diff --git a/weed/filer/empty_folder_cleanup/empty_folder_cleaner.go b/weed/filer/empty_folder_cleanup/empty_folder_cleaner.go index 6af15a464..4feb16184 100644 --- a/weed/filer/empty_folder_cleanup/empty_folder_cleaner.go +++ b/weed/filer/empty_folder_cleanup/empty_folder_cleaner.go @@ -107,24 +107,11 @@ func (efc *EmptyFolderCleaner) IsEnabled() bool { // ownsFolder checks if this filer owns the folder via consistent hashing func (efc *EmptyFolderCleaner) ownsFolder(folder string) bool { - servers := efc.lockRing.GetSnapshot() - if len(servers) <= 1 { - return true // Single filer case + primary := efc.lockRing.GetPrimary(folder) + if primary == "" { + return true // Single filer case or no servers } - return efc.hashKeyToServer(folder, servers) == efc.host -} - -// hashKeyToServer uses consistent hashing to map a folder to a server -func (efc *EmptyFolderCleaner) hashKeyToServer(key string, servers []pb.ServerAddress) pb.ServerAddress { - if len(servers) == 0 { - return "" - } - x := util.HashStringToLong(key) - if x < 0 { - x = -x - } - x = x % int64(len(servers)) - return servers[x] + return primary == efc.host } // OnDeleteEvent is called when a file or directory is deleted diff --git a/weed/filer/empty_folder_cleanup/empty_folder_cleaner_test.go b/weed/filer/empty_folder_cleanup/empty_folder_cleaner_test.go index 645746d55..e3621dcd3 100644 --- a/weed/filer/empty_folder_cleanup/empty_folder_cleaner_test.go +++ b/weed/filer/empty_folder_cleanup/empty_folder_cleaner_test.go @@ -163,7 +163,7 @@ func TestEmptyFolderCleaner_ownsFolder(t *testing.T) { "filer2:8888", "filer3:8888", } - lockRing.SetSnapshot(servers) + lockRing.SetSnapshot(servers, 0) // Create cleaner for filer1 cleaner1 := &EmptyFolderCleaner{ @@ -213,7 +213,7 @@ func TestEmptyFolderCleaner_ownsFolder(t *testing.T) { func TestEmptyFolderCleaner_ownsFolder_singleServer(t *testing.T) { // Create a LockRing with a single server lockRing := lock_manager.NewLockRing(5 * time.Second) - lockRing.SetSnapshot([]pb.ServerAddress{"filer1:8888"}) + lockRing.SetSnapshot([]pb.ServerAddress{"filer1:8888"}, 0) cleaner := &EmptyFolderCleaner{ lockRing: lockRing, @@ -251,7 +251,7 @@ func TestEmptyFolderCleaner_ownsFolder_emptyRing(t *testing.T) { func TestEmptyFolderCleaner_OnCreateEvent_cancelsCleanup(t *testing.T) { lockRing := lock_manager.NewLockRing(5 * time.Second) - lockRing.SetSnapshot([]pb.ServerAddress{"filer1:8888"}) + lockRing.SetSnapshot([]pb.ServerAddress{"filer1:8888"}, 0) cleaner := &EmptyFolderCleaner{ lockRing: lockRing, @@ -287,7 +287,7 @@ func TestEmptyFolderCleaner_OnCreateEvent_cancelsCleanup(t *testing.T) { func TestEmptyFolderCleaner_OnDeleteEvent_deduplication(t *testing.T) { lockRing := lock_manager.NewLockRing(5 * time.Second) - lockRing.SetSnapshot([]pb.ServerAddress{"filer1:8888"}) + lockRing.SetSnapshot([]pb.ServerAddress{"filer1:8888"}, 0) cleaner := &EmptyFolderCleaner{ lockRing: lockRing, @@ -317,7 +317,7 @@ func TestEmptyFolderCleaner_OnDeleteEvent_deduplication(t *testing.T) { func TestEmptyFolderCleaner_OnDeleteEvent_multipleFolders(t *testing.T) { lockRing := lock_manager.NewLockRing(5 * time.Second) - lockRing.SetSnapshot([]pb.ServerAddress{"filer1:8888"}) + lockRing.SetSnapshot([]pb.ServerAddress{"filer1:8888"}, 0) cleaner := &EmptyFolderCleaner{ lockRing: lockRing, @@ -346,7 +346,7 @@ func TestEmptyFolderCleaner_OnDeleteEvent_multipleFolders(t *testing.T) { func TestEmptyFolderCleaner_OnDeleteEvent_notOwner(t *testing.T) { lockRing := lock_manager.NewLockRing(5 * time.Second) - lockRing.SetSnapshot([]pb.ServerAddress{"filer1:8888", "filer2:8888"}) + lockRing.SetSnapshot([]pb.ServerAddress{"filer1:8888", "filer2:8888"}, 0) // Create cleaner for filer that doesn't own the folder cleaner := &EmptyFolderCleaner{ @@ -385,7 +385,7 @@ func TestEmptyFolderCleaner_OnDeleteEvent_notOwner(t *testing.T) { func TestEmptyFolderCleaner_OnDeleteEvent_disabled(t *testing.T) { lockRing := lock_manager.NewLockRing(5 * time.Second) - lockRing.SetSnapshot([]pb.ServerAddress{"filer1:8888"}) + lockRing.SetSnapshot([]pb.ServerAddress{"filer1:8888"}, 0) cleaner := &EmptyFolderCleaner{ lockRing: lockRing, @@ -413,7 +413,7 @@ func TestEmptyFolderCleaner_OnDeleteEvent_disabled(t *testing.T) { func TestEmptyFolderCleaner_OnDeleteEvent_directoryDeletion(t *testing.T) { lockRing := lock_manager.NewLockRing(5 * time.Second) - lockRing.SetSnapshot([]pb.ServerAddress{"filer1:8888"}) + lockRing.SetSnapshot([]pb.ServerAddress{"filer1:8888"}, 0) cleaner := &EmptyFolderCleaner{ lockRing: lockRing, @@ -442,7 +442,7 @@ func TestEmptyFolderCleaner_OnDeleteEvent_directoryDeletion(t *testing.T) { func TestEmptyFolderCleaner_cachedCounts(t *testing.T) { lockRing := lock_manager.NewLockRing(5 * time.Second) - lockRing.SetSnapshot([]pb.ServerAddress{"filer1:8888"}) + lockRing.SetSnapshot([]pb.ServerAddress{"filer1:8888"}, 0) cleaner := &EmptyFolderCleaner{ lockRing: lockRing, @@ -491,7 +491,7 @@ func TestEmptyFolderCleaner_cachedCounts(t *testing.T) { func TestEmptyFolderCleaner_Stop(t *testing.T) { lockRing := lock_manager.NewLockRing(5 * time.Second) - lockRing.SetSnapshot([]pb.ServerAddress{"filer1:8888"}) + lockRing.SetSnapshot([]pb.ServerAddress{"filer1:8888"}, 0) cleaner := &EmptyFolderCleaner{ lockRing: lockRing, @@ -526,7 +526,7 @@ func TestEmptyFolderCleaner_Stop(t *testing.T) { func TestEmptyFolderCleaner_cacheEviction(t *testing.T) { lockRing := lock_manager.NewLockRing(5 * time.Second) - lockRing.SetSnapshot([]pb.ServerAddress{"filer1:8888"}) + lockRing.SetSnapshot([]pb.ServerAddress{"filer1:8888"}, 0) cleaner := &EmptyFolderCleaner{ lockRing: lockRing, @@ -581,7 +581,7 @@ func TestEmptyFolderCleaner_cacheEviction(t *testing.T) { func TestEmptyFolderCleaner_cacheEviction_skipsEntriesInQueue(t *testing.T) { lockRing := lock_manager.NewLockRing(5 * time.Second) - lockRing.SetSnapshot([]pb.ServerAddress{"filer1:8888"}) + lockRing.SetSnapshot([]pb.ServerAddress{"filer1:8888"}, 0) cleaner := &EmptyFolderCleaner{ lockRing: lockRing, @@ -615,7 +615,7 @@ func TestEmptyFolderCleaner_cacheEviction_skipsEntriesInQueue(t *testing.T) { func TestEmptyFolderCleaner_queueFIFOOrder(t *testing.T) { lockRing := lock_manager.NewLockRing(5 * time.Second) - lockRing.SetSnapshot([]pb.ServerAddress{"filer1:8888"}) + lockRing.SetSnapshot([]pb.ServerAddress{"filer1:8888"}, 0) cleaner := &EmptyFolderCleaner{ lockRing: lockRing, @@ -657,7 +657,7 @@ func TestEmptyFolderCleaner_queueFIFOOrder(t *testing.T) { func TestEmptyFolderCleaner_processCleanupQueue_drainsAllOnceTriggered(t *testing.T) { lockRing := lock_manager.NewLockRing(5 * time.Second) - lockRing.SetSnapshot([]pb.ServerAddress{"filer1:8888"}) + lockRing.SetSnapshot([]pb.ServerAddress{"filer1:8888"}, 0) var deleted []string mock := &mockFilerOps{ @@ -701,7 +701,7 @@ func TestEmptyFolderCleaner_processCleanupQueue_drainsAllOnceTriggered(t *testin func TestEmptyFolderCleaner_executeCleanup_bucketPolicyDisabledSkips(t *testing.T) { lockRing := lock_manager.NewLockRing(5 * time.Second) - lockRing.SetSnapshot([]pb.ServerAddress{"filer1:8888"}) + lockRing.SetSnapshot([]pb.ServerAddress{"filer1:8888"}, 0) var deleted []string mock := &mockFilerOps{ @@ -763,7 +763,7 @@ func TestEmptyFolderCleaner_executeCleanup_directoryMarker(t *testing.T) { for _, tc := range testCases { t.Run(tc.name, func(t *testing.T) { lockRing := lock_manager.NewLockRing(5 * time.Second) - lockRing.SetSnapshot([]pb.ServerAddress{"filer1:8888"}) + lockRing.SetSnapshot([]pb.ServerAddress{"filer1:8888"}, 0) var deleted []string mock := &mockFilerOps{ diff --git a/weed/filer/filer.go b/weed/filer/filer.go index 49a37b138..c16c41f57 100644 --- a/weed/filer/filer.go +++ b/weed/filer/filer.go @@ -119,7 +119,7 @@ func (f *Filer) AggregateFromPeers(self pb.ServerAddress, existingNodes []*maste address := pb.ServerAddress(node.Address) snapshot = append(snapshot, address) } - f.Dlm.LockRing.SetSnapshot(snapshot) + f.Dlm.LockRing.SetSnapshot(snapshot, 0) glog.V(0).Infof("%s aggregate from peers %+v", self, snapshot) // Initialize the empty folder cleaner using the same LockRing as Dlm for consistent hashing @@ -130,15 +130,18 @@ func (f *Filer) AggregateFromPeers(self pb.ServerAddress, existingNodes []*maste if update.NodeType != cluster.FilerType { return } - address := pb.ServerAddress(update.Address) - - if update.IsAdd { - f.Dlm.LockRing.AddServer(address) - } else { - f.Dlm.LockRing.RemoveServer(address) - } + // Lock ring is now managed by the master via LockRingUpdate, + // so we no longer call AddServer/RemoveServer here. f.MetaAggregator.OnPeerUpdate(update, startFrom) }) + f.MasterClient.SetOnLockRingUpdateFn(func(update *master_pb.LockRingUpdate) { + var servers []pb.ServerAddress + for _, s := range update.Servers { + servers = append(servers, pb.ServerAddress(s)) + } + glog.V(0).Infof("LockRing: applying master ring update v%d: %v", update.Version, servers) + f.Dlm.LockRing.SetSnapshot(servers, update.Version) + }) for _, peerUpdate := range existingNodes { f.MetaAggregator.OnPeerUpdate(peerUpdate, startFrom) diff --git a/weed/pb/filer.proto b/weed/pb/filer.proto index 47f50a385..d7067def0 100644 --- a/weed/pb/filer.proto +++ b/weed/pb/filer.proto @@ -84,6 +84,8 @@ service SeaweedFiler { // distributed lock management internal use only rpc TransferLocks(TransferLocksRequest) returns (TransferLocksResponse) { } + rpc ReplicateLock(ReplicateLockRequest) returns (ReplicateLockResponse) { + } } ////////////////////////////////////////////////// @@ -522,6 +524,7 @@ message LockResponse { string lock_owner = 2; string lock_host_moved_to = 3; string error = 4; + int64 generation = 5; } message UnlockRequest { string name = 1; @@ -544,12 +547,26 @@ message Lock { string renew_token = 2; int64 expired_at_ns = 3; string owner = 4; + int64 generation = 5; + bool is_backup = 6; + int64 seq = 7; } message TransferLocksRequest { repeated Lock locks = 1; } message TransferLocksResponse { } +message ReplicateLockRequest { + string name = 1; + string renew_token = 2; + int64 expired_at_ns = 3; + string owner = 4; + int64 generation = 5; + bool is_unlock = 6; + int64 seq = 7; +} +message ReplicateLockResponse { +} ////////////////////////////////////////////////// // StreamMutateEntry: ordered bidirectional streaming for all filer mutations. diff --git a/weed/pb/filer_pb/filer.pb.go b/weed/pb/filer_pb/filer.pb.go index d8399fec4..45ea561a8 100644 --- a/weed/pb/filer_pb/filer.pb.go +++ b/weed/pb/filer_pb/filer.pb.go @@ -3991,6 +3991,7 @@ type LockResponse struct { LockOwner string `protobuf:"bytes,2,opt,name=lock_owner,json=lockOwner,proto3" json:"lock_owner,omitempty"` LockHostMovedTo string `protobuf:"bytes,3,opt,name=lock_host_moved_to,json=lockHostMovedTo,proto3" json:"lock_host_moved_to,omitempty"` Error string `protobuf:"bytes,4,opt,name=error,proto3" json:"error,omitempty"` + Generation int64 `protobuf:"varint,5,opt,name=generation,proto3" json:"generation,omitempty"` unknownFields protoimpl.UnknownFields sizeCache protoimpl.SizeCache } @@ -4053,6 +4054,13 @@ func (x *LockResponse) GetError() string { return "" } +func (x *LockResponse) GetGeneration() int64 { + if x != nil { + return x.Generation + } + return 0 +} + type UnlockRequest struct { state protoimpl.MessageState `protogen:"open.v1"` Name string `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"` @@ -4267,6 +4275,9 @@ type Lock struct { RenewToken string `protobuf:"bytes,2,opt,name=renew_token,json=renewToken,proto3" json:"renew_token,omitempty"` ExpiredAtNs int64 `protobuf:"varint,3,opt,name=expired_at_ns,json=expiredAtNs,proto3" json:"expired_at_ns,omitempty"` Owner string `protobuf:"bytes,4,opt,name=owner,proto3" json:"owner,omitempty"` + Generation int64 `protobuf:"varint,5,opt,name=generation,proto3" json:"generation,omitempty"` + IsBackup bool `protobuf:"varint,6,opt,name=is_backup,json=isBackup,proto3" json:"is_backup,omitempty"` + Seq int64 `protobuf:"varint,7,opt,name=seq,proto3" json:"seq,omitempty"` unknownFields protoimpl.UnknownFields sizeCache protoimpl.SizeCache } @@ -4329,6 +4340,27 @@ func (x *Lock) GetOwner() string { return "" } +func (x *Lock) GetGeneration() int64 { + if x != nil { + return x.Generation + } + return 0 +} + +func (x *Lock) GetIsBackup() bool { + if x != nil { + return x.IsBackup + } + return false +} + +func (x *Lock) GetSeq() int64 { + if x != nil { + return x.Seq + } + return 0 +} + type TransferLocksRequest struct { state protoimpl.MessageState `protogen:"open.v1"` Locks []*Lock `protobuf:"bytes,1,rep,name=locks,proto3" json:"locks,omitempty"` @@ -4409,6 +4441,134 @@ func (*TransferLocksResponse) Descriptor() ([]byte, []int) { return file_filer_proto_rawDescGZIP(), []int{66} } +type ReplicateLockRequest struct { + state protoimpl.MessageState `protogen:"open.v1"` + Name string `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"` + RenewToken string `protobuf:"bytes,2,opt,name=renew_token,json=renewToken,proto3" json:"renew_token,omitempty"` + ExpiredAtNs int64 `protobuf:"varint,3,opt,name=expired_at_ns,json=expiredAtNs,proto3" json:"expired_at_ns,omitempty"` + Owner string `protobuf:"bytes,4,opt,name=owner,proto3" json:"owner,omitempty"` + Generation int64 `protobuf:"varint,5,opt,name=generation,proto3" json:"generation,omitempty"` + IsUnlock bool `protobuf:"varint,6,opt,name=is_unlock,json=isUnlock,proto3" json:"is_unlock,omitempty"` + Seq int64 `protobuf:"varint,7,opt,name=seq,proto3" json:"seq,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ReplicateLockRequest) Reset() { + *x = ReplicateLockRequest{} + mi := &file_filer_proto_msgTypes[67] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ReplicateLockRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ReplicateLockRequest) ProtoMessage() {} + +func (x *ReplicateLockRequest) ProtoReflect() protoreflect.Message { + mi := &file_filer_proto_msgTypes[67] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ReplicateLockRequest.ProtoReflect.Descriptor instead. +func (*ReplicateLockRequest) Descriptor() ([]byte, []int) { + return file_filer_proto_rawDescGZIP(), []int{67} +} + +func (x *ReplicateLockRequest) GetName() string { + if x != nil { + return x.Name + } + return "" +} + +func (x *ReplicateLockRequest) GetRenewToken() string { + if x != nil { + return x.RenewToken + } + return "" +} + +func (x *ReplicateLockRequest) GetExpiredAtNs() int64 { + if x != nil { + return x.ExpiredAtNs + } + return 0 +} + +func (x *ReplicateLockRequest) GetOwner() string { + if x != nil { + return x.Owner + } + return "" +} + +func (x *ReplicateLockRequest) GetGeneration() int64 { + if x != nil { + return x.Generation + } + return 0 +} + +func (x *ReplicateLockRequest) GetIsUnlock() bool { + if x != nil { + return x.IsUnlock + } + return false +} + +func (x *ReplicateLockRequest) GetSeq() int64 { + if x != nil { + return x.Seq + } + return 0 +} + +type ReplicateLockResponse struct { + state protoimpl.MessageState `protogen:"open.v1"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ReplicateLockResponse) Reset() { + *x = ReplicateLockResponse{} + mi := &file_filer_proto_msgTypes[68] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ReplicateLockResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ReplicateLockResponse) ProtoMessage() {} + +func (x *ReplicateLockResponse) ProtoReflect() protoreflect.Message { + mi := &file_filer_proto_msgTypes[68] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ReplicateLockResponse.ProtoReflect.Descriptor instead. +func (*ReplicateLockResponse) Descriptor() ([]byte, []int) { + return file_filer_proto_rawDescGZIP(), []int{68} +} + type StreamMutateEntryRequest struct { state protoimpl.MessageState `protogen:"open.v1"` RequestId uint64 `protobuf:"varint,1,opt,name=request_id,json=requestId,proto3" json:"request_id,omitempty"` @@ -4425,7 +4585,7 @@ type StreamMutateEntryRequest struct { func (x *StreamMutateEntryRequest) Reset() { *x = StreamMutateEntryRequest{} - mi := &file_filer_proto_msgTypes[67] + mi := &file_filer_proto_msgTypes[69] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -4437,7 +4597,7 @@ func (x *StreamMutateEntryRequest) String() string { func (*StreamMutateEntryRequest) ProtoMessage() {} func (x *StreamMutateEntryRequest) ProtoReflect() protoreflect.Message { - mi := &file_filer_proto_msgTypes[67] + mi := &file_filer_proto_msgTypes[69] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -4450,7 +4610,7 @@ func (x *StreamMutateEntryRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use StreamMutateEntryRequest.ProtoReflect.Descriptor instead. func (*StreamMutateEntryRequest) Descriptor() ([]byte, []int) { - return file_filer_proto_rawDescGZIP(), []int{67} + return file_filer_proto_rawDescGZIP(), []int{69} } func (x *StreamMutateEntryRequest) GetRequestId() uint64 { @@ -4550,7 +4710,7 @@ type StreamMutateEntryResponse struct { func (x *StreamMutateEntryResponse) Reset() { *x = StreamMutateEntryResponse{} - mi := &file_filer_proto_msgTypes[68] + mi := &file_filer_proto_msgTypes[70] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -4562,7 +4722,7 @@ func (x *StreamMutateEntryResponse) String() string { func (*StreamMutateEntryResponse) ProtoMessage() {} func (x *StreamMutateEntryResponse) ProtoReflect() protoreflect.Message { - mi := &file_filer_proto_msgTypes[68] + mi := &file_filer_proto_msgTypes[70] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -4575,7 +4735,7 @@ func (x *StreamMutateEntryResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use StreamMutateEntryResponse.ProtoReflect.Descriptor instead. func (*StreamMutateEntryResponse) Descriptor() ([]byte, []int) { - return file_filer_proto_rawDescGZIP(), []int{68} + return file_filer_proto_rawDescGZIP(), []int{70} } func (x *StreamMutateEntryResponse) GetRequestId() uint64 { @@ -4689,7 +4849,7 @@ type LocateBrokerResponse_Resource struct { func (x *LocateBrokerResponse_Resource) Reset() { *x = LocateBrokerResponse_Resource{} - mi := &file_filer_proto_msgTypes[72] + mi := &file_filer_proto_msgTypes[74] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -4701,7 +4861,7 @@ func (x *LocateBrokerResponse_Resource) String() string { func (*LocateBrokerResponse_Resource) ProtoMessage() {} func (x *LocateBrokerResponse_Resource) ProtoReflect() protoreflect.Message { - mi := &file_filer_proto_msgTypes[72] + mi := &file_filer_proto_msgTypes[74] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -4755,7 +4915,7 @@ type FilerConf_PathConf struct { func (x *FilerConf_PathConf) Reset() { *x = FilerConf_PathConf{} - mi := &file_filer_proto_msgTypes[73] + mi := &file_filer_proto_msgTypes[75] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -4767,7 +4927,7 @@ func (x *FilerConf_PathConf) String() string { func (*FilerConf_PathConf) ProtoMessage() {} func (x *FilerConf_PathConf) ProtoReflect() protoreflect.Message { - mi := &file_filer_proto_msgTypes[73] + mi := &file_filer_proto_msgTypes[75] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5258,14 +5418,17 @@ const file_filer_proto_rawDesc = "" + "\vrenew_token\x18\x03 \x01(\tR\n" + "renewToken\x12\x19\n" + "\bis_moved\x18\x04 \x01(\bR\aisMoved\x12\x14\n" + - "\x05owner\x18\x05 \x01(\tR\x05owner\"\x91\x01\n" + + "\x05owner\x18\x05 \x01(\tR\x05owner\"\xb1\x01\n" + "\fLockResponse\x12\x1f\n" + "\vrenew_token\x18\x01 \x01(\tR\n" + "renewToken\x12\x1d\n" + "\n" + "lock_owner\x18\x02 \x01(\tR\tlockOwner\x12+\n" + "\x12lock_host_moved_to\x18\x03 \x01(\tR\x0flockHostMovedTo\x12\x14\n" + - "\x05error\x18\x04 \x01(\tR\x05error\"_\n" + + "\x05error\x18\x04 \x01(\tR\x05error\x12\x1e\n" + + "\n" + + "generation\x18\x05 \x01(\x03R\n" + + "generation\"_\n" + "\rUnlockRequest\x12\x12\n" + "\x04name\x18\x01 \x01(\tR\x04name\x12\x1f\n" + "\vrenew_token\x18\x02 \x01(\tR\n" + @@ -5278,16 +5441,33 @@ const file_filer_proto_rawDesc = "" + "\x04name\x18\x01 \x01(\tR\x04name\x12\x19\n" + "\bis_moved\x18\x02 \x01(\bR\aisMoved\"-\n" + "\x15FindLockOwnerResponse\x12\x14\n" + - "\x05owner\x18\x01 \x01(\tR\x05owner\"u\n" + + "\x05owner\x18\x01 \x01(\tR\x05owner\"\xc4\x01\n" + "\x04Lock\x12\x12\n" + "\x04name\x18\x01 \x01(\tR\x04name\x12\x1f\n" + "\vrenew_token\x18\x02 \x01(\tR\n" + "renewToken\x12\"\n" + "\rexpired_at_ns\x18\x03 \x01(\x03R\vexpiredAtNs\x12\x14\n" + - "\x05owner\x18\x04 \x01(\tR\x05owner\"<\n" + + "\x05owner\x18\x04 \x01(\tR\x05owner\x12\x1e\n" + + "\n" + + "generation\x18\x05 \x01(\x03R\n" + + "generation\x12\x1b\n" + + "\tis_backup\x18\x06 \x01(\bR\bisBackup\x12\x10\n" + + "\x03seq\x18\a \x01(\x03R\x03seq\"<\n" + "\x14TransferLocksRequest\x12$\n" + "\x05locks\x18\x01 \x03(\v2\x0e.filer_pb.LockR\x05locks\"\x17\n" + - "\x15TransferLocksResponse\"\xe6\x02\n" + + "\x15TransferLocksResponse\"\xd4\x01\n" + + "\x14ReplicateLockRequest\x12\x12\n" + + "\x04name\x18\x01 \x01(\tR\x04name\x12\x1f\n" + + "\vrenew_token\x18\x02 \x01(\tR\n" + + "renewToken\x12\"\n" + + "\rexpired_at_ns\x18\x03 \x01(\x03R\vexpiredAtNs\x12\x14\n" + + "\x05owner\x18\x04 \x01(\tR\x05owner\x12\x1e\n" + + "\n" + + "generation\x18\x05 \x01(\x03R\n" + + "generation\x12\x1b\n" + + "\tis_unlock\x18\x06 \x01(\bR\bisUnlock\x12\x10\n" + + "\x03seq\x18\a \x01(\x03R\x03seq\"\x17\n" + + "\x15ReplicateLockResponse\"\xe6\x02\n" + "\x18StreamMutateEntryRequest\x12\x1d\n" + "\n" + "request_id\x18\x01 \x01(\x04R\trequestId\x12E\n" + @@ -5321,7 +5501,7 @@ const file_filer_proto_rawDesc = "" + "\x0ePARENT_IS_FILE\x10\x02\x12\x19\n" + "\x15EXISTING_IS_DIRECTORY\x10\x03\x12\x14\n" + "\x10EXISTING_IS_FILE\x10\x04\x12\x18\n" + - "\x14ENTRY_ALREADY_EXISTS\x10\x052\xdb\x11\n" + + "\x14ENTRY_ALREADY_EXISTS\x10\x052\xaf\x12\n" + "\fSeaweedFiler\x12g\n" + "\x14LookupDirectoryEntry\x12%.filer_pb.LookupDirectoryEntryRequest\x1a&.filer_pb.LookupDirectoryEntryResponse\"\x00\x12N\n" + "\vListEntries\x12\x1c.filer_pb.ListEntriesRequest\x1a\x1d.filer_pb.ListEntriesResponse\"\x000\x01\x12L\n" + @@ -5349,7 +5529,8 @@ const file_filer_proto_rawDesc = "" + "\x0fDistributedLock\x12\x15.filer_pb.LockRequest\x1a\x16.filer_pb.LockResponse\"\x00\x12H\n" + "\x11DistributedUnlock\x12\x17.filer_pb.UnlockRequest\x1a\x18.filer_pb.UnlockResponse\"\x00\x12R\n" + "\rFindLockOwner\x12\x1e.filer_pb.FindLockOwnerRequest\x1a\x1f.filer_pb.FindLockOwnerResponse\"\x00\x12R\n" + - "\rTransferLocks\x12\x1e.filer_pb.TransferLocksRequest\x1a\x1f.filer_pb.TransferLocksResponse\"\x00BO\n" + + "\rTransferLocks\x12\x1e.filer_pb.TransferLocksRequest\x1a\x1f.filer_pb.TransferLocksResponse\"\x00\x12R\n" + + "\rReplicateLock\x12\x1e.filer_pb.ReplicateLockRequest\x1a\x1f.filer_pb.ReplicateLockResponse\"\x00BO\n" + "\x10seaweedfs.clientB\n" + "FilerProtoZ/github.com/seaweedfs/seaweedfs/weed/pb/filer_pbb\x06proto3" @@ -5366,7 +5547,7 @@ func file_filer_proto_rawDescGZIP() []byte { } var file_filer_proto_enumTypes = make([]protoimpl.EnumInfo, 2) -var file_filer_proto_msgTypes = make([]protoimpl.MessageInfo, 74) +var file_filer_proto_msgTypes = make([]protoimpl.MessageInfo, 76) var file_filer_proto_goTypes = []any{ (SSEType)(0), // 0: filer_pb.SSEType (FilerError)(0), // 1: filer_pb.FilerError @@ -5437,20 +5618,22 @@ var file_filer_proto_goTypes = []any{ (*Lock)(nil), // 66: filer_pb.Lock (*TransferLocksRequest)(nil), // 67: filer_pb.TransferLocksRequest (*TransferLocksResponse)(nil), // 68: filer_pb.TransferLocksResponse - (*StreamMutateEntryRequest)(nil), // 69: filer_pb.StreamMutateEntryRequest - (*StreamMutateEntryResponse)(nil), // 70: filer_pb.StreamMutateEntryResponse - nil, // 71: filer_pb.Entry.ExtendedEntry - nil, // 72: filer_pb.UpdateEntryRequest.ExpectedExtendedEntry - nil, // 73: filer_pb.LookupVolumeResponse.LocationsMapEntry - (*LocateBrokerResponse_Resource)(nil), // 74: filer_pb.LocateBrokerResponse.Resource - (*FilerConf_PathConf)(nil), // 75: filer_pb.FilerConf.PathConf + (*ReplicateLockRequest)(nil), // 69: filer_pb.ReplicateLockRequest + (*ReplicateLockResponse)(nil), // 70: filer_pb.ReplicateLockResponse + (*StreamMutateEntryRequest)(nil), // 71: filer_pb.StreamMutateEntryRequest + (*StreamMutateEntryResponse)(nil), // 72: filer_pb.StreamMutateEntryResponse + nil, // 73: filer_pb.Entry.ExtendedEntry + nil, // 74: filer_pb.UpdateEntryRequest.ExpectedExtendedEntry + nil, // 75: filer_pb.LookupVolumeResponse.LocationsMapEntry + (*LocateBrokerResponse_Resource)(nil), // 76: filer_pb.LocateBrokerResponse.Resource + (*FilerConf_PathConf)(nil), // 77: filer_pb.FilerConf.PathConf } var file_filer_proto_depIdxs = []int32{ 7, // 0: filer_pb.LookupDirectoryEntryResponse.entry:type_name -> filer_pb.Entry 7, // 1: filer_pb.ListEntriesResponse.entry:type_name -> filer_pb.Entry 10, // 2: filer_pb.Entry.chunks:type_name -> filer_pb.FileChunk 13, // 3: filer_pb.Entry.attributes:type_name -> filer_pb.FuseAttributes - 71, // 4: filer_pb.Entry.extended:type_name -> filer_pb.Entry.ExtendedEntry + 73, // 4: filer_pb.Entry.extended:type_name -> filer_pb.Entry.ExtendedEntry 6, // 5: filer_pb.Entry.remote_entry:type_name -> filer_pb.RemoteEntry 7, // 6: filer_pb.FullEntry.entry:type_name -> filer_pb.Entry 7, // 7: filer_pb.EventNotification.old_entry:type_name -> filer_pb.Entry @@ -5463,22 +5646,22 @@ var file_filer_proto_depIdxs = []int32{ 44, // 14: filer_pb.CreateEntryResponse.metadata_event:type_name -> filer_pb.SubscribeMetadataResponse 1, // 15: filer_pb.CreateEntryResponse.error_code:type_name -> filer_pb.FilerError 7, // 16: filer_pb.UpdateEntryRequest.entry:type_name -> filer_pb.Entry - 72, // 17: filer_pb.UpdateEntryRequest.expected_extended:type_name -> filer_pb.UpdateEntryRequest.ExpectedExtendedEntry + 74, // 17: filer_pb.UpdateEntryRequest.expected_extended:type_name -> filer_pb.UpdateEntryRequest.ExpectedExtendedEntry 44, // 18: filer_pb.UpdateEntryResponse.metadata_event:type_name -> filer_pb.SubscribeMetadataResponse 10, // 19: filer_pb.AppendToEntryRequest.chunks:type_name -> filer_pb.FileChunk 44, // 20: filer_pb.DeleteEntryResponse.metadata_event:type_name -> filer_pb.SubscribeMetadataResponse 9, // 21: filer_pb.StreamRenameEntryResponse.event_notification:type_name -> filer_pb.EventNotification 30, // 22: filer_pb.AssignVolumeResponse.location:type_name -> filer_pb.Location 30, // 23: filer_pb.Locations.locations:type_name -> filer_pb.Location - 73, // 24: filer_pb.LookupVolumeResponse.locations_map:type_name -> filer_pb.LookupVolumeResponse.LocationsMapEntry + 75, // 24: filer_pb.LookupVolumeResponse.locations_map:type_name -> filer_pb.LookupVolumeResponse.LocationsMapEntry 32, // 25: filer_pb.CollectionListResponse.collections:type_name -> filer_pb.Collection 9, // 26: filer_pb.SubscribeMetadataResponse.event_notification:type_name -> filer_pb.EventNotification 44, // 27: filer_pb.SubscribeMetadataResponse.events:type_name -> filer_pb.SubscribeMetadataResponse 45, // 28: filer_pb.SubscribeMetadataResponse.log_file_refs:type_name -> filer_pb.LogFileChunkRef 10, // 29: filer_pb.LogFileChunkRef.chunks:type_name -> filer_pb.FileChunk 7, // 30: filer_pb.TraverseBfsMetadataResponse.entry:type_name -> filer_pb.Entry - 74, // 31: filer_pb.LocateBrokerResponse.resources:type_name -> filer_pb.LocateBrokerResponse.Resource - 75, // 32: filer_pb.FilerConf.locations:type_name -> filer_pb.FilerConf.PathConf + 76, // 31: filer_pb.LocateBrokerResponse.resources:type_name -> filer_pb.LocateBrokerResponse.Resource + 77, // 32: filer_pb.FilerConf.locations:type_name -> filer_pb.FilerConf.PathConf 7, // 33: filer_pb.CacheRemoteObjectToLocalClusterResponse.entry:type_name -> filer_pb.Entry 44, // 34: filer_pb.CacheRemoteObjectToLocalClusterResponse.metadata_event:type_name -> filer_pb.SubscribeMetadataResponse 66, // 35: filer_pb.TransferLocksRequest.locks:type_name -> filer_pb.Lock @@ -5499,7 +5682,7 @@ var file_filer_proto_depIdxs = []int32{ 20, // 50: filer_pb.SeaweedFiler.DeleteEntry:input_type -> filer_pb.DeleteEntryRequest 22, // 51: filer_pb.SeaweedFiler.AtomicRenameEntry:input_type -> filer_pb.AtomicRenameEntryRequest 24, // 52: filer_pb.SeaweedFiler.StreamRenameEntry:input_type -> filer_pb.StreamRenameEntryRequest - 69, // 53: filer_pb.SeaweedFiler.StreamMutateEntry:input_type -> filer_pb.StreamMutateEntryRequest + 71, // 53: filer_pb.SeaweedFiler.StreamMutateEntry:input_type -> filer_pb.StreamMutateEntryRequest 26, // 54: filer_pb.SeaweedFiler.AssignVolume:input_type -> filer_pb.AssignVolumeRequest 28, // 55: filer_pb.SeaweedFiler.LookupVolume:input_type -> filer_pb.LookupVolumeRequest 33, // 56: filer_pb.SeaweedFiler.CollectionList:input_type -> filer_pb.CollectionListRequest @@ -5517,34 +5700,36 @@ var file_filer_proto_depIdxs = []int32{ 62, // 68: filer_pb.SeaweedFiler.DistributedUnlock:input_type -> filer_pb.UnlockRequest 64, // 69: filer_pb.SeaweedFiler.FindLockOwner:input_type -> filer_pb.FindLockOwnerRequest 67, // 70: filer_pb.SeaweedFiler.TransferLocks:input_type -> filer_pb.TransferLocksRequest - 3, // 71: filer_pb.SeaweedFiler.LookupDirectoryEntry:output_type -> filer_pb.LookupDirectoryEntryResponse - 5, // 72: filer_pb.SeaweedFiler.ListEntries:output_type -> filer_pb.ListEntriesResponse - 15, // 73: filer_pb.SeaweedFiler.CreateEntry:output_type -> filer_pb.CreateEntryResponse - 17, // 74: filer_pb.SeaweedFiler.UpdateEntry:output_type -> filer_pb.UpdateEntryResponse - 19, // 75: filer_pb.SeaweedFiler.AppendToEntry:output_type -> filer_pb.AppendToEntryResponse - 21, // 76: filer_pb.SeaweedFiler.DeleteEntry:output_type -> filer_pb.DeleteEntryResponse - 23, // 77: filer_pb.SeaweedFiler.AtomicRenameEntry:output_type -> filer_pb.AtomicRenameEntryResponse - 25, // 78: filer_pb.SeaweedFiler.StreamRenameEntry:output_type -> filer_pb.StreamRenameEntryResponse - 70, // 79: filer_pb.SeaweedFiler.StreamMutateEntry:output_type -> filer_pb.StreamMutateEntryResponse - 27, // 80: filer_pb.SeaweedFiler.AssignVolume:output_type -> filer_pb.AssignVolumeResponse - 31, // 81: filer_pb.SeaweedFiler.LookupVolume:output_type -> filer_pb.LookupVolumeResponse - 34, // 82: filer_pb.SeaweedFiler.CollectionList:output_type -> filer_pb.CollectionListResponse - 36, // 83: filer_pb.SeaweedFiler.DeleteCollection:output_type -> filer_pb.DeleteCollectionResponse - 38, // 84: filer_pb.SeaweedFiler.Statistics:output_type -> filer_pb.StatisticsResponse - 40, // 85: filer_pb.SeaweedFiler.Ping:output_type -> filer_pb.PingResponse - 42, // 86: filer_pb.SeaweedFiler.GetFilerConfiguration:output_type -> filer_pb.GetFilerConfigurationResponse - 47, // 87: filer_pb.SeaweedFiler.TraverseBfsMetadata:output_type -> filer_pb.TraverseBfsMetadataResponse - 44, // 88: filer_pb.SeaweedFiler.SubscribeMetadata:output_type -> filer_pb.SubscribeMetadataResponse - 44, // 89: filer_pb.SeaweedFiler.SubscribeLocalMetadata:output_type -> filer_pb.SubscribeMetadataResponse - 54, // 90: filer_pb.SeaweedFiler.KvGet:output_type -> filer_pb.KvGetResponse - 56, // 91: filer_pb.SeaweedFiler.KvPut:output_type -> filer_pb.KvPutResponse - 59, // 92: filer_pb.SeaweedFiler.CacheRemoteObjectToLocalCluster:output_type -> filer_pb.CacheRemoteObjectToLocalClusterResponse - 61, // 93: filer_pb.SeaweedFiler.DistributedLock:output_type -> filer_pb.LockResponse - 63, // 94: filer_pb.SeaweedFiler.DistributedUnlock:output_type -> filer_pb.UnlockResponse - 65, // 95: filer_pb.SeaweedFiler.FindLockOwner:output_type -> filer_pb.FindLockOwnerResponse - 68, // 96: filer_pb.SeaweedFiler.TransferLocks:output_type -> filer_pb.TransferLocksResponse - 71, // [71:97] is the sub-list for method output_type - 45, // [45:71] is the sub-list for method input_type + 69, // 71: filer_pb.SeaweedFiler.ReplicateLock:input_type -> filer_pb.ReplicateLockRequest + 3, // 72: filer_pb.SeaweedFiler.LookupDirectoryEntry:output_type -> filer_pb.LookupDirectoryEntryResponse + 5, // 73: filer_pb.SeaweedFiler.ListEntries:output_type -> filer_pb.ListEntriesResponse + 15, // 74: filer_pb.SeaweedFiler.CreateEntry:output_type -> filer_pb.CreateEntryResponse + 17, // 75: filer_pb.SeaweedFiler.UpdateEntry:output_type -> filer_pb.UpdateEntryResponse + 19, // 76: filer_pb.SeaweedFiler.AppendToEntry:output_type -> filer_pb.AppendToEntryResponse + 21, // 77: filer_pb.SeaweedFiler.DeleteEntry:output_type -> filer_pb.DeleteEntryResponse + 23, // 78: filer_pb.SeaweedFiler.AtomicRenameEntry:output_type -> filer_pb.AtomicRenameEntryResponse + 25, // 79: filer_pb.SeaweedFiler.StreamRenameEntry:output_type -> filer_pb.StreamRenameEntryResponse + 72, // 80: filer_pb.SeaweedFiler.StreamMutateEntry:output_type -> filer_pb.StreamMutateEntryResponse + 27, // 81: filer_pb.SeaweedFiler.AssignVolume:output_type -> filer_pb.AssignVolumeResponse + 31, // 82: filer_pb.SeaweedFiler.LookupVolume:output_type -> filer_pb.LookupVolumeResponse + 34, // 83: filer_pb.SeaweedFiler.CollectionList:output_type -> filer_pb.CollectionListResponse + 36, // 84: filer_pb.SeaweedFiler.DeleteCollection:output_type -> filer_pb.DeleteCollectionResponse + 38, // 85: filer_pb.SeaweedFiler.Statistics:output_type -> filer_pb.StatisticsResponse + 40, // 86: filer_pb.SeaweedFiler.Ping:output_type -> filer_pb.PingResponse + 42, // 87: filer_pb.SeaweedFiler.GetFilerConfiguration:output_type -> filer_pb.GetFilerConfigurationResponse + 47, // 88: filer_pb.SeaweedFiler.TraverseBfsMetadata:output_type -> filer_pb.TraverseBfsMetadataResponse + 44, // 89: filer_pb.SeaweedFiler.SubscribeMetadata:output_type -> filer_pb.SubscribeMetadataResponse + 44, // 90: filer_pb.SeaweedFiler.SubscribeLocalMetadata:output_type -> filer_pb.SubscribeMetadataResponse + 54, // 91: filer_pb.SeaweedFiler.KvGet:output_type -> filer_pb.KvGetResponse + 56, // 92: filer_pb.SeaweedFiler.KvPut:output_type -> filer_pb.KvPutResponse + 59, // 93: filer_pb.SeaweedFiler.CacheRemoteObjectToLocalCluster:output_type -> filer_pb.CacheRemoteObjectToLocalClusterResponse + 61, // 94: filer_pb.SeaweedFiler.DistributedLock:output_type -> filer_pb.LockResponse + 63, // 95: filer_pb.SeaweedFiler.DistributedUnlock:output_type -> filer_pb.UnlockResponse + 65, // 96: filer_pb.SeaweedFiler.FindLockOwner:output_type -> filer_pb.FindLockOwnerResponse + 68, // 97: filer_pb.SeaweedFiler.TransferLocks:output_type -> filer_pb.TransferLocksResponse + 70, // 98: filer_pb.SeaweedFiler.ReplicateLock:output_type -> filer_pb.ReplicateLockResponse + 72, // [72:99] is the sub-list for method output_type + 45, // [45:72] is the sub-list for method input_type 45, // [45:45] is the sub-list for extension type_name 45, // [45:45] is the sub-list for extension extendee 0, // [0:45] is the sub-list for field type_name @@ -5555,13 +5740,13 @@ func file_filer_proto_init() { if File_filer_proto != nil { return } - file_filer_proto_msgTypes[67].OneofWrappers = []any{ + file_filer_proto_msgTypes[69].OneofWrappers = []any{ (*StreamMutateEntryRequest_CreateRequest)(nil), (*StreamMutateEntryRequest_UpdateRequest)(nil), (*StreamMutateEntryRequest_DeleteRequest)(nil), (*StreamMutateEntryRequest_RenameRequest)(nil), } - file_filer_proto_msgTypes[68].OneofWrappers = []any{ + file_filer_proto_msgTypes[70].OneofWrappers = []any{ (*StreamMutateEntryResponse_CreateResponse)(nil), (*StreamMutateEntryResponse_UpdateResponse)(nil), (*StreamMutateEntryResponse_DeleteResponse)(nil), @@ -5573,7 +5758,7 @@ func file_filer_proto_init() { GoPackagePath: reflect.TypeOf(x{}).PkgPath(), RawDescriptor: unsafe.Slice(unsafe.StringData(file_filer_proto_rawDesc), len(file_filer_proto_rawDesc)), NumEnums: 2, - NumMessages: 74, + NumMessages: 76, NumExtensions: 0, NumServices: 1, }, diff --git a/weed/pb/filer_pb/filer_grpc.pb.go b/weed/pb/filer_pb/filer_grpc.pb.go index 858070462..e7498280d 100644 --- a/weed/pb/filer_pb/filer_grpc.pb.go +++ b/weed/pb/filer_pb/filer_grpc.pb.go @@ -45,6 +45,7 @@ const ( SeaweedFiler_DistributedUnlock_FullMethodName = "/filer_pb.SeaweedFiler/DistributedUnlock" SeaweedFiler_FindLockOwner_FullMethodName = "/filer_pb.SeaweedFiler/FindLockOwner" SeaweedFiler_TransferLocks_FullMethodName = "/filer_pb.SeaweedFiler/TransferLocks" + SeaweedFiler_ReplicateLock_FullMethodName = "/filer_pb.SeaweedFiler/ReplicateLock" ) // SeaweedFilerClient is the client API for SeaweedFiler service. @@ -78,6 +79,7 @@ type SeaweedFilerClient interface { FindLockOwner(ctx context.Context, in *FindLockOwnerRequest, opts ...grpc.CallOption) (*FindLockOwnerResponse, error) // distributed lock management internal use only TransferLocks(ctx context.Context, in *TransferLocksRequest, opts ...grpc.CallOption) (*TransferLocksResponse, error) + ReplicateLock(ctx context.Context, in *ReplicateLockRequest, opts ...grpc.CallOption) (*ReplicateLockResponse, error) } type seaweedFilerClient struct { @@ -396,6 +398,16 @@ func (c *seaweedFilerClient) TransferLocks(ctx context.Context, in *TransferLock return out, nil } +func (c *seaweedFilerClient) ReplicateLock(ctx context.Context, in *ReplicateLockRequest, opts ...grpc.CallOption) (*ReplicateLockResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(ReplicateLockResponse) + err := c.cc.Invoke(ctx, SeaweedFiler_ReplicateLock_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + // SeaweedFilerServer is the server API for SeaweedFiler service. // All implementations must embed UnimplementedSeaweedFilerServer // for forward compatibility. @@ -427,6 +439,7 @@ type SeaweedFilerServer interface { FindLockOwner(context.Context, *FindLockOwnerRequest) (*FindLockOwnerResponse, error) // distributed lock management internal use only TransferLocks(context.Context, *TransferLocksRequest) (*TransferLocksResponse, error) + ReplicateLock(context.Context, *ReplicateLockRequest) (*ReplicateLockResponse, error) mustEmbedUnimplementedSeaweedFilerServer() } @@ -515,6 +528,9 @@ func (UnimplementedSeaweedFilerServer) FindLockOwner(context.Context, *FindLockO func (UnimplementedSeaweedFilerServer) TransferLocks(context.Context, *TransferLocksRequest) (*TransferLocksResponse, error) { return nil, status.Errorf(codes.Unimplemented, "method TransferLocks not implemented") } +func (UnimplementedSeaweedFilerServer) ReplicateLock(context.Context, *ReplicateLockRequest) (*ReplicateLockResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method ReplicateLock not implemented") +} func (UnimplementedSeaweedFilerServer) mustEmbedUnimplementedSeaweedFilerServer() {} func (UnimplementedSeaweedFilerServer) testEmbeddedByValue() {} @@ -958,6 +974,24 @@ func _SeaweedFiler_TransferLocks_Handler(srv interface{}, ctx context.Context, d return interceptor(ctx, in, info, handler) } +func _SeaweedFiler_ReplicateLock_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(ReplicateLockRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(SeaweedFilerServer).ReplicateLock(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: SeaweedFiler_ReplicateLock_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(SeaweedFilerServer).ReplicateLock(ctx, req.(*ReplicateLockRequest)) + } + return interceptor(ctx, in, info, handler) +} + // SeaweedFiler_ServiceDesc is the grpc.ServiceDesc for SeaweedFiler service. // It's only intended for direct use with grpc.RegisterService, // and not to be introspected or modified (even as a copy) @@ -1045,6 +1079,10 @@ var SeaweedFiler_ServiceDesc = grpc.ServiceDesc{ MethodName: "TransferLocks", Handler: _SeaweedFiler_TransferLocks_Handler, }, + { + MethodName: "ReplicateLock", + Handler: _SeaweedFiler_ReplicateLock_Handler, + }, }, Streams: []grpc.StreamDesc{ { diff --git a/weed/pb/master.proto b/weed/pb/master.proto index a11a31e99..0a60847e1 100644 --- a/weed/pb/master.proto +++ b/weed/pb/master.proto @@ -197,6 +197,17 @@ message ClusterNodeUpdate { message KeepConnectedResponse { VolumeLocation volume_location = 1; ClusterNodeUpdate cluster_node_update = 2; + LockRingUpdate lock_ring_update = 3; +} + +// LockRingUpdate is sent by the master to all filers when the lock ring +// membership changes. The master batches rapid changes (e.g., node drop + join) +// and sends the complete member list atomically, avoiding intermediate ring +// states that would cause unnecessary lock churn. +message LockRingUpdate { + string filer_group = 1; + repeated string servers = 2; + int64 version = 3; } message LookupVolumeRequest { diff --git a/weed/pb/master_pb/master.pb.go b/weed/pb/master_pb/master.pb.go index 2ffb4d306..7e161d3fb 100644 --- a/weed/pb/master_pb/master.pb.go +++ b/weed/pb/master_pb/master.pb.go @@ -1134,6 +1134,7 @@ type KeepConnectedResponse struct { state protoimpl.MessageState `protogen:"open.v1"` VolumeLocation *VolumeLocation `protobuf:"bytes,1,opt,name=volume_location,json=volumeLocation,proto3" json:"volume_location,omitempty"` ClusterNodeUpdate *ClusterNodeUpdate `protobuf:"bytes,2,opt,name=cluster_node_update,json=clusterNodeUpdate,proto3" json:"cluster_node_update,omitempty"` + LockRingUpdate *LockRingUpdate `protobuf:"bytes,3,opt,name=lock_ring_update,json=lockRingUpdate,proto3" json:"lock_ring_update,omitempty"` unknownFields protoimpl.UnknownFields sizeCache protoimpl.SizeCache } @@ -1182,6 +1183,77 @@ func (x *KeepConnectedResponse) GetClusterNodeUpdate() *ClusterNodeUpdate { return nil } +func (x *KeepConnectedResponse) GetLockRingUpdate() *LockRingUpdate { + if x != nil { + return x.LockRingUpdate + } + return nil +} + +// LockRingUpdate is sent by the master to all filers when the lock ring +// membership changes. The master batches rapid changes (e.g., node drop + join) +// and sends the complete member list atomically, avoiding intermediate ring +// states that would cause unnecessary lock churn. +type LockRingUpdate struct { + state protoimpl.MessageState `protogen:"open.v1"` + FilerGroup string `protobuf:"bytes,1,opt,name=filer_group,json=filerGroup,proto3" json:"filer_group,omitempty"` + Servers []string `protobuf:"bytes,2,rep,name=servers,proto3" json:"servers,omitempty"` + Version int64 `protobuf:"varint,3,opt,name=version,proto3" json:"version,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *LockRingUpdate) Reset() { + *x = LockRingUpdate{} + mi := &file_master_proto_msgTypes[13] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *LockRingUpdate) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*LockRingUpdate) ProtoMessage() {} + +func (x *LockRingUpdate) ProtoReflect() protoreflect.Message { + mi := &file_master_proto_msgTypes[13] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use LockRingUpdate.ProtoReflect.Descriptor instead. +func (*LockRingUpdate) Descriptor() ([]byte, []int) { + return file_master_proto_rawDescGZIP(), []int{13} +} + +func (x *LockRingUpdate) GetFilerGroup() string { + if x != nil { + return x.FilerGroup + } + return "" +} + +func (x *LockRingUpdate) GetServers() []string { + if x != nil { + return x.Servers + } + return nil +} + +func (x *LockRingUpdate) GetVersion() int64 { + if x != nil { + return x.Version + } + return 0 +} + type LookupVolumeRequest struct { state protoimpl.MessageState `protogen:"open.v1"` VolumeOrFileIds []string `protobuf:"bytes,1,rep,name=volume_or_file_ids,json=volumeOrFileIds,proto3" json:"volume_or_file_ids,omitempty"` @@ -1192,7 +1264,7 @@ type LookupVolumeRequest struct { func (x *LookupVolumeRequest) Reset() { *x = LookupVolumeRequest{} - mi := &file_master_proto_msgTypes[13] + mi := &file_master_proto_msgTypes[14] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1204,7 +1276,7 @@ func (x *LookupVolumeRequest) String() string { func (*LookupVolumeRequest) ProtoMessage() {} func (x *LookupVolumeRequest) ProtoReflect() protoreflect.Message { - mi := &file_master_proto_msgTypes[13] + mi := &file_master_proto_msgTypes[14] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1217,7 +1289,7 @@ func (x *LookupVolumeRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use LookupVolumeRequest.ProtoReflect.Descriptor instead. func (*LookupVolumeRequest) Descriptor() ([]byte, []int) { - return file_master_proto_rawDescGZIP(), []int{13} + return file_master_proto_rawDescGZIP(), []int{14} } func (x *LookupVolumeRequest) GetVolumeOrFileIds() []string { @@ -1243,7 +1315,7 @@ type LookupVolumeResponse struct { func (x *LookupVolumeResponse) Reset() { *x = LookupVolumeResponse{} - mi := &file_master_proto_msgTypes[14] + mi := &file_master_proto_msgTypes[15] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1255,7 +1327,7 @@ func (x *LookupVolumeResponse) String() string { func (*LookupVolumeResponse) ProtoMessage() {} func (x *LookupVolumeResponse) ProtoReflect() protoreflect.Message { - mi := &file_master_proto_msgTypes[14] + mi := &file_master_proto_msgTypes[15] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1268,7 +1340,7 @@ func (x *LookupVolumeResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use LookupVolumeResponse.ProtoReflect.Descriptor instead. func (*LookupVolumeResponse) Descriptor() ([]byte, []int) { - return file_master_proto_rawDescGZIP(), []int{14} + return file_master_proto_rawDescGZIP(), []int{15} } func (x *LookupVolumeResponse) GetVolumeIdLocations() []*LookupVolumeResponse_VolumeIdLocation { @@ -1290,7 +1362,7 @@ type Location struct { func (x *Location) Reset() { *x = Location{} - mi := &file_master_proto_msgTypes[15] + mi := &file_master_proto_msgTypes[16] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1302,7 +1374,7 @@ func (x *Location) String() string { func (*Location) ProtoMessage() {} func (x *Location) ProtoReflect() protoreflect.Message { - mi := &file_master_proto_msgTypes[15] + mi := &file_master_proto_msgTypes[16] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1315,7 +1387,7 @@ func (x *Location) ProtoReflect() protoreflect.Message { // Deprecated: Use Location.ProtoReflect.Descriptor instead. func (*Location) Descriptor() ([]byte, []int) { - return file_master_proto_rawDescGZIP(), []int{15} + return file_master_proto_rawDescGZIP(), []int{16} } func (x *Location) GetUrl() string { @@ -1364,7 +1436,7 @@ type AssignRequest struct { func (x *AssignRequest) Reset() { *x = AssignRequest{} - mi := &file_master_proto_msgTypes[16] + mi := &file_master_proto_msgTypes[17] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1376,7 +1448,7 @@ func (x *AssignRequest) String() string { func (*AssignRequest) ProtoMessage() {} func (x *AssignRequest) ProtoReflect() protoreflect.Message { - mi := &file_master_proto_msgTypes[16] + mi := &file_master_proto_msgTypes[17] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1389,7 +1461,7 @@ func (x *AssignRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use AssignRequest.ProtoReflect.Descriptor instead. func (*AssignRequest) Descriptor() ([]byte, []int) { - return file_master_proto_rawDescGZIP(), []int{16} + return file_master_proto_rawDescGZIP(), []int{17} } func (x *AssignRequest) GetCount() uint64 { @@ -1479,7 +1551,7 @@ type VolumeGrowRequest struct { func (x *VolumeGrowRequest) Reset() { *x = VolumeGrowRequest{} - mi := &file_master_proto_msgTypes[17] + mi := &file_master_proto_msgTypes[18] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1491,7 +1563,7 @@ func (x *VolumeGrowRequest) String() string { func (*VolumeGrowRequest) ProtoMessage() {} func (x *VolumeGrowRequest) ProtoReflect() protoreflect.Message { - mi := &file_master_proto_msgTypes[17] + mi := &file_master_proto_msgTypes[18] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1504,7 +1576,7 @@ func (x *VolumeGrowRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use VolumeGrowRequest.ProtoReflect.Descriptor instead. func (*VolumeGrowRequest) Descriptor() ([]byte, []int) { - return file_master_proto_rawDescGZIP(), []int{17} + return file_master_proto_rawDescGZIP(), []int{18} } func (x *VolumeGrowRequest) GetWritableVolumeCount() uint32 { @@ -1584,7 +1656,7 @@ type AssignResponse struct { func (x *AssignResponse) Reset() { *x = AssignResponse{} - mi := &file_master_proto_msgTypes[18] + mi := &file_master_proto_msgTypes[19] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1596,7 +1668,7 @@ func (x *AssignResponse) String() string { func (*AssignResponse) ProtoMessage() {} func (x *AssignResponse) ProtoReflect() protoreflect.Message { - mi := &file_master_proto_msgTypes[18] + mi := &file_master_proto_msgTypes[19] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1609,7 +1681,7 @@ func (x *AssignResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use AssignResponse.ProtoReflect.Descriptor instead. func (*AssignResponse) Descriptor() ([]byte, []int) { - return file_master_proto_rawDescGZIP(), []int{18} + return file_master_proto_rawDescGZIP(), []int{19} } func (x *AssignResponse) GetFid() string { @@ -1666,7 +1738,7 @@ type StatisticsRequest struct { func (x *StatisticsRequest) Reset() { *x = StatisticsRequest{} - mi := &file_master_proto_msgTypes[19] + mi := &file_master_proto_msgTypes[20] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1678,7 +1750,7 @@ func (x *StatisticsRequest) String() string { func (*StatisticsRequest) ProtoMessage() {} func (x *StatisticsRequest) ProtoReflect() protoreflect.Message { - mi := &file_master_proto_msgTypes[19] + mi := &file_master_proto_msgTypes[20] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1691,7 +1763,7 @@ func (x *StatisticsRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use StatisticsRequest.ProtoReflect.Descriptor instead. func (*StatisticsRequest) Descriptor() ([]byte, []int) { - return file_master_proto_rawDescGZIP(), []int{19} + return file_master_proto_rawDescGZIP(), []int{20} } func (x *StatisticsRequest) GetReplication() string { @@ -1733,7 +1805,7 @@ type StatisticsResponse struct { func (x *StatisticsResponse) Reset() { *x = StatisticsResponse{} - mi := &file_master_proto_msgTypes[20] + mi := &file_master_proto_msgTypes[21] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1745,7 +1817,7 @@ func (x *StatisticsResponse) String() string { func (*StatisticsResponse) ProtoMessage() {} func (x *StatisticsResponse) ProtoReflect() protoreflect.Message { - mi := &file_master_proto_msgTypes[20] + mi := &file_master_proto_msgTypes[21] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1758,7 +1830,7 @@ func (x *StatisticsResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use StatisticsResponse.ProtoReflect.Descriptor instead. func (*StatisticsResponse) Descriptor() ([]byte, []int) { - return file_master_proto_rawDescGZIP(), []int{20} + return file_master_proto_rawDescGZIP(), []int{21} } func (x *StatisticsResponse) GetTotalSize() uint64 { @@ -1792,7 +1864,7 @@ type Collection struct { func (x *Collection) Reset() { *x = Collection{} - mi := &file_master_proto_msgTypes[21] + mi := &file_master_proto_msgTypes[22] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1804,7 +1876,7 @@ func (x *Collection) String() string { func (*Collection) ProtoMessage() {} func (x *Collection) ProtoReflect() protoreflect.Message { - mi := &file_master_proto_msgTypes[21] + mi := &file_master_proto_msgTypes[22] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1817,7 +1889,7 @@ func (x *Collection) ProtoReflect() protoreflect.Message { // Deprecated: Use Collection.ProtoReflect.Descriptor instead. func (*Collection) Descriptor() ([]byte, []int) { - return file_master_proto_rawDescGZIP(), []int{21} + return file_master_proto_rawDescGZIP(), []int{22} } func (x *Collection) GetName() string { @@ -1837,7 +1909,7 @@ type CollectionListRequest struct { func (x *CollectionListRequest) Reset() { *x = CollectionListRequest{} - mi := &file_master_proto_msgTypes[22] + mi := &file_master_proto_msgTypes[23] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1849,7 +1921,7 @@ func (x *CollectionListRequest) String() string { func (*CollectionListRequest) ProtoMessage() {} func (x *CollectionListRequest) ProtoReflect() protoreflect.Message { - mi := &file_master_proto_msgTypes[22] + mi := &file_master_proto_msgTypes[23] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1862,7 +1934,7 @@ func (x *CollectionListRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use CollectionListRequest.ProtoReflect.Descriptor instead. func (*CollectionListRequest) Descriptor() ([]byte, []int) { - return file_master_proto_rawDescGZIP(), []int{22} + return file_master_proto_rawDescGZIP(), []int{23} } func (x *CollectionListRequest) GetIncludeNormalVolumes() bool { @@ -1888,7 +1960,7 @@ type CollectionListResponse struct { func (x *CollectionListResponse) Reset() { *x = CollectionListResponse{} - mi := &file_master_proto_msgTypes[23] + mi := &file_master_proto_msgTypes[24] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1900,7 +1972,7 @@ func (x *CollectionListResponse) String() string { func (*CollectionListResponse) ProtoMessage() {} func (x *CollectionListResponse) ProtoReflect() protoreflect.Message { - mi := &file_master_proto_msgTypes[23] + mi := &file_master_proto_msgTypes[24] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1913,7 +1985,7 @@ func (x *CollectionListResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use CollectionListResponse.ProtoReflect.Descriptor instead. func (*CollectionListResponse) Descriptor() ([]byte, []int) { - return file_master_proto_rawDescGZIP(), []int{23} + return file_master_proto_rawDescGZIP(), []int{24} } func (x *CollectionListResponse) GetCollections() []*Collection { @@ -1932,7 +2004,7 @@ type CollectionDeleteRequest struct { func (x *CollectionDeleteRequest) Reset() { *x = CollectionDeleteRequest{} - mi := &file_master_proto_msgTypes[24] + mi := &file_master_proto_msgTypes[25] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1944,7 +2016,7 @@ func (x *CollectionDeleteRequest) String() string { func (*CollectionDeleteRequest) ProtoMessage() {} func (x *CollectionDeleteRequest) ProtoReflect() protoreflect.Message { - mi := &file_master_proto_msgTypes[24] + mi := &file_master_proto_msgTypes[25] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1957,7 +2029,7 @@ func (x *CollectionDeleteRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use CollectionDeleteRequest.ProtoReflect.Descriptor instead. func (*CollectionDeleteRequest) Descriptor() ([]byte, []int) { - return file_master_proto_rawDescGZIP(), []int{24} + return file_master_proto_rawDescGZIP(), []int{25} } func (x *CollectionDeleteRequest) GetName() string { @@ -1975,7 +2047,7 @@ type CollectionDeleteResponse struct { func (x *CollectionDeleteResponse) Reset() { *x = CollectionDeleteResponse{} - mi := &file_master_proto_msgTypes[25] + mi := &file_master_proto_msgTypes[26] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1987,7 +2059,7 @@ func (x *CollectionDeleteResponse) String() string { func (*CollectionDeleteResponse) ProtoMessage() {} func (x *CollectionDeleteResponse) ProtoReflect() protoreflect.Message { - mi := &file_master_proto_msgTypes[25] + mi := &file_master_proto_msgTypes[26] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2000,7 +2072,7 @@ func (x *CollectionDeleteResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use CollectionDeleteResponse.ProtoReflect.Descriptor instead. func (*CollectionDeleteResponse) Descriptor() ([]byte, []int) { - return file_master_proto_rawDescGZIP(), []int{25} + return file_master_proto_rawDescGZIP(), []int{26} } // volume related @@ -2022,7 +2094,7 @@ type DiskInfo struct { func (x *DiskInfo) Reset() { *x = DiskInfo{} - mi := &file_master_proto_msgTypes[26] + mi := &file_master_proto_msgTypes[27] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2034,7 +2106,7 @@ func (x *DiskInfo) String() string { func (*DiskInfo) ProtoMessage() {} func (x *DiskInfo) ProtoReflect() protoreflect.Message { - mi := &file_master_proto_msgTypes[26] + mi := &file_master_proto_msgTypes[27] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2047,7 +2119,7 @@ func (x *DiskInfo) ProtoReflect() protoreflect.Message { // Deprecated: Use DiskInfo.ProtoReflect.Descriptor instead. func (*DiskInfo) Descriptor() ([]byte, []int) { - return file_master_proto_rawDescGZIP(), []int{26} + return file_master_proto_rawDescGZIP(), []int{27} } func (x *DiskInfo) GetType() string { @@ -2132,7 +2204,7 @@ type DataNodeInfo struct { func (x *DataNodeInfo) Reset() { *x = DataNodeInfo{} - mi := &file_master_proto_msgTypes[27] + mi := &file_master_proto_msgTypes[28] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2144,7 +2216,7 @@ func (x *DataNodeInfo) String() string { func (*DataNodeInfo) ProtoMessage() {} func (x *DataNodeInfo) ProtoReflect() protoreflect.Message { - mi := &file_master_proto_msgTypes[27] + mi := &file_master_proto_msgTypes[28] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2157,7 +2229,7 @@ func (x *DataNodeInfo) ProtoReflect() protoreflect.Message { // Deprecated: Use DataNodeInfo.ProtoReflect.Descriptor instead. func (*DataNodeInfo) Descriptor() ([]byte, []int) { - return file_master_proto_rawDescGZIP(), []int{27} + return file_master_proto_rawDescGZIP(), []int{28} } func (x *DataNodeInfo) GetId() string { @@ -2199,7 +2271,7 @@ type RackInfo struct { func (x *RackInfo) Reset() { *x = RackInfo{} - mi := &file_master_proto_msgTypes[28] + mi := &file_master_proto_msgTypes[29] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2211,7 +2283,7 @@ func (x *RackInfo) String() string { func (*RackInfo) ProtoMessage() {} func (x *RackInfo) ProtoReflect() protoreflect.Message { - mi := &file_master_proto_msgTypes[28] + mi := &file_master_proto_msgTypes[29] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2224,7 +2296,7 @@ func (x *RackInfo) ProtoReflect() protoreflect.Message { // Deprecated: Use RackInfo.ProtoReflect.Descriptor instead. func (*RackInfo) Descriptor() ([]byte, []int) { - return file_master_proto_rawDescGZIP(), []int{28} + return file_master_proto_rawDescGZIP(), []int{29} } func (x *RackInfo) GetId() string { @@ -2259,7 +2331,7 @@ type DataCenterInfo struct { func (x *DataCenterInfo) Reset() { *x = DataCenterInfo{} - mi := &file_master_proto_msgTypes[29] + mi := &file_master_proto_msgTypes[30] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2271,7 +2343,7 @@ func (x *DataCenterInfo) String() string { func (*DataCenterInfo) ProtoMessage() {} func (x *DataCenterInfo) ProtoReflect() protoreflect.Message { - mi := &file_master_proto_msgTypes[29] + mi := &file_master_proto_msgTypes[30] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2284,7 +2356,7 @@ func (x *DataCenterInfo) ProtoReflect() protoreflect.Message { // Deprecated: Use DataCenterInfo.ProtoReflect.Descriptor instead. func (*DataCenterInfo) Descriptor() ([]byte, []int) { - return file_master_proto_rawDescGZIP(), []int{29} + return file_master_proto_rawDescGZIP(), []int{30} } func (x *DataCenterInfo) GetId() string { @@ -2319,7 +2391,7 @@ type TopologyInfo struct { func (x *TopologyInfo) Reset() { *x = TopologyInfo{} - mi := &file_master_proto_msgTypes[30] + mi := &file_master_proto_msgTypes[31] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2331,7 +2403,7 @@ func (x *TopologyInfo) String() string { func (*TopologyInfo) ProtoMessage() {} func (x *TopologyInfo) ProtoReflect() protoreflect.Message { - mi := &file_master_proto_msgTypes[30] + mi := &file_master_proto_msgTypes[31] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2344,7 +2416,7 @@ func (x *TopologyInfo) ProtoReflect() protoreflect.Message { // Deprecated: Use TopologyInfo.ProtoReflect.Descriptor instead. func (*TopologyInfo) Descriptor() ([]byte, []int) { - return file_master_proto_rawDescGZIP(), []int{30} + return file_master_proto_rawDescGZIP(), []int{31} } func (x *TopologyInfo) GetId() string { @@ -2376,7 +2448,7 @@ type VolumeListRequest struct { func (x *VolumeListRequest) Reset() { *x = VolumeListRequest{} - mi := &file_master_proto_msgTypes[31] + mi := &file_master_proto_msgTypes[32] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2388,7 +2460,7 @@ func (x *VolumeListRequest) String() string { func (*VolumeListRequest) ProtoMessage() {} func (x *VolumeListRequest) ProtoReflect() protoreflect.Message { - mi := &file_master_proto_msgTypes[31] + mi := &file_master_proto_msgTypes[32] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2401,7 +2473,7 @@ func (x *VolumeListRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use VolumeListRequest.ProtoReflect.Descriptor instead. func (*VolumeListRequest) Descriptor() ([]byte, []int) { - return file_master_proto_rawDescGZIP(), []int{31} + return file_master_proto_rawDescGZIP(), []int{32} } type VolumeListResponse struct { @@ -2414,7 +2486,7 @@ type VolumeListResponse struct { func (x *VolumeListResponse) Reset() { *x = VolumeListResponse{} - mi := &file_master_proto_msgTypes[32] + mi := &file_master_proto_msgTypes[33] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2426,7 +2498,7 @@ func (x *VolumeListResponse) String() string { func (*VolumeListResponse) ProtoMessage() {} func (x *VolumeListResponse) ProtoReflect() protoreflect.Message { - mi := &file_master_proto_msgTypes[32] + mi := &file_master_proto_msgTypes[33] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2439,7 +2511,7 @@ func (x *VolumeListResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use VolumeListResponse.ProtoReflect.Descriptor instead. func (*VolumeListResponse) Descriptor() ([]byte, []int) { - return file_master_proto_rawDescGZIP(), []int{32} + return file_master_proto_rawDescGZIP(), []int{33} } func (x *VolumeListResponse) GetTopologyInfo() *TopologyInfo { @@ -2465,7 +2537,7 @@ type LookupEcVolumeRequest struct { func (x *LookupEcVolumeRequest) Reset() { *x = LookupEcVolumeRequest{} - mi := &file_master_proto_msgTypes[33] + mi := &file_master_proto_msgTypes[34] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2477,7 +2549,7 @@ func (x *LookupEcVolumeRequest) String() string { func (*LookupEcVolumeRequest) ProtoMessage() {} func (x *LookupEcVolumeRequest) ProtoReflect() protoreflect.Message { - mi := &file_master_proto_msgTypes[33] + mi := &file_master_proto_msgTypes[34] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2490,7 +2562,7 @@ func (x *LookupEcVolumeRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use LookupEcVolumeRequest.ProtoReflect.Descriptor instead. func (*LookupEcVolumeRequest) Descriptor() ([]byte, []int) { - return file_master_proto_rawDescGZIP(), []int{33} + return file_master_proto_rawDescGZIP(), []int{34} } func (x *LookupEcVolumeRequest) GetVolumeId() uint32 { @@ -2510,7 +2582,7 @@ type LookupEcVolumeResponse struct { func (x *LookupEcVolumeResponse) Reset() { *x = LookupEcVolumeResponse{} - mi := &file_master_proto_msgTypes[34] + mi := &file_master_proto_msgTypes[35] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2522,7 +2594,7 @@ func (x *LookupEcVolumeResponse) String() string { func (*LookupEcVolumeResponse) ProtoMessage() {} func (x *LookupEcVolumeResponse) ProtoReflect() protoreflect.Message { - mi := &file_master_proto_msgTypes[34] + mi := &file_master_proto_msgTypes[35] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2535,7 +2607,7 @@ func (x *LookupEcVolumeResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use LookupEcVolumeResponse.ProtoReflect.Descriptor instead. func (*LookupEcVolumeResponse) Descriptor() ([]byte, []int) { - return file_master_proto_rawDescGZIP(), []int{34} + return file_master_proto_rawDescGZIP(), []int{35} } func (x *LookupEcVolumeResponse) GetVolumeId() uint32 { @@ -2563,7 +2635,7 @@ type VacuumVolumeRequest struct { func (x *VacuumVolumeRequest) Reset() { *x = VacuumVolumeRequest{} - mi := &file_master_proto_msgTypes[35] + mi := &file_master_proto_msgTypes[36] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2575,7 +2647,7 @@ func (x *VacuumVolumeRequest) String() string { func (*VacuumVolumeRequest) ProtoMessage() {} func (x *VacuumVolumeRequest) ProtoReflect() protoreflect.Message { - mi := &file_master_proto_msgTypes[35] + mi := &file_master_proto_msgTypes[36] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2588,7 +2660,7 @@ func (x *VacuumVolumeRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use VacuumVolumeRequest.ProtoReflect.Descriptor instead. func (*VacuumVolumeRequest) Descriptor() ([]byte, []int) { - return file_master_proto_rawDescGZIP(), []int{35} + return file_master_proto_rawDescGZIP(), []int{36} } func (x *VacuumVolumeRequest) GetGarbageThreshold() float32 { @@ -2620,7 +2692,7 @@ type VacuumVolumeResponse struct { func (x *VacuumVolumeResponse) Reset() { *x = VacuumVolumeResponse{} - mi := &file_master_proto_msgTypes[36] + mi := &file_master_proto_msgTypes[37] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2632,7 +2704,7 @@ func (x *VacuumVolumeResponse) String() string { func (*VacuumVolumeResponse) ProtoMessage() {} func (x *VacuumVolumeResponse) ProtoReflect() protoreflect.Message { - mi := &file_master_proto_msgTypes[36] + mi := &file_master_proto_msgTypes[37] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2645,7 +2717,7 @@ func (x *VacuumVolumeResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use VacuumVolumeResponse.ProtoReflect.Descriptor instead. func (*VacuumVolumeResponse) Descriptor() ([]byte, []int) { - return file_master_proto_rawDescGZIP(), []int{36} + return file_master_proto_rawDescGZIP(), []int{37} } type DisableVacuumRequest struct { @@ -2657,7 +2729,7 @@ type DisableVacuumRequest struct { func (x *DisableVacuumRequest) Reset() { *x = DisableVacuumRequest{} - mi := &file_master_proto_msgTypes[37] + mi := &file_master_proto_msgTypes[38] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2669,7 +2741,7 @@ func (x *DisableVacuumRequest) String() string { func (*DisableVacuumRequest) ProtoMessage() {} func (x *DisableVacuumRequest) ProtoReflect() protoreflect.Message { - mi := &file_master_proto_msgTypes[37] + mi := &file_master_proto_msgTypes[38] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2682,7 +2754,7 @@ func (x *DisableVacuumRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use DisableVacuumRequest.ProtoReflect.Descriptor instead. func (*DisableVacuumRequest) Descriptor() ([]byte, []int) { - return file_master_proto_rawDescGZIP(), []int{37} + return file_master_proto_rawDescGZIP(), []int{38} } func (x *DisableVacuumRequest) GetByPlugin() bool { @@ -2700,7 +2772,7 @@ type DisableVacuumResponse struct { func (x *DisableVacuumResponse) Reset() { *x = DisableVacuumResponse{} - mi := &file_master_proto_msgTypes[38] + mi := &file_master_proto_msgTypes[39] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2712,7 +2784,7 @@ func (x *DisableVacuumResponse) String() string { func (*DisableVacuumResponse) ProtoMessage() {} func (x *DisableVacuumResponse) ProtoReflect() protoreflect.Message { - mi := &file_master_proto_msgTypes[38] + mi := &file_master_proto_msgTypes[39] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2725,7 +2797,7 @@ func (x *DisableVacuumResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use DisableVacuumResponse.ProtoReflect.Descriptor instead. func (*DisableVacuumResponse) Descriptor() ([]byte, []int) { - return file_master_proto_rawDescGZIP(), []int{38} + return file_master_proto_rawDescGZIP(), []int{39} } type EnableVacuumRequest struct { @@ -2737,7 +2809,7 @@ type EnableVacuumRequest struct { func (x *EnableVacuumRequest) Reset() { *x = EnableVacuumRequest{} - mi := &file_master_proto_msgTypes[39] + mi := &file_master_proto_msgTypes[40] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2749,7 +2821,7 @@ func (x *EnableVacuumRequest) String() string { func (*EnableVacuumRequest) ProtoMessage() {} func (x *EnableVacuumRequest) ProtoReflect() protoreflect.Message { - mi := &file_master_proto_msgTypes[39] + mi := &file_master_proto_msgTypes[40] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2762,7 +2834,7 @@ func (x *EnableVacuumRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use EnableVacuumRequest.ProtoReflect.Descriptor instead. func (*EnableVacuumRequest) Descriptor() ([]byte, []int) { - return file_master_proto_rawDescGZIP(), []int{39} + return file_master_proto_rawDescGZIP(), []int{40} } func (x *EnableVacuumRequest) GetByPlugin() bool { @@ -2780,7 +2852,7 @@ type EnableVacuumResponse struct { func (x *EnableVacuumResponse) Reset() { *x = EnableVacuumResponse{} - mi := &file_master_proto_msgTypes[40] + mi := &file_master_proto_msgTypes[41] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2792,7 +2864,7 @@ func (x *EnableVacuumResponse) String() string { func (*EnableVacuumResponse) ProtoMessage() {} func (x *EnableVacuumResponse) ProtoReflect() protoreflect.Message { - mi := &file_master_proto_msgTypes[40] + mi := &file_master_proto_msgTypes[41] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2805,7 +2877,7 @@ func (x *EnableVacuumResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use EnableVacuumResponse.ProtoReflect.Descriptor instead. func (*EnableVacuumResponse) Descriptor() ([]byte, []int) { - return file_master_proto_rawDescGZIP(), []int{40} + return file_master_proto_rawDescGZIP(), []int{41} } type VolumeMarkReadonlyRequest struct { @@ -2825,7 +2897,7 @@ type VolumeMarkReadonlyRequest struct { func (x *VolumeMarkReadonlyRequest) Reset() { *x = VolumeMarkReadonlyRequest{} - mi := &file_master_proto_msgTypes[41] + mi := &file_master_proto_msgTypes[42] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2837,7 +2909,7 @@ func (x *VolumeMarkReadonlyRequest) String() string { func (*VolumeMarkReadonlyRequest) ProtoMessage() {} func (x *VolumeMarkReadonlyRequest) ProtoReflect() protoreflect.Message { - mi := &file_master_proto_msgTypes[41] + mi := &file_master_proto_msgTypes[42] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2850,7 +2922,7 @@ func (x *VolumeMarkReadonlyRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use VolumeMarkReadonlyRequest.ProtoReflect.Descriptor instead. func (*VolumeMarkReadonlyRequest) Descriptor() ([]byte, []int) { - return file_master_proto_rawDescGZIP(), []int{41} + return file_master_proto_rawDescGZIP(), []int{42} } func (x *VolumeMarkReadonlyRequest) GetIp() string { @@ -2924,7 +2996,7 @@ type VolumeMarkReadonlyResponse struct { func (x *VolumeMarkReadonlyResponse) Reset() { *x = VolumeMarkReadonlyResponse{} - mi := &file_master_proto_msgTypes[42] + mi := &file_master_proto_msgTypes[43] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2936,7 +3008,7 @@ func (x *VolumeMarkReadonlyResponse) String() string { func (*VolumeMarkReadonlyResponse) ProtoMessage() {} func (x *VolumeMarkReadonlyResponse) ProtoReflect() protoreflect.Message { - mi := &file_master_proto_msgTypes[42] + mi := &file_master_proto_msgTypes[43] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2949,7 +3021,7 @@ func (x *VolumeMarkReadonlyResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use VolumeMarkReadonlyResponse.ProtoReflect.Descriptor instead. func (*VolumeMarkReadonlyResponse) Descriptor() ([]byte, []int) { - return file_master_proto_rawDescGZIP(), []int{42} + return file_master_proto_rawDescGZIP(), []int{43} } type GetMasterConfigurationRequest struct { @@ -2960,7 +3032,7 @@ type GetMasterConfigurationRequest struct { func (x *GetMasterConfigurationRequest) Reset() { *x = GetMasterConfigurationRequest{} - mi := &file_master_proto_msgTypes[43] + mi := &file_master_proto_msgTypes[44] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2972,7 +3044,7 @@ func (x *GetMasterConfigurationRequest) String() string { func (*GetMasterConfigurationRequest) ProtoMessage() {} func (x *GetMasterConfigurationRequest) ProtoReflect() protoreflect.Message { - mi := &file_master_proto_msgTypes[43] + mi := &file_master_proto_msgTypes[44] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2985,7 +3057,7 @@ func (x *GetMasterConfigurationRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use GetMasterConfigurationRequest.ProtoReflect.Descriptor instead. func (*GetMasterConfigurationRequest) Descriptor() ([]byte, []int) { - return file_master_proto_rawDescGZIP(), []int{43} + return file_master_proto_rawDescGZIP(), []int{44} } type GetMasterConfigurationResponse struct { @@ -3006,7 +3078,7 @@ type GetMasterConfigurationResponse struct { func (x *GetMasterConfigurationResponse) Reset() { *x = GetMasterConfigurationResponse{} - mi := &file_master_proto_msgTypes[44] + mi := &file_master_proto_msgTypes[45] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -3018,7 +3090,7 @@ func (x *GetMasterConfigurationResponse) String() string { func (*GetMasterConfigurationResponse) ProtoMessage() {} func (x *GetMasterConfigurationResponse) ProtoReflect() protoreflect.Message { - mi := &file_master_proto_msgTypes[44] + mi := &file_master_proto_msgTypes[45] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -3031,7 +3103,7 @@ func (x *GetMasterConfigurationResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use GetMasterConfigurationResponse.ProtoReflect.Descriptor instead. func (*GetMasterConfigurationResponse) Descriptor() ([]byte, []int) { - return file_master_proto_rawDescGZIP(), []int{44} + return file_master_proto_rawDescGZIP(), []int{45} } func (x *GetMasterConfigurationResponse) GetMetricsAddress() string { @@ -3108,7 +3180,7 @@ type ListClusterNodesRequest struct { func (x *ListClusterNodesRequest) Reset() { *x = ListClusterNodesRequest{} - mi := &file_master_proto_msgTypes[45] + mi := &file_master_proto_msgTypes[46] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -3120,7 +3192,7 @@ func (x *ListClusterNodesRequest) String() string { func (*ListClusterNodesRequest) ProtoMessage() {} func (x *ListClusterNodesRequest) ProtoReflect() protoreflect.Message { - mi := &file_master_proto_msgTypes[45] + mi := &file_master_proto_msgTypes[46] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -3133,7 +3205,7 @@ func (x *ListClusterNodesRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use ListClusterNodesRequest.ProtoReflect.Descriptor instead. func (*ListClusterNodesRequest) Descriptor() ([]byte, []int) { - return file_master_proto_rawDescGZIP(), []int{45} + return file_master_proto_rawDescGZIP(), []int{46} } func (x *ListClusterNodesRequest) GetClientType() string { @@ -3166,7 +3238,7 @@ type ListClusterNodesResponse struct { func (x *ListClusterNodesResponse) Reset() { *x = ListClusterNodesResponse{} - mi := &file_master_proto_msgTypes[46] + mi := &file_master_proto_msgTypes[47] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -3178,7 +3250,7 @@ func (x *ListClusterNodesResponse) String() string { func (*ListClusterNodesResponse) ProtoMessage() {} func (x *ListClusterNodesResponse) ProtoReflect() protoreflect.Message { - mi := &file_master_proto_msgTypes[46] + mi := &file_master_proto_msgTypes[47] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -3191,7 +3263,7 @@ func (x *ListClusterNodesResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use ListClusterNodesResponse.ProtoReflect.Descriptor instead. func (*ListClusterNodesResponse) Descriptor() ([]byte, []int) { - return file_master_proto_rawDescGZIP(), []int{46} + return file_master_proto_rawDescGZIP(), []int{47} } func (x *ListClusterNodesResponse) GetClusterNodes() []*ListClusterNodesResponse_ClusterNode { @@ -3214,7 +3286,7 @@ type LeaseAdminTokenRequest struct { func (x *LeaseAdminTokenRequest) Reset() { *x = LeaseAdminTokenRequest{} - mi := &file_master_proto_msgTypes[47] + mi := &file_master_proto_msgTypes[48] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -3226,7 +3298,7 @@ func (x *LeaseAdminTokenRequest) String() string { func (*LeaseAdminTokenRequest) ProtoMessage() {} func (x *LeaseAdminTokenRequest) ProtoReflect() protoreflect.Message { - mi := &file_master_proto_msgTypes[47] + mi := &file_master_proto_msgTypes[48] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -3239,7 +3311,7 @@ func (x *LeaseAdminTokenRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use LeaseAdminTokenRequest.ProtoReflect.Descriptor instead. func (*LeaseAdminTokenRequest) Descriptor() ([]byte, []int) { - return file_master_proto_rawDescGZIP(), []int{47} + return file_master_proto_rawDescGZIP(), []int{48} } func (x *LeaseAdminTokenRequest) GetPreviousToken() int64 { @@ -3287,7 +3359,7 @@ type LeaseAdminTokenResponse struct { func (x *LeaseAdminTokenResponse) Reset() { *x = LeaseAdminTokenResponse{} - mi := &file_master_proto_msgTypes[48] + mi := &file_master_proto_msgTypes[49] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -3299,7 +3371,7 @@ func (x *LeaseAdminTokenResponse) String() string { func (*LeaseAdminTokenResponse) ProtoMessage() {} func (x *LeaseAdminTokenResponse) ProtoReflect() protoreflect.Message { - mi := &file_master_proto_msgTypes[48] + mi := &file_master_proto_msgTypes[49] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -3312,7 +3384,7 @@ func (x *LeaseAdminTokenResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use LeaseAdminTokenResponse.ProtoReflect.Descriptor instead. func (*LeaseAdminTokenResponse) Descriptor() ([]byte, []int) { - return file_master_proto_rawDescGZIP(), []int{48} + return file_master_proto_rawDescGZIP(), []int{49} } func (x *LeaseAdminTokenResponse) GetToken() int64 { @@ -3340,7 +3412,7 @@ type ReleaseAdminTokenRequest struct { func (x *ReleaseAdminTokenRequest) Reset() { *x = ReleaseAdminTokenRequest{} - mi := &file_master_proto_msgTypes[49] + mi := &file_master_proto_msgTypes[50] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -3352,7 +3424,7 @@ func (x *ReleaseAdminTokenRequest) String() string { func (*ReleaseAdminTokenRequest) ProtoMessage() {} func (x *ReleaseAdminTokenRequest) ProtoReflect() protoreflect.Message { - mi := &file_master_proto_msgTypes[49] + mi := &file_master_proto_msgTypes[50] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -3365,7 +3437,7 @@ func (x *ReleaseAdminTokenRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use ReleaseAdminTokenRequest.ProtoReflect.Descriptor instead. func (*ReleaseAdminTokenRequest) Descriptor() ([]byte, []int) { - return file_master_proto_rawDescGZIP(), []int{49} + return file_master_proto_rawDescGZIP(), []int{50} } func (x *ReleaseAdminTokenRequest) GetPreviousToken() int64 { @@ -3397,7 +3469,7 @@ type ReleaseAdminTokenResponse struct { func (x *ReleaseAdminTokenResponse) Reset() { *x = ReleaseAdminTokenResponse{} - mi := &file_master_proto_msgTypes[50] + mi := &file_master_proto_msgTypes[51] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -3409,7 +3481,7 @@ func (x *ReleaseAdminTokenResponse) String() string { func (*ReleaseAdminTokenResponse) ProtoMessage() {} func (x *ReleaseAdminTokenResponse) ProtoReflect() protoreflect.Message { - mi := &file_master_proto_msgTypes[50] + mi := &file_master_proto_msgTypes[51] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -3422,7 +3494,7 @@ func (x *ReleaseAdminTokenResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use ReleaseAdminTokenResponse.ProtoReflect.Descriptor instead. func (*ReleaseAdminTokenResponse) Descriptor() ([]byte, []int) { - return file_master_proto_rawDescGZIP(), []int{50} + return file_master_proto_rawDescGZIP(), []int{51} } type PingRequest struct { @@ -3435,7 +3507,7 @@ type PingRequest struct { func (x *PingRequest) Reset() { *x = PingRequest{} - mi := &file_master_proto_msgTypes[51] + mi := &file_master_proto_msgTypes[52] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -3447,7 +3519,7 @@ func (x *PingRequest) String() string { func (*PingRequest) ProtoMessage() {} func (x *PingRequest) ProtoReflect() protoreflect.Message { - mi := &file_master_proto_msgTypes[51] + mi := &file_master_proto_msgTypes[52] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -3460,7 +3532,7 @@ func (x *PingRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use PingRequest.ProtoReflect.Descriptor instead. func (*PingRequest) Descriptor() ([]byte, []int) { - return file_master_proto_rawDescGZIP(), []int{51} + return file_master_proto_rawDescGZIP(), []int{52} } func (x *PingRequest) GetTarget() string { @@ -3488,7 +3560,7 @@ type PingResponse struct { func (x *PingResponse) Reset() { *x = PingResponse{} - mi := &file_master_proto_msgTypes[52] + mi := &file_master_proto_msgTypes[53] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -3500,7 +3572,7 @@ func (x *PingResponse) String() string { func (*PingResponse) ProtoMessage() {} func (x *PingResponse) ProtoReflect() protoreflect.Message { - mi := &file_master_proto_msgTypes[52] + mi := &file_master_proto_msgTypes[53] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -3513,7 +3585,7 @@ func (x *PingResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use PingResponse.ProtoReflect.Descriptor instead. func (*PingResponse) Descriptor() ([]byte, []int) { - return file_master_proto_rawDescGZIP(), []int{52} + return file_master_proto_rawDescGZIP(), []int{53} } func (x *PingResponse) GetStartTimeNs() int64 { @@ -3548,7 +3620,7 @@ type RaftAddServerRequest struct { func (x *RaftAddServerRequest) Reset() { *x = RaftAddServerRequest{} - mi := &file_master_proto_msgTypes[53] + mi := &file_master_proto_msgTypes[54] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -3560,7 +3632,7 @@ func (x *RaftAddServerRequest) String() string { func (*RaftAddServerRequest) ProtoMessage() {} func (x *RaftAddServerRequest) ProtoReflect() protoreflect.Message { - mi := &file_master_proto_msgTypes[53] + mi := &file_master_proto_msgTypes[54] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -3573,7 +3645,7 @@ func (x *RaftAddServerRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use RaftAddServerRequest.ProtoReflect.Descriptor instead. func (*RaftAddServerRequest) Descriptor() ([]byte, []int) { - return file_master_proto_rawDescGZIP(), []int{53} + return file_master_proto_rawDescGZIP(), []int{54} } func (x *RaftAddServerRequest) GetId() string { @@ -3605,7 +3677,7 @@ type RaftAddServerResponse struct { func (x *RaftAddServerResponse) Reset() { *x = RaftAddServerResponse{} - mi := &file_master_proto_msgTypes[54] + mi := &file_master_proto_msgTypes[55] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -3617,7 +3689,7 @@ func (x *RaftAddServerResponse) String() string { func (*RaftAddServerResponse) ProtoMessage() {} func (x *RaftAddServerResponse) ProtoReflect() protoreflect.Message { - mi := &file_master_proto_msgTypes[54] + mi := &file_master_proto_msgTypes[55] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -3630,7 +3702,7 @@ func (x *RaftAddServerResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use RaftAddServerResponse.ProtoReflect.Descriptor instead. func (*RaftAddServerResponse) Descriptor() ([]byte, []int) { - return file_master_proto_rawDescGZIP(), []int{54} + return file_master_proto_rawDescGZIP(), []int{55} } type RaftRemoveServerRequest struct { @@ -3643,7 +3715,7 @@ type RaftRemoveServerRequest struct { func (x *RaftRemoveServerRequest) Reset() { *x = RaftRemoveServerRequest{} - mi := &file_master_proto_msgTypes[55] + mi := &file_master_proto_msgTypes[56] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -3655,7 +3727,7 @@ func (x *RaftRemoveServerRequest) String() string { func (*RaftRemoveServerRequest) ProtoMessage() {} func (x *RaftRemoveServerRequest) ProtoReflect() protoreflect.Message { - mi := &file_master_proto_msgTypes[55] + mi := &file_master_proto_msgTypes[56] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -3668,7 +3740,7 @@ func (x *RaftRemoveServerRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use RaftRemoveServerRequest.ProtoReflect.Descriptor instead. func (*RaftRemoveServerRequest) Descriptor() ([]byte, []int) { - return file_master_proto_rawDescGZIP(), []int{55} + return file_master_proto_rawDescGZIP(), []int{56} } func (x *RaftRemoveServerRequest) GetId() string { @@ -3693,7 +3765,7 @@ type RaftRemoveServerResponse struct { func (x *RaftRemoveServerResponse) Reset() { *x = RaftRemoveServerResponse{} - mi := &file_master_proto_msgTypes[56] + mi := &file_master_proto_msgTypes[57] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -3705,7 +3777,7 @@ func (x *RaftRemoveServerResponse) String() string { func (*RaftRemoveServerResponse) ProtoMessage() {} func (x *RaftRemoveServerResponse) ProtoReflect() protoreflect.Message { - mi := &file_master_proto_msgTypes[56] + mi := &file_master_proto_msgTypes[57] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -3718,7 +3790,7 @@ func (x *RaftRemoveServerResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use RaftRemoveServerResponse.ProtoReflect.Descriptor instead. func (*RaftRemoveServerResponse) Descriptor() ([]byte, []int) { - return file_master_proto_rawDescGZIP(), []int{56} + return file_master_proto_rawDescGZIP(), []int{57} } type RaftListClusterServersRequest struct { @@ -3729,7 +3801,7 @@ type RaftListClusterServersRequest struct { func (x *RaftListClusterServersRequest) Reset() { *x = RaftListClusterServersRequest{} - mi := &file_master_proto_msgTypes[57] + mi := &file_master_proto_msgTypes[58] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -3741,7 +3813,7 @@ func (x *RaftListClusterServersRequest) String() string { func (*RaftListClusterServersRequest) ProtoMessage() {} func (x *RaftListClusterServersRequest) ProtoReflect() protoreflect.Message { - mi := &file_master_proto_msgTypes[57] + mi := &file_master_proto_msgTypes[58] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -3754,7 +3826,7 @@ func (x *RaftListClusterServersRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use RaftListClusterServersRequest.ProtoReflect.Descriptor instead. func (*RaftListClusterServersRequest) Descriptor() ([]byte, []int) { - return file_master_proto_rawDescGZIP(), []int{57} + return file_master_proto_rawDescGZIP(), []int{58} } type RaftListClusterServersResponse struct { @@ -3766,7 +3838,7 @@ type RaftListClusterServersResponse struct { func (x *RaftListClusterServersResponse) Reset() { *x = RaftListClusterServersResponse{} - mi := &file_master_proto_msgTypes[58] + mi := &file_master_proto_msgTypes[59] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -3778,7 +3850,7 @@ func (x *RaftListClusterServersResponse) String() string { func (*RaftListClusterServersResponse) ProtoMessage() {} func (x *RaftListClusterServersResponse) ProtoReflect() protoreflect.Message { - mi := &file_master_proto_msgTypes[58] + mi := &file_master_proto_msgTypes[59] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -3791,7 +3863,7 @@ func (x *RaftListClusterServersResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use RaftListClusterServersResponse.ProtoReflect.Descriptor instead. func (*RaftListClusterServersResponse) Descriptor() ([]byte, []int) { - return file_master_proto_rawDescGZIP(), []int{58} + return file_master_proto_rawDescGZIP(), []int{59} } func (x *RaftListClusterServersResponse) GetClusterServers() []*RaftListClusterServersResponse_ClusterServers { @@ -3811,7 +3883,7 @@ type RaftLeadershipTransferRequest struct { func (x *RaftLeadershipTransferRequest) Reset() { *x = RaftLeadershipTransferRequest{} - mi := &file_master_proto_msgTypes[59] + mi := &file_master_proto_msgTypes[60] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -3823,7 +3895,7 @@ func (x *RaftLeadershipTransferRequest) String() string { func (*RaftLeadershipTransferRequest) ProtoMessage() {} func (x *RaftLeadershipTransferRequest) ProtoReflect() protoreflect.Message { - mi := &file_master_proto_msgTypes[59] + mi := &file_master_proto_msgTypes[60] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -3836,7 +3908,7 @@ func (x *RaftLeadershipTransferRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use RaftLeadershipTransferRequest.ProtoReflect.Descriptor instead. func (*RaftLeadershipTransferRequest) Descriptor() ([]byte, []int) { - return file_master_proto_rawDescGZIP(), []int{59} + return file_master_proto_rawDescGZIP(), []int{60} } func (x *RaftLeadershipTransferRequest) GetTargetId() string { @@ -3863,7 +3935,7 @@ type RaftLeadershipTransferResponse struct { func (x *RaftLeadershipTransferResponse) Reset() { *x = RaftLeadershipTransferResponse{} - mi := &file_master_proto_msgTypes[60] + mi := &file_master_proto_msgTypes[61] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -3875,7 +3947,7 @@ func (x *RaftLeadershipTransferResponse) String() string { func (*RaftLeadershipTransferResponse) ProtoMessage() {} func (x *RaftLeadershipTransferResponse) ProtoReflect() protoreflect.Message { - mi := &file_master_proto_msgTypes[60] + mi := &file_master_proto_msgTypes[61] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -3888,7 +3960,7 @@ func (x *RaftLeadershipTransferResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use RaftLeadershipTransferResponse.ProtoReflect.Descriptor instead. func (*RaftLeadershipTransferResponse) Descriptor() ([]byte, []int) { - return file_master_proto_rawDescGZIP(), []int{60} + return file_master_proto_rawDescGZIP(), []int{61} } func (x *RaftLeadershipTransferResponse) GetPreviousLeader() string { @@ -3913,7 +3985,7 @@ type VolumeGrowResponse struct { func (x *VolumeGrowResponse) Reset() { *x = VolumeGrowResponse{} - mi := &file_master_proto_msgTypes[61] + mi := &file_master_proto_msgTypes[62] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -3925,7 +3997,7 @@ func (x *VolumeGrowResponse) String() string { func (*VolumeGrowResponse) ProtoMessage() {} func (x *VolumeGrowResponse) ProtoReflect() protoreflect.Message { - mi := &file_master_proto_msgTypes[61] + mi := &file_master_proto_msgTypes[62] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -3938,7 +4010,7 @@ func (x *VolumeGrowResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use VolumeGrowResponse.ProtoReflect.Descriptor instead. func (*VolumeGrowResponse) Descriptor() ([]byte, []int) { - return file_master_proto_rawDescGZIP(), []int{61} + return file_master_proto_rawDescGZIP(), []int{62} } type SuperBlockExtra_ErasureCoding struct { @@ -3952,7 +4024,7 @@ type SuperBlockExtra_ErasureCoding struct { func (x *SuperBlockExtra_ErasureCoding) Reset() { *x = SuperBlockExtra_ErasureCoding{} - mi := &file_master_proto_msgTypes[64] + mi := &file_master_proto_msgTypes[65] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -3964,7 +4036,7 @@ func (x *SuperBlockExtra_ErasureCoding) String() string { func (*SuperBlockExtra_ErasureCoding) ProtoMessage() {} func (x *SuperBlockExtra_ErasureCoding) ProtoReflect() protoreflect.Message { - mi := &file_master_proto_msgTypes[64] + mi := &file_master_proto_msgTypes[65] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -4013,7 +4085,7 @@ type LookupVolumeResponse_VolumeIdLocation struct { func (x *LookupVolumeResponse_VolumeIdLocation) Reset() { *x = LookupVolumeResponse_VolumeIdLocation{} - mi := &file_master_proto_msgTypes[65] + mi := &file_master_proto_msgTypes[66] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -4025,7 +4097,7 @@ func (x *LookupVolumeResponse_VolumeIdLocation) String() string { func (*LookupVolumeResponse_VolumeIdLocation) ProtoMessage() {} func (x *LookupVolumeResponse_VolumeIdLocation) ProtoReflect() protoreflect.Message { - mi := &file_master_proto_msgTypes[65] + mi := &file_master_proto_msgTypes[66] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -4038,7 +4110,7 @@ func (x *LookupVolumeResponse_VolumeIdLocation) ProtoReflect() protoreflect.Mess // Deprecated: Use LookupVolumeResponse_VolumeIdLocation.ProtoReflect.Descriptor instead. func (*LookupVolumeResponse_VolumeIdLocation) Descriptor() ([]byte, []int) { - return file_master_proto_rawDescGZIP(), []int{14, 0} + return file_master_proto_rawDescGZIP(), []int{15, 0} } func (x *LookupVolumeResponse_VolumeIdLocation) GetVolumeOrFileId() string { @@ -4079,7 +4151,7 @@ type LookupEcVolumeResponse_EcShardIdLocation struct { func (x *LookupEcVolumeResponse_EcShardIdLocation) Reset() { *x = LookupEcVolumeResponse_EcShardIdLocation{} - mi := &file_master_proto_msgTypes[70] + mi := &file_master_proto_msgTypes[71] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -4091,7 +4163,7 @@ func (x *LookupEcVolumeResponse_EcShardIdLocation) String() string { func (*LookupEcVolumeResponse_EcShardIdLocation) ProtoMessage() {} func (x *LookupEcVolumeResponse_EcShardIdLocation) ProtoReflect() protoreflect.Message { - mi := &file_master_proto_msgTypes[70] + mi := &file_master_proto_msgTypes[71] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -4104,7 +4176,7 @@ func (x *LookupEcVolumeResponse_EcShardIdLocation) ProtoReflect() protoreflect.M // Deprecated: Use LookupEcVolumeResponse_EcShardIdLocation.ProtoReflect.Descriptor instead. func (*LookupEcVolumeResponse_EcShardIdLocation) Descriptor() ([]byte, []int) { - return file_master_proto_rawDescGZIP(), []int{34, 0} + return file_master_proto_rawDescGZIP(), []int{35, 0} } func (x *LookupEcVolumeResponse_EcShardIdLocation) GetShardId() uint32 { @@ -4134,7 +4206,7 @@ type ListClusterNodesResponse_ClusterNode struct { func (x *ListClusterNodesResponse_ClusterNode) Reset() { *x = ListClusterNodesResponse_ClusterNode{} - mi := &file_master_proto_msgTypes[71] + mi := &file_master_proto_msgTypes[72] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -4146,7 +4218,7 @@ func (x *ListClusterNodesResponse_ClusterNode) String() string { func (*ListClusterNodesResponse_ClusterNode) ProtoMessage() {} func (x *ListClusterNodesResponse_ClusterNode) ProtoReflect() protoreflect.Message { - mi := &file_master_proto_msgTypes[71] + mi := &file_master_proto_msgTypes[72] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -4159,7 +4231,7 @@ func (x *ListClusterNodesResponse_ClusterNode) ProtoReflect() protoreflect.Messa // Deprecated: Use ListClusterNodesResponse_ClusterNode.ProtoReflect.Descriptor instead. func (*ListClusterNodesResponse_ClusterNode) Descriptor() ([]byte, []int) { - return file_master_proto_rawDescGZIP(), []int{46, 0} + return file_master_proto_rawDescGZIP(), []int{47, 0} } func (x *ListClusterNodesResponse_ClusterNode) GetAddress() string { @@ -4209,7 +4281,7 @@ type RaftListClusterServersResponse_ClusterServers struct { func (x *RaftListClusterServersResponse_ClusterServers) Reset() { *x = RaftListClusterServersResponse_ClusterServers{} - mi := &file_master_proto_msgTypes[72] + mi := &file_master_proto_msgTypes[73] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -4221,7 +4293,7 @@ func (x *RaftListClusterServersResponse_ClusterServers) String() string { func (*RaftListClusterServersResponse_ClusterServers) ProtoMessage() {} func (x *RaftListClusterServersResponse_ClusterServers) ProtoReflect() protoreflect.Message { - mi := &file_master_proto_msgTypes[72] + mi := &file_master_proto_msgTypes[73] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -4234,7 +4306,7 @@ func (x *RaftListClusterServersResponse_ClusterServers) ProtoReflect() protorefl // Deprecated: Use RaftListClusterServersResponse_ClusterServers.ProtoReflect.Descriptor instead. func (*RaftListClusterServersResponse_ClusterServers) Descriptor() ([]byte, []int) { - return file_master_proto_rawDescGZIP(), []int{58, 0} + return file_master_proto_rawDescGZIP(), []int{59, 0} } func (x *RaftListClusterServersResponse_ClusterServers) GetId() string { @@ -4400,10 +4472,16 @@ const file_master_proto_rawDesc = "" + "\x06is_add\x18\x04 \x01(\bR\x05isAdd\x12\x1f\n" + "\vfiler_group\x18\x05 \x01(\tR\n" + "filerGroup\x12\"\n" + - "\rcreated_at_ns\x18\x06 \x01(\x03R\vcreatedAtNs\"\xa9\x01\n" + + "\rcreated_at_ns\x18\x06 \x01(\x03R\vcreatedAtNs\"\xee\x01\n" + "\x15KeepConnectedResponse\x12B\n" + "\x0fvolume_location\x18\x01 \x01(\v2\x19.master_pb.VolumeLocationR\x0evolumeLocation\x12L\n" + - "\x13cluster_node_update\x18\x02 \x01(\v2\x1c.master_pb.ClusterNodeUpdateR\x11clusterNodeUpdate\"b\n" + + "\x13cluster_node_update\x18\x02 \x01(\v2\x1c.master_pb.ClusterNodeUpdateR\x11clusterNodeUpdate\x12C\n" + + "\x10lock_ring_update\x18\x03 \x01(\v2\x19.master_pb.LockRingUpdateR\x0elockRingUpdate\"e\n" + + "\x0eLockRingUpdate\x12\x1f\n" + + "\vfiler_group\x18\x01 \x01(\tR\n" + + "filerGroup\x12\x18\n" + + "\aservers\x18\x02 \x03(\tR\aservers\x12\x18\n" + + "\aversion\x18\x03 \x01(\x03R\aversion\"b\n" + "\x13LookupVolumeRequest\x12+\n" + "\x12volume_or_file_ids\x18\x01 \x03(\tR\x0fvolumeOrFileIds\x12\x1e\n" + "\n" + @@ -4681,7 +4759,7 @@ func file_master_proto_rawDescGZIP() []byte { return file_master_proto_rawDescData } -var file_master_proto_msgTypes = make([]protoimpl.MessageInfo, 73) +var file_master_proto_msgTypes = make([]protoimpl.MessageInfo, 74) var file_master_proto_goTypes = []any{ (*DiskTag)(nil), // 0: master_pb.DiskTag (*Heartbeat)(nil), // 1: master_pb.Heartbeat @@ -4696,67 +4774,68 @@ var file_master_proto_goTypes = []any{ (*VolumeLocation)(nil), // 10: master_pb.VolumeLocation (*ClusterNodeUpdate)(nil), // 11: master_pb.ClusterNodeUpdate (*KeepConnectedResponse)(nil), // 12: master_pb.KeepConnectedResponse - (*LookupVolumeRequest)(nil), // 13: master_pb.LookupVolumeRequest - (*LookupVolumeResponse)(nil), // 14: master_pb.LookupVolumeResponse - (*Location)(nil), // 15: master_pb.Location - (*AssignRequest)(nil), // 16: master_pb.AssignRequest - (*VolumeGrowRequest)(nil), // 17: master_pb.VolumeGrowRequest - (*AssignResponse)(nil), // 18: master_pb.AssignResponse - (*StatisticsRequest)(nil), // 19: master_pb.StatisticsRequest - (*StatisticsResponse)(nil), // 20: master_pb.StatisticsResponse - (*Collection)(nil), // 21: master_pb.Collection - (*CollectionListRequest)(nil), // 22: master_pb.CollectionListRequest - (*CollectionListResponse)(nil), // 23: master_pb.CollectionListResponse - (*CollectionDeleteRequest)(nil), // 24: master_pb.CollectionDeleteRequest - (*CollectionDeleteResponse)(nil), // 25: master_pb.CollectionDeleteResponse - (*DiskInfo)(nil), // 26: master_pb.DiskInfo - (*DataNodeInfo)(nil), // 27: master_pb.DataNodeInfo - (*RackInfo)(nil), // 28: master_pb.RackInfo - (*DataCenterInfo)(nil), // 29: master_pb.DataCenterInfo - (*TopologyInfo)(nil), // 30: master_pb.TopologyInfo - (*VolumeListRequest)(nil), // 31: master_pb.VolumeListRequest - (*VolumeListResponse)(nil), // 32: master_pb.VolumeListResponse - (*LookupEcVolumeRequest)(nil), // 33: master_pb.LookupEcVolumeRequest - (*LookupEcVolumeResponse)(nil), // 34: master_pb.LookupEcVolumeResponse - (*VacuumVolumeRequest)(nil), // 35: master_pb.VacuumVolumeRequest - (*VacuumVolumeResponse)(nil), // 36: master_pb.VacuumVolumeResponse - (*DisableVacuumRequest)(nil), // 37: master_pb.DisableVacuumRequest - (*DisableVacuumResponse)(nil), // 38: master_pb.DisableVacuumResponse - (*EnableVacuumRequest)(nil), // 39: master_pb.EnableVacuumRequest - (*EnableVacuumResponse)(nil), // 40: master_pb.EnableVacuumResponse - (*VolumeMarkReadonlyRequest)(nil), // 41: master_pb.VolumeMarkReadonlyRequest - (*VolumeMarkReadonlyResponse)(nil), // 42: master_pb.VolumeMarkReadonlyResponse - (*GetMasterConfigurationRequest)(nil), // 43: master_pb.GetMasterConfigurationRequest - (*GetMasterConfigurationResponse)(nil), // 44: master_pb.GetMasterConfigurationResponse - (*ListClusterNodesRequest)(nil), // 45: master_pb.ListClusterNodesRequest - (*ListClusterNodesResponse)(nil), // 46: master_pb.ListClusterNodesResponse - (*LeaseAdminTokenRequest)(nil), // 47: master_pb.LeaseAdminTokenRequest - (*LeaseAdminTokenResponse)(nil), // 48: master_pb.LeaseAdminTokenResponse - (*ReleaseAdminTokenRequest)(nil), // 49: master_pb.ReleaseAdminTokenRequest - (*ReleaseAdminTokenResponse)(nil), // 50: master_pb.ReleaseAdminTokenResponse - (*PingRequest)(nil), // 51: master_pb.PingRequest - (*PingResponse)(nil), // 52: master_pb.PingResponse - (*RaftAddServerRequest)(nil), // 53: master_pb.RaftAddServerRequest - (*RaftAddServerResponse)(nil), // 54: master_pb.RaftAddServerResponse - (*RaftRemoveServerRequest)(nil), // 55: master_pb.RaftRemoveServerRequest - (*RaftRemoveServerResponse)(nil), // 56: master_pb.RaftRemoveServerResponse - (*RaftListClusterServersRequest)(nil), // 57: master_pb.RaftListClusterServersRequest - (*RaftListClusterServersResponse)(nil), // 58: master_pb.RaftListClusterServersResponse - (*RaftLeadershipTransferRequest)(nil), // 59: master_pb.RaftLeadershipTransferRequest - (*RaftLeadershipTransferResponse)(nil), // 60: master_pb.RaftLeadershipTransferResponse - (*VolumeGrowResponse)(nil), // 61: master_pb.VolumeGrowResponse - nil, // 62: master_pb.Heartbeat.MaxVolumeCountsEntry - nil, // 63: master_pb.StorageBackend.PropertiesEntry - (*SuperBlockExtra_ErasureCoding)(nil), // 64: master_pb.SuperBlockExtra.ErasureCoding - (*LookupVolumeResponse_VolumeIdLocation)(nil), // 65: master_pb.LookupVolumeResponse.VolumeIdLocation - nil, // 66: master_pb.DataNodeInfo.DiskInfosEntry - nil, // 67: master_pb.RackInfo.DiskInfosEntry - nil, // 68: master_pb.DataCenterInfo.DiskInfosEntry - nil, // 69: master_pb.TopologyInfo.DiskInfosEntry - (*LookupEcVolumeResponse_EcShardIdLocation)(nil), // 70: master_pb.LookupEcVolumeResponse.EcShardIdLocation - (*ListClusterNodesResponse_ClusterNode)(nil), // 71: master_pb.ListClusterNodesResponse.ClusterNode - (*RaftListClusterServersResponse_ClusterServers)(nil), // 72: master_pb.RaftListClusterServersResponse.ClusterServers - (*volume_server_pb.VolumeServerState)(nil), // 73: volume_server_pb.VolumeServerState + (*LockRingUpdate)(nil), // 13: master_pb.LockRingUpdate + (*LookupVolumeRequest)(nil), // 14: master_pb.LookupVolumeRequest + (*LookupVolumeResponse)(nil), // 15: master_pb.LookupVolumeResponse + (*Location)(nil), // 16: master_pb.Location + (*AssignRequest)(nil), // 17: master_pb.AssignRequest + (*VolumeGrowRequest)(nil), // 18: master_pb.VolumeGrowRequest + (*AssignResponse)(nil), // 19: master_pb.AssignResponse + (*StatisticsRequest)(nil), // 20: master_pb.StatisticsRequest + (*StatisticsResponse)(nil), // 21: master_pb.StatisticsResponse + (*Collection)(nil), // 22: master_pb.Collection + (*CollectionListRequest)(nil), // 23: master_pb.CollectionListRequest + (*CollectionListResponse)(nil), // 24: master_pb.CollectionListResponse + (*CollectionDeleteRequest)(nil), // 25: master_pb.CollectionDeleteRequest + (*CollectionDeleteResponse)(nil), // 26: master_pb.CollectionDeleteResponse + (*DiskInfo)(nil), // 27: master_pb.DiskInfo + (*DataNodeInfo)(nil), // 28: master_pb.DataNodeInfo + (*RackInfo)(nil), // 29: master_pb.RackInfo + (*DataCenterInfo)(nil), // 30: master_pb.DataCenterInfo + (*TopologyInfo)(nil), // 31: master_pb.TopologyInfo + (*VolumeListRequest)(nil), // 32: master_pb.VolumeListRequest + (*VolumeListResponse)(nil), // 33: master_pb.VolumeListResponse + (*LookupEcVolumeRequest)(nil), // 34: master_pb.LookupEcVolumeRequest + (*LookupEcVolumeResponse)(nil), // 35: master_pb.LookupEcVolumeResponse + (*VacuumVolumeRequest)(nil), // 36: master_pb.VacuumVolumeRequest + (*VacuumVolumeResponse)(nil), // 37: master_pb.VacuumVolumeResponse + (*DisableVacuumRequest)(nil), // 38: master_pb.DisableVacuumRequest + (*DisableVacuumResponse)(nil), // 39: master_pb.DisableVacuumResponse + (*EnableVacuumRequest)(nil), // 40: master_pb.EnableVacuumRequest + (*EnableVacuumResponse)(nil), // 41: master_pb.EnableVacuumResponse + (*VolumeMarkReadonlyRequest)(nil), // 42: master_pb.VolumeMarkReadonlyRequest + (*VolumeMarkReadonlyResponse)(nil), // 43: master_pb.VolumeMarkReadonlyResponse + (*GetMasterConfigurationRequest)(nil), // 44: master_pb.GetMasterConfigurationRequest + (*GetMasterConfigurationResponse)(nil), // 45: master_pb.GetMasterConfigurationResponse + (*ListClusterNodesRequest)(nil), // 46: master_pb.ListClusterNodesRequest + (*ListClusterNodesResponse)(nil), // 47: master_pb.ListClusterNodesResponse + (*LeaseAdminTokenRequest)(nil), // 48: master_pb.LeaseAdminTokenRequest + (*LeaseAdminTokenResponse)(nil), // 49: master_pb.LeaseAdminTokenResponse + (*ReleaseAdminTokenRequest)(nil), // 50: master_pb.ReleaseAdminTokenRequest + (*ReleaseAdminTokenResponse)(nil), // 51: master_pb.ReleaseAdminTokenResponse + (*PingRequest)(nil), // 52: master_pb.PingRequest + (*PingResponse)(nil), // 53: master_pb.PingResponse + (*RaftAddServerRequest)(nil), // 54: master_pb.RaftAddServerRequest + (*RaftAddServerResponse)(nil), // 55: master_pb.RaftAddServerResponse + (*RaftRemoveServerRequest)(nil), // 56: master_pb.RaftRemoveServerRequest + (*RaftRemoveServerResponse)(nil), // 57: master_pb.RaftRemoveServerResponse + (*RaftListClusterServersRequest)(nil), // 58: master_pb.RaftListClusterServersRequest + (*RaftListClusterServersResponse)(nil), // 59: master_pb.RaftListClusterServersResponse + (*RaftLeadershipTransferRequest)(nil), // 60: master_pb.RaftLeadershipTransferRequest + (*RaftLeadershipTransferResponse)(nil), // 61: master_pb.RaftLeadershipTransferResponse + (*VolumeGrowResponse)(nil), // 62: master_pb.VolumeGrowResponse + nil, // 63: master_pb.Heartbeat.MaxVolumeCountsEntry + nil, // 64: master_pb.StorageBackend.PropertiesEntry + (*SuperBlockExtra_ErasureCoding)(nil), // 65: master_pb.SuperBlockExtra.ErasureCoding + (*LookupVolumeResponse_VolumeIdLocation)(nil), // 66: master_pb.LookupVolumeResponse.VolumeIdLocation + nil, // 67: master_pb.DataNodeInfo.DiskInfosEntry + nil, // 68: master_pb.RackInfo.DiskInfosEntry + nil, // 69: master_pb.DataCenterInfo.DiskInfosEntry + nil, // 70: master_pb.TopologyInfo.DiskInfosEntry + (*LookupEcVolumeResponse_EcShardIdLocation)(nil), // 71: master_pb.LookupEcVolumeResponse.EcShardIdLocation + (*ListClusterNodesResponse_ClusterNode)(nil), // 72: master_pb.ListClusterNodesResponse.ClusterNode + (*RaftListClusterServersResponse_ClusterServers)(nil), // 73: master_pb.RaftListClusterServersResponse.ClusterServers + (*volume_server_pb.VolumeServerState)(nil), // 74: volume_server_pb.VolumeServerState } var file_master_proto_depIdxs = []int32{ 3, // 0: master_pb.Heartbeat.volumes:type_name -> master_pb.VolumeInformationMessage @@ -4765,91 +4844,92 @@ var file_master_proto_depIdxs = []int32{ 5, // 3: master_pb.Heartbeat.ec_shards:type_name -> master_pb.VolumeEcShardInformationMessage 5, // 4: master_pb.Heartbeat.new_ec_shards:type_name -> master_pb.VolumeEcShardInformationMessage 5, // 5: master_pb.Heartbeat.deleted_ec_shards:type_name -> master_pb.VolumeEcShardInformationMessage - 62, // 6: master_pb.Heartbeat.max_volume_counts:type_name -> master_pb.Heartbeat.MaxVolumeCountsEntry - 73, // 7: master_pb.Heartbeat.state:type_name -> volume_server_pb.VolumeServerState + 63, // 6: master_pb.Heartbeat.max_volume_counts:type_name -> master_pb.Heartbeat.MaxVolumeCountsEntry + 74, // 7: master_pb.Heartbeat.state:type_name -> volume_server_pb.VolumeServerState 0, // 8: master_pb.Heartbeat.disk_tags:type_name -> master_pb.DiskTag 6, // 9: master_pb.HeartbeatResponse.storage_backends:type_name -> master_pb.StorageBackend - 63, // 10: master_pb.StorageBackend.properties:type_name -> master_pb.StorageBackend.PropertiesEntry - 64, // 11: master_pb.SuperBlockExtra.erasure_coding:type_name -> master_pb.SuperBlockExtra.ErasureCoding + 64, // 10: master_pb.StorageBackend.properties:type_name -> master_pb.StorageBackend.PropertiesEntry + 65, // 11: master_pb.SuperBlockExtra.erasure_coding:type_name -> master_pb.SuperBlockExtra.ErasureCoding 10, // 12: master_pb.KeepConnectedResponse.volume_location:type_name -> master_pb.VolumeLocation 11, // 13: master_pb.KeepConnectedResponse.cluster_node_update:type_name -> master_pb.ClusterNodeUpdate - 65, // 14: master_pb.LookupVolumeResponse.volume_id_locations:type_name -> master_pb.LookupVolumeResponse.VolumeIdLocation - 15, // 15: master_pb.AssignResponse.replicas:type_name -> master_pb.Location - 15, // 16: master_pb.AssignResponse.location:type_name -> master_pb.Location - 21, // 17: master_pb.CollectionListResponse.collections:type_name -> master_pb.Collection - 3, // 18: master_pb.DiskInfo.volume_infos:type_name -> master_pb.VolumeInformationMessage - 5, // 19: master_pb.DiskInfo.ec_shard_infos:type_name -> master_pb.VolumeEcShardInformationMessage - 66, // 20: master_pb.DataNodeInfo.diskInfos:type_name -> master_pb.DataNodeInfo.DiskInfosEntry - 27, // 21: master_pb.RackInfo.data_node_infos:type_name -> master_pb.DataNodeInfo - 67, // 22: master_pb.RackInfo.diskInfos:type_name -> master_pb.RackInfo.DiskInfosEntry - 28, // 23: master_pb.DataCenterInfo.rack_infos:type_name -> master_pb.RackInfo - 68, // 24: master_pb.DataCenterInfo.diskInfos:type_name -> master_pb.DataCenterInfo.DiskInfosEntry - 29, // 25: master_pb.TopologyInfo.data_center_infos:type_name -> master_pb.DataCenterInfo - 69, // 26: master_pb.TopologyInfo.diskInfos:type_name -> master_pb.TopologyInfo.DiskInfosEntry - 30, // 27: master_pb.VolumeListResponse.topology_info:type_name -> master_pb.TopologyInfo - 70, // 28: master_pb.LookupEcVolumeResponse.shard_id_locations:type_name -> master_pb.LookupEcVolumeResponse.EcShardIdLocation - 6, // 29: master_pb.GetMasterConfigurationResponse.storage_backends:type_name -> master_pb.StorageBackend - 71, // 30: master_pb.ListClusterNodesResponse.cluster_nodes:type_name -> master_pb.ListClusterNodesResponse.ClusterNode - 72, // 31: master_pb.RaftListClusterServersResponse.cluster_servers:type_name -> master_pb.RaftListClusterServersResponse.ClusterServers - 15, // 32: master_pb.LookupVolumeResponse.VolumeIdLocation.locations:type_name -> master_pb.Location - 26, // 33: master_pb.DataNodeInfo.DiskInfosEntry.value:type_name -> master_pb.DiskInfo - 26, // 34: master_pb.RackInfo.DiskInfosEntry.value:type_name -> master_pb.DiskInfo - 26, // 35: master_pb.DataCenterInfo.DiskInfosEntry.value:type_name -> master_pb.DiskInfo - 26, // 36: master_pb.TopologyInfo.DiskInfosEntry.value:type_name -> master_pb.DiskInfo - 15, // 37: master_pb.LookupEcVolumeResponse.EcShardIdLocation.locations:type_name -> master_pb.Location - 1, // 38: master_pb.Seaweed.SendHeartbeat:input_type -> master_pb.Heartbeat - 9, // 39: master_pb.Seaweed.KeepConnected:input_type -> master_pb.KeepConnectedRequest - 13, // 40: master_pb.Seaweed.LookupVolume:input_type -> master_pb.LookupVolumeRequest - 16, // 41: master_pb.Seaweed.Assign:input_type -> master_pb.AssignRequest - 16, // 42: master_pb.Seaweed.StreamAssign:input_type -> master_pb.AssignRequest - 19, // 43: master_pb.Seaweed.Statistics:input_type -> master_pb.StatisticsRequest - 22, // 44: master_pb.Seaweed.CollectionList:input_type -> master_pb.CollectionListRequest - 24, // 45: master_pb.Seaweed.CollectionDelete:input_type -> master_pb.CollectionDeleteRequest - 31, // 46: master_pb.Seaweed.VolumeList:input_type -> master_pb.VolumeListRequest - 33, // 47: master_pb.Seaweed.LookupEcVolume:input_type -> master_pb.LookupEcVolumeRequest - 35, // 48: master_pb.Seaweed.VacuumVolume:input_type -> master_pb.VacuumVolumeRequest - 37, // 49: master_pb.Seaweed.DisableVacuum:input_type -> master_pb.DisableVacuumRequest - 39, // 50: master_pb.Seaweed.EnableVacuum:input_type -> master_pb.EnableVacuumRequest - 41, // 51: master_pb.Seaweed.VolumeMarkReadonly:input_type -> master_pb.VolumeMarkReadonlyRequest - 43, // 52: master_pb.Seaweed.GetMasterConfiguration:input_type -> master_pb.GetMasterConfigurationRequest - 45, // 53: master_pb.Seaweed.ListClusterNodes:input_type -> master_pb.ListClusterNodesRequest - 47, // 54: master_pb.Seaweed.LeaseAdminToken:input_type -> master_pb.LeaseAdminTokenRequest - 49, // 55: master_pb.Seaweed.ReleaseAdminToken:input_type -> master_pb.ReleaseAdminTokenRequest - 51, // 56: master_pb.Seaweed.Ping:input_type -> master_pb.PingRequest - 57, // 57: master_pb.Seaweed.RaftListClusterServers:input_type -> master_pb.RaftListClusterServersRequest - 53, // 58: master_pb.Seaweed.RaftAddServer:input_type -> master_pb.RaftAddServerRequest - 55, // 59: master_pb.Seaweed.RaftRemoveServer:input_type -> master_pb.RaftRemoveServerRequest - 59, // 60: master_pb.Seaweed.RaftLeadershipTransfer:input_type -> master_pb.RaftLeadershipTransferRequest - 17, // 61: master_pb.Seaweed.VolumeGrow:input_type -> master_pb.VolumeGrowRequest - 2, // 62: master_pb.Seaweed.SendHeartbeat:output_type -> master_pb.HeartbeatResponse - 12, // 63: master_pb.Seaweed.KeepConnected:output_type -> master_pb.KeepConnectedResponse - 14, // 64: master_pb.Seaweed.LookupVolume:output_type -> master_pb.LookupVolumeResponse - 18, // 65: master_pb.Seaweed.Assign:output_type -> master_pb.AssignResponse - 18, // 66: master_pb.Seaweed.StreamAssign:output_type -> master_pb.AssignResponse - 20, // 67: master_pb.Seaweed.Statistics:output_type -> master_pb.StatisticsResponse - 23, // 68: master_pb.Seaweed.CollectionList:output_type -> master_pb.CollectionListResponse - 25, // 69: master_pb.Seaweed.CollectionDelete:output_type -> master_pb.CollectionDeleteResponse - 32, // 70: master_pb.Seaweed.VolumeList:output_type -> master_pb.VolumeListResponse - 34, // 71: master_pb.Seaweed.LookupEcVolume:output_type -> master_pb.LookupEcVolumeResponse - 36, // 72: master_pb.Seaweed.VacuumVolume:output_type -> master_pb.VacuumVolumeResponse - 38, // 73: master_pb.Seaweed.DisableVacuum:output_type -> master_pb.DisableVacuumResponse - 40, // 74: master_pb.Seaweed.EnableVacuum:output_type -> master_pb.EnableVacuumResponse - 42, // 75: master_pb.Seaweed.VolumeMarkReadonly:output_type -> master_pb.VolumeMarkReadonlyResponse - 44, // 76: master_pb.Seaweed.GetMasterConfiguration:output_type -> master_pb.GetMasterConfigurationResponse - 46, // 77: master_pb.Seaweed.ListClusterNodes:output_type -> master_pb.ListClusterNodesResponse - 48, // 78: master_pb.Seaweed.LeaseAdminToken:output_type -> master_pb.LeaseAdminTokenResponse - 50, // 79: master_pb.Seaweed.ReleaseAdminToken:output_type -> master_pb.ReleaseAdminTokenResponse - 52, // 80: master_pb.Seaweed.Ping:output_type -> master_pb.PingResponse - 58, // 81: master_pb.Seaweed.RaftListClusterServers:output_type -> master_pb.RaftListClusterServersResponse - 54, // 82: master_pb.Seaweed.RaftAddServer:output_type -> master_pb.RaftAddServerResponse - 56, // 83: master_pb.Seaweed.RaftRemoveServer:output_type -> master_pb.RaftRemoveServerResponse - 60, // 84: master_pb.Seaweed.RaftLeadershipTransfer:output_type -> master_pb.RaftLeadershipTransferResponse - 61, // 85: master_pb.Seaweed.VolumeGrow:output_type -> master_pb.VolumeGrowResponse - 62, // [62:86] is the sub-list for method output_type - 38, // [38:62] is the sub-list for method input_type - 38, // [38:38] is the sub-list for extension type_name - 38, // [38:38] is the sub-list for extension extendee - 0, // [0:38] is the sub-list for field type_name + 13, // 14: master_pb.KeepConnectedResponse.lock_ring_update:type_name -> master_pb.LockRingUpdate + 66, // 15: master_pb.LookupVolumeResponse.volume_id_locations:type_name -> master_pb.LookupVolumeResponse.VolumeIdLocation + 16, // 16: master_pb.AssignResponse.replicas:type_name -> master_pb.Location + 16, // 17: master_pb.AssignResponse.location:type_name -> master_pb.Location + 22, // 18: master_pb.CollectionListResponse.collections:type_name -> master_pb.Collection + 3, // 19: master_pb.DiskInfo.volume_infos:type_name -> master_pb.VolumeInformationMessage + 5, // 20: master_pb.DiskInfo.ec_shard_infos:type_name -> master_pb.VolumeEcShardInformationMessage + 67, // 21: master_pb.DataNodeInfo.diskInfos:type_name -> master_pb.DataNodeInfo.DiskInfosEntry + 28, // 22: master_pb.RackInfo.data_node_infos:type_name -> master_pb.DataNodeInfo + 68, // 23: master_pb.RackInfo.diskInfos:type_name -> master_pb.RackInfo.DiskInfosEntry + 29, // 24: master_pb.DataCenterInfo.rack_infos:type_name -> master_pb.RackInfo + 69, // 25: master_pb.DataCenterInfo.diskInfos:type_name -> master_pb.DataCenterInfo.DiskInfosEntry + 30, // 26: master_pb.TopologyInfo.data_center_infos:type_name -> master_pb.DataCenterInfo + 70, // 27: master_pb.TopologyInfo.diskInfos:type_name -> master_pb.TopologyInfo.DiskInfosEntry + 31, // 28: master_pb.VolumeListResponse.topology_info:type_name -> master_pb.TopologyInfo + 71, // 29: master_pb.LookupEcVolumeResponse.shard_id_locations:type_name -> master_pb.LookupEcVolumeResponse.EcShardIdLocation + 6, // 30: master_pb.GetMasterConfigurationResponse.storage_backends:type_name -> master_pb.StorageBackend + 72, // 31: master_pb.ListClusterNodesResponse.cluster_nodes:type_name -> master_pb.ListClusterNodesResponse.ClusterNode + 73, // 32: master_pb.RaftListClusterServersResponse.cluster_servers:type_name -> master_pb.RaftListClusterServersResponse.ClusterServers + 16, // 33: master_pb.LookupVolumeResponse.VolumeIdLocation.locations:type_name -> master_pb.Location + 27, // 34: master_pb.DataNodeInfo.DiskInfosEntry.value:type_name -> master_pb.DiskInfo + 27, // 35: master_pb.RackInfo.DiskInfosEntry.value:type_name -> master_pb.DiskInfo + 27, // 36: master_pb.DataCenterInfo.DiskInfosEntry.value:type_name -> master_pb.DiskInfo + 27, // 37: master_pb.TopologyInfo.DiskInfosEntry.value:type_name -> master_pb.DiskInfo + 16, // 38: master_pb.LookupEcVolumeResponse.EcShardIdLocation.locations:type_name -> master_pb.Location + 1, // 39: master_pb.Seaweed.SendHeartbeat:input_type -> master_pb.Heartbeat + 9, // 40: master_pb.Seaweed.KeepConnected:input_type -> master_pb.KeepConnectedRequest + 14, // 41: master_pb.Seaweed.LookupVolume:input_type -> master_pb.LookupVolumeRequest + 17, // 42: master_pb.Seaweed.Assign:input_type -> master_pb.AssignRequest + 17, // 43: master_pb.Seaweed.StreamAssign:input_type -> master_pb.AssignRequest + 20, // 44: master_pb.Seaweed.Statistics:input_type -> master_pb.StatisticsRequest + 23, // 45: master_pb.Seaweed.CollectionList:input_type -> master_pb.CollectionListRequest + 25, // 46: master_pb.Seaweed.CollectionDelete:input_type -> master_pb.CollectionDeleteRequest + 32, // 47: master_pb.Seaweed.VolumeList:input_type -> master_pb.VolumeListRequest + 34, // 48: master_pb.Seaweed.LookupEcVolume:input_type -> master_pb.LookupEcVolumeRequest + 36, // 49: master_pb.Seaweed.VacuumVolume:input_type -> master_pb.VacuumVolumeRequest + 38, // 50: master_pb.Seaweed.DisableVacuum:input_type -> master_pb.DisableVacuumRequest + 40, // 51: master_pb.Seaweed.EnableVacuum:input_type -> master_pb.EnableVacuumRequest + 42, // 52: master_pb.Seaweed.VolumeMarkReadonly:input_type -> master_pb.VolumeMarkReadonlyRequest + 44, // 53: master_pb.Seaweed.GetMasterConfiguration:input_type -> master_pb.GetMasterConfigurationRequest + 46, // 54: master_pb.Seaweed.ListClusterNodes:input_type -> master_pb.ListClusterNodesRequest + 48, // 55: master_pb.Seaweed.LeaseAdminToken:input_type -> master_pb.LeaseAdminTokenRequest + 50, // 56: master_pb.Seaweed.ReleaseAdminToken:input_type -> master_pb.ReleaseAdminTokenRequest + 52, // 57: master_pb.Seaweed.Ping:input_type -> master_pb.PingRequest + 58, // 58: master_pb.Seaweed.RaftListClusterServers:input_type -> master_pb.RaftListClusterServersRequest + 54, // 59: master_pb.Seaweed.RaftAddServer:input_type -> master_pb.RaftAddServerRequest + 56, // 60: master_pb.Seaweed.RaftRemoveServer:input_type -> master_pb.RaftRemoveServerRequest + 60, // 61: master_pb.Seaweed.RaftLeadershipTransfer:input_type -> master_pb.RaftLeadershipTransferRequest + 18, // 62: master_pb.Seaweed.VolumeGrow:input_type -> master_pb.VolumeGrowRequest + 2, // 63: master_pb.Seaweed.SendHeartbeat:output_type -> master_pb.HeartbeatResponse + 12, // 64: master_pb.Seaweed.KeepConnected:output_type -> master_pb.KeepConnectedResponse + 15, // 65: master_pb.Seaweed.LookupVolume:output_type -> master_pb.LookupVolumeResponse + 19, // 66: master_pb.Seaweed.Assign:output_type -> master_pb.AssignResponse + 19, // 67: master_pb.Seaweed.StreamAssign:output_type -> master_pb.AssignResponse + 21, // 68: master_pb.Seaweed.Statistics:output_type -> master_pb.StatisticsResponse + 24, // 69: master_pb.Seaweed.CollectionList:output_type -> master_pb.CollectionListResponse + 26, // 70: master_pb.Seaweed.CollectionDelete:output_type -> master_pb.CollectionDeleteResponse + 33, // 71: master_pb.Seaweed.VolumeList:output_type -> master_pb.VolumeListResponse + 35, // 72: master_pb.Seaweed.LookupEcVolume:output_type -> master_pb.LookupEcVolumeResponse + 37, // 73: master_pb.Seaweed.VacuumVolume:output_type -> master_pb.VacuumVolumeResponse + 39, // 74: master_pb.Seaweed.DisableVacuum:output_type -> master_pb.DisableVacuumResponse + 41, // 75: master_pb.Seaweed.EnableVacuum:output_type -> master_pb.EnableVacuumResponse + 43, // 76: master_pb.Seaweed.VolumeMarkReadonly:output_type -> master_pb.VolumeMarkReadonlyResponse + 45, // 77: master_pb.Seaweed.GetMasterConfiguration:output_type -> master_pb.GetMasterConfigurationResponse + 47, // 78: master_pb.Seaweed.ListClusterNodes:output_type -> master_pb.ListClusterNodesResponse + 49, // 79: master_pb.Seaweed.LeaseAdminToken:output_type -> master_pb.LeaseAdminTokenResponse + 51, // 80: master_pb.Seaweed.ReleaseAdminToken:output_type -> master_pb.ReleaseAdminTokenResponse + 53, // 81: master_pb.Seaweed.Ping:output_type -> master_pb.PingResponse + 59, // 82: master_pb.Seaweed.RaftListClusterServers:output_type -> master_pb.RaftListClusterServersResponse + 55, // 83: master_pb.Seaweed.RaftAddServer:output_type -> master_pb.RaftAddServerResponse + 57, // 84: master_pb.Seaweed.RaftRemoveServer:output_type -> master_pb.RaftRemoveServerResponse + 61, // 85: master_pb.Seaweed.RaftLeadershipTransfer:output_type -> master_pb.RaftLeadershipTransferResponse + 62, // 86: master_pb.Seaweed.VolumeGrow:output_type -> master_pb.VolumeGrowResponse + 63, // [63:87] is the sub-list for method output_type + 39, // [39:63] is the sub-list for method input_type + 39, // [39:39] is the sub-list for extension type_name + 39, // [39:39] is the sub-list for extension extendee + 0, // [0:39] is the sub-list for field type_name } func init() { file_master_proto_init() } @@ -4863,7 +4943,7 @@ func file_master_proto_init() { GoPackagePath: reflect.TypeOf(x{}).PkgPath(), RawDescriptor: unsafe.Slice(unsafe.StringData(file_master_proto_rawDesc), len(file_master_proto_rawDesc)), NumEnums: 0, - NumMessages: 73, + NumMessages: 74, NumExtensions: 0, NumServices: 1, }, diff --git a/weed/server/filer_grpc_server_dlm.go b/weed/server/filer_grpc_server_dlm.go index 0dc744c48..3ff9b5f44 100644 --- a/weed/server/filer_grpc_server_dlm.go +++ b/weed/server/filer_grpc_server_dlm.go @@ -6,7 +6,6 @@ import ( "strings" "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" @@ -24,7 +23,9 @@ func (fs *FilerServer) DistributedLock(ctx context.Context, req *filer_pb.LockRe var movedTo pb.ServerAddress expiredAtNs := time.Now().Add(time.Duration(req.SecondsToLock) * time.Second).UnixNano() - resp.LockOwner, resp.RenewToken, movedTo, err = fs.filer.Dlm.LockWithTimeout(req.Name, expiredAtNs, req.RenewToken, req.Owner) + var generation int64 + resp.LockOwner, resp.RenewToken, generation, movedTo, err = fs.filer.Dlm.LockWithTimeout(req.Name, expiredAtNs, req.RenewToken, req.Owner) + resp.Generation = generation glog.V(4).Infof("FILER LOCK: LockWithTimeout result - name=%s lockOwner=%s renewToken=%s movedTo=%s err=%v", req.Name, resp.LockOwner, resp.RenewToken, movedTo, err) glog.V(4).Infof("lock %s %v %v %v, isMoved=%v %v", req.Name, req.SecondsToLock, req.RenewToken, req.Owner, req.IsMoved, movedTo) @@ -42,6 +43,7 @@ func (fs *FilerServer) DistributedLock(ctx context.Context, req *filer_pb.LockRe resp.RenewToken = secondResp.RenewToken resp.LockOwner = secondResp.LockOwner resp.Error = secondResp.Error + resp.Generation = secondResp.Generation glog.V(0).Infof("FILER LOCK: Forwarded lock acquired - name=%s renewToken=%s", req.Name, resp.RenewToken) } else { glog.V(0).Infof("FILER LOCK: Forward failed - name=%s err=%v", req.Name, err) @@ -99,7 +101,7 @@ func (fs *FilerServer) DistributedUnlock(ctx context.Context, req *filer_pb.Unlo func (fs *FilerServer) FindLockOwner(ctx context.Context, req *filer_pb.FindLockOwnerRequest) (*filer_pb.FindLockOwnerResponse, error) { owner, movedTo, err := fs.filer.Dlm.FindLockOwner(req.Name) - if !req.IsMoved && movedTo != "" || err == lock_manager.LockNotFound { + if !req.IsMoved && movedTo != "" { err = pb.WithFilerClient(false, 0, movedTo, fs.grpcDialOption, func(client filer_pb.SeaweedFilerClient) error { secondResp, err := client.FindLockOwner(ctx, &filer_pb.FindLockOwnerRequest{ Name: req.Name, @@ -129,45 +131,133 @@ func (fs *FilerServer) FindLockOwner(ctx context.Context, req *filer_pb.FindLock }, nil } -// TransferLocks is a grpc handler to handle FilerServer's TransferLocksRequest +// TransferLocks is a grpc handler to handle FilerServer's TransferLocksRequest. +// Preserves generation/seq and re-replicates to this node's backup. func (fs *FilerServer) TransferLocks(ctx context.Context, req *filer_pb.TransferLocksRequest) (*filer_pb.TransferLocksResponse, error) { for _, lock := range req.Locks { - fs.filer.Dlm.InsertLock(lock.Name, lock.ExpiredAtNs, lock.RenewToken, lock.Owner) + fs.filer.Dlm.InsertLock(lock.Name, lock.ExpiredAtNs, lock.RenewToken, lock.Owner, lock.Generation, lock.Seq) } return &filer_pb.TransferLocksResponse{}, nil } +// ReplicateLock handles lock replication from a primary to this backup node. +// Uses seq for causal ordering — rejects stale mutations. +func (fs *FilerServer) ReplicateLock(ctx context.Context, req *filer_pb.ReplicateLockRequest) (*filer_pb.ReplicateLockResponse, error) { + if req.IsUnlock { + fs.filer.Dlm.RemoveBackupLockIfSeq(req.Name, req.Generation, req.Seq) + glog.V(4).Infof("FILER REPLICATE: removed backup lock %s generation=%d seq=%d", req.Name, req.Generation, req.Seq) + } else { + fs.filer.Dlm.InsertBackupLock(req.Name, req.ExpiredAtNs, req.RenewToken, req.Owner, req.Generation, req.Seq) + glog.V(4).Infof("FILER REPLICATE: inserted backup lock %s owner=%s generation=%d seq=%d", req.Name, req.Owner, req.Generation, req.Seq) + } + return &filer_pb.ReplicateLockResponse{}, nil +} + +// OnDlmChangeSnapshot is called when the lock ring topology changes. +// It handles: +// 1. Promoting backup locks to primary when this node becomes the new primary +// 2. Re-replicating to the new backup +// 3. Transferring locks that no longer belong here func (fs *FilerServer) OnDlmChangeSnapshot(snapshot []pb.ServerAddress) { - locks := fs.filer.Dlm.SelectNotOwnedLocks(snapshot) + locks := fs.filer.Dlm.AllLocks() if len(locks) == 0 { return } for _, lock := range locks { - server := fs.filer.Dlm.CalculateTargetServer(lock.Key, snapshot) - // Use a context with timeout for lock transfer to avoid hanging indefinitely - ctx, cancel := context.WithTimeout(context.Background(), 5*time.Second) - err := pb.WithFilerClient(false, 0, server, fs.grpcDialOption, func(client filer_pb.SeaweedFilerClient) error { - _, err := client.TransferLocks(ctx, &filer_pb.TransferLocksRequest{ - Locks: []*filer_pb.Lock{ - { - Name: lock.Key, - RenewToken: lock.Token, - ExpiredAtNs: lock.ExpiredAtNs, - Owner: lock.Owner, + primary, backup := fs.filer.Dlm.LockRing.GetPrimaryAndBackup(lock.Key) + + if primary == fs.option.Host { + if lock.IsBackup { + // We held this as backup, now we're primary → promote + fs.filer.Dlm.PromoteLock(lock.Key) + glog.V(0).Infof("DLM: promoted backup lock %s to primary", lock.Key) + } + // Replicate to (possibly new) backup + if backup != "" && fs.filer.Dlm.ReplicateFn != nil { + go fs.filer.Dlm.ReplicateFn(backup, lock.Key, lock.ExpiredAtNs, lock.Token, lock.Owner, lock.Generation, lock.Seq, false) + } + } else if backup == fs.option.Host { + if !lock.IsBackup { + // We were primary, now we're backup → transfer to new primary first, then demote + glog.V(0).Infof("DLM: transferring primary lock %s to new primary=%s", lock.Key, primary) + ctx, cancel := context.WithTimeout(context.Background(), 5*time.Second) + err := pb.WithFilerClient(false, 0, primary, fs.grpcDialOption, func(client filer_pb.SeaweedFilerClient) error { + _, err := client.TransferLocks(ctx, &filer_pb.TransferLocksRequest{ + Locks: []*filer_pb.Lock{ + { + Name: lock.Key, + RenewToken: lock.Token, + ExpiredAtNs: lock.ExpiredAtNs, + Owner: lock.Owner, + Generation: lock.Generation, + Seq: lock.Seq, + }, + }, + }) + return err + }) + cancel() + if err != nil { + glog.Errorf("DLM: failed to transfer lock %s to new primary %s: %v, keeping as primary", lock.Key, primary, err) + } else { + // Only demote after successful handoff + fs.filer.Dlm.DemoteLock(lock.Key) + glog.V(0).Infof("DLM: demoted lock %s to backup after successful transfer", lock.Key) + } + } + // else: already backup, keep as is + } else { + // We're neither primary nor backup → transfer to primary and remove locally + ctx, cancel := context.WithTimeout(context.Background(), 5*time.Second) + err := pb.WithFilerClient(false, 0, primary, fs.grpcDialOption, func(client filer_pb.SeaweedFilerClient) error { + _, err := client.TransferLocks(ctx, &filer_pb.TransferLocksRequest{ + Locks: []*filer_pb.Lock{ + { + Name: lock.Key, + RenewToken: lock.Token, + ExpiredAtNs: lock.ExpiredAtNs, + Owner: lock.Owner, + Generation: lock.Generation, + Seq: lock.Seq, + }, }, - }, + }) + return err + }) + cancel() + if err != nil { + glog.Errorf("DLM: failed to transfer lock %s to %s: %v", lock.Key, primary, err) + } else { + fs.filer.Dlm.RemoveBackupLock(lock.Key) + } + } + } +} + +// SetupDlmReplication sets up the replication callback for the DLM. +// Called during filer server initialization. +func (fs *FilerServer) SetupDlmReplication() { + fs.filer.Dlm.ReplicateFn = func(server pb.ServerAddress, key string, expiredAtNs int64, token string, owner string, generation int64, seq int64, isUnlock bool) { + ctx, cancel := context.WithTimeout(context.Background(), 3*time.Second) + defer cancel() + err := pb.WithFilerClient(false, 0, server, fs.grpcDialOption, func(client filer_pb.SeaweedFilerClient) error { + _, err := client.ReplicateLock(ctx, &filer_pb.ReplicateLockRequest{ + Name: key, + RenewToken: token, + ExpiredAtNs: expiredAtNs, + Owner: owner, + Generation: generation, + Seq: seq, + IsUnlock: isUnlock, }) return err }) - cancel() if err != nil { - // it may not be worth retrying, since the lock may have expired - glog.Errorf("transfer lock %v to %v: %v", lock.Key, server, err) + glog.Warningf("DLM: failed to replicate lock %s to %s: %v", key, server, err) } } - } diff --git a/weed/server/filer_grpc_server_dlm_test.go b/weed/server/filer_grpc_server_dlm_test.go new file mode 100644 index 000000000..5713a9959 --- /dev/null +++ b/weed/server/filer_grpc_server_dlm_test.go @@ -0,0 +1,36 @@ +package weed_server + +import ( + "context" + "testing" + "time" + + "github.com/seaweedfs/seaweedfs/weed/cluster/lock_manager" + "github.com/seaweedfs/seaweedfs/weed/filer" + "github.com/seaweedfs/seaweedfs/weed/pb" + "github.com/seaweedfs/seaweedfs/weed/pb/filer_pb" + "github.com/stretchr/testify/require" + "google.golang.org/grpc/codes" + "google.golang.org/grpc/status" +) + +func TestFindLockOwnerExpiredLockReturnsNotFound(t *testing.T) { + fs := &FilerServer{ + option: &FilerOption{Host: "filer1:8888"}, + filer: &filer.Filer{ + Dlm: lock_manager.NewDistributedLockManager("filer1:8888"), + }, + } + fs.filer.Dlm.LockRing.SetSnapshot([]pb.ServerAddress{"filer1:8888"}, 0) + fs.filer.Dlm.InsertLock("expired-lock", time.Now().Add(-time.Second).UnixNano(), "token1", "owner1", 5, 2) + + resp, err := fs.FindLockOwner(context.Background(), &filer_pb.FindLockOwnerRequest{ + Name: "expired-lock", + }) + require.Nil(t, resp) + require.Error(t, err) + + st, ok := status.FromError(err) + require.True(t, ok) + require.Equal(t, codes.NotFound, st.Code()) +} diff --git a/weed/server/filer_server.go b/weed/server/filer_server.go index e637b2c0d..809c2737e 100644 --- a/weed/server/filer_server.go +++ b/weed/server/filer_server.go @@ -258,6 +258,7 @@ func NewFilerServer(defaultMux, readonlyMux *http.ServeMux, option *FilerOption) fs.filer.Shutdown() }) + fs.SetupDlmReplication() fs.filer.Dlm.LockRing.SetTakeSnapshotCallback(fs.OnDlmChangeSnapshot) if fs.CredentialManager != nil { diff --git a/weed/server/master_grpc_server.go b/weed/server/master_grpc_server.go index 1e355d294..918fb31b7 100644 --- a/weed/server/master_grpc_server.go +++ b/weed/server/master_grpc_server.go @@ -296,11 +296,17 @@ func (ms *MasterServer) KeepConnected(stream master_pb.Seaweed_KeepConnectedServ glog.V(1).Infof("Cluster: %s node %s added to group '%s'", req.ClientType, peerAddress, req.FilerGroup) ms.broadcastToClients(update) } + if req.ClientType == cluster.FilerType { + ms.LockRingManager.AddServer(cluster.FilerGroupName(req.FilerGroup), peerAddress) + } defer func() { for _, update := range ms.Cluster.RemoveClusterNode(req.FilerGroup, req.ClientType, peerAddress) { ms.broadcastToClients(update) } + if req.ClientType == cluster.FilerType { + ms.LockRingManager.RemoveServer(cluster.FilerGroupName(req.FilerGroup), peerAddress) + } ms.deleteClient(clientName) }() @@ -329,6 +335,12 @@ func (ms *MasterServer) KeepConnected(stream master_pb.Seaweed_KeepConnectedServ } } + if initialLockRingUpdate := ms.initialLockRingUpdate(req.ClientType, req.FilerGroup); initialLockRingUpdate != nil { + if sendErr := stream.Send(initialLockRingUpdate); sendErr != nil { + return sendErr + } + } + go func() { for { _, err := stream.Recv() @@ -371,6 +383,21 @@ func (ms *MasterServer) KeepConnected(stream master_pb.Seaweed_KeepConnectedServ } +func (ms *MasterServer) initialLockRingUpdate(clientType string, filerGroup string) *master_pb.KeepConnectedResponse { + if clientType != cluster.FilerType || ms.LockRingManager == nil { + return nil + } + + update := ms.LockRingManager.GetLastUpdate(cluster.FilerGroupName(filerGroup)) + if update == nil { + return nil + } + + return &master_pb.KeepConnectedResponse{ + LockRingUpdate: update, + } +} + func (ms *MasterServer) broadcastToClients(message *master_pb.KeepConnectedResponse) { ms.clientChansLock.RLock() for client, ch := range ms.clientChans { diff --git a/weed/server/master_grpc_server_test.go b/weed/server/master_grpc_server_test.go new file mode 100644 index 000000000..4c085436f --- /dev/null +++ b/weed/server/master_grpc_server_test.go @@ -0,0 +1,37 @@ +package weed_server + +import ( + "testing" + + "github.com/seaweedfs/seaweedfs/weed/cluster" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" +) + +func TestInitialLockRingUpdateReturnsLastBroadcastForFilers(t *testing.T) { + ms := &MasterServer{ + LockRingManager: cluster.NewLockRingManager(nil), + } + + ms.LockRingManager.AddServer("group-a", "filer1:8888") + ms.LockRingManager.AddServer("group-a", "filer2:8888") + ms.LockRingManager.FlushPending("group-a") + + resp := ms.initialLockRingUpdate(cluster.FilerType, "group-a") + require.NotNil(t, resp) + require.NotNil(t, resp.LockRingUpdate) + assert.Equal(t, "group-a", resp.LockRingUpdate.FilerGroup) + assert.ElementsMatch(t, []string{"filer1:8888", "filer2:8888"}, resp.LockRingUpdate.Servers) + assert.Greater(t, resp.LockRingUpdate.Version, int64(0)) +} + +func TestInitialLockRingUpdateSkipsNonFilers(t *testing.T) { + ms := &MasterServer{ + LockRingManager: cluster.NewLockRingManager(nil), + } + + ms.LockRingManager.AddServer("group-a", "filer1:8888") + ms.LockRingManager.FlushPending("group-a") + + assert.Nil(t, ms.initialLockRingUpdate(cluster.BrokerType, "group-a")) +} diff --git a/weed/server/master_server.go b/weed/server/master_server.go index f403ff771..fd85976a9 100644 --- a/weed/server/master_server.go +++ b/weed/server/master_server.go @@ -88,6 +88,8 @@ type MasterServer struct { Cluster *cluster.Cluster + LockRingManager *cluster.LockRingManager + // telemetry telemetryCollector *telemetry.Collector } @@ -138,6 +140,8 @@ func NewMasterServer(r *mux.Router, option *MasterOption, peers map[string]pb.Se Cluster: cluster.NewCluster(), } + ms.LockRingManager = cluster.NewLockRingManager(ms.broadcastToClients) + ms.MasterClient.SetOnPeerUpdateFn(ms.OnPeerUpdate) seq := ms.createSequencer(option) diff --git a/weed/wdclient/masterclient.go b/weed/wdclient/masterclient.go index 9f705bbcc..5fe854aa8 100644 --- a/weed/wdclient/masterclient.go +++ b/weed/wdclient/masterclient.go @@ -151,8 +151,10 @@ type MasterClient struct { 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 + 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 { @@ -181,6 +183,12 @@ func (mc *MasterClient) SetOnPeerUpdateFn(onPeerUpdate func(update *master_pb.Cl 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{}) @@ -311,6 +319,17 @@ func (mc *MasterClient) tryConnectToMaster(ctx context.Context, master pb.Server } 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)