dlm: resilient distributed locks via consistent hashing + backup replication (#8860)
* dlm: replace modulo hashing with consistent hash ring Introduce HashRing with virtual nodes (CRC32-based consistent hashing) to replace the modulo-based hashKeyToServer. When a filer node is removed, only keys that hashed to that node are remapped to the next server on the ring, leaving all other mappings stable. This is the foundation for backup replication — the successor on the ring is always the natural takeover node. * dlm: add Generation and IsBackup fields to Lock Lock now carries IsBackup (whether this node holds the lock as a backup replica) and Generation (a monotonic fencing token that increments on each fresh acquisition, stays the same on renewal). Add helper methods: AllLocks, PromoteLock, DemoteLock, InsertBackupLock, RemoveLock, GetLock. * dlm: add ReplicateLock RPC and generation/is_backup proto fields Add generation field to LockResponse for fencing tokens. Add generation and is_backup fields to Lock message. Add ReplicateLock RPC for primary-to-backup lock replication. Add ReplicateLockRequest/ReplicateLockResponse messages. * dlm: add async backup replication to DistributedLockManager Route lock/unlock via consistent hash ring's GetPrimaryAndBackup(). After a successful lock or unlock on the primary, asynchronously replicate the operation to the backup server via ReplicateFunc callback. Single-server deployments skip replication. * dlm: add ReplicateLock handler and backup-aware topology changes Add ReplicateLock gRPC handler for primary-to-backup replication. Revise OnDlmChangeSnapshot to handle three cases on topology change: - Promote backup locks when this node becomes primary - Demote primary locks when this node becomes backup - Transfer locks when this node is neither primary nor backup Wire up SetupDlmReplication during filer server initialization. * dlm: expose generation fencing token in lock client LiveLock now captures the generation from LockResponse and exposes it via Generation() method. Consumers can use this as a fencing token to detect stale lock holders. * dlm: update empty folder cleaner to use consistent hash ring Replace local modulo-based hashKeyToServer with LockRing.GetPrimary() which uses the shared consistent hash ring for folder ownership. * dlm: add unit tests for consistent hash ring Test basic operations, consistency on server removal (only keys from removed server move), backup-is-successor property (backup becomes new primary when primary is removed), and key distribution balance. * dlm: add integration tests for lock replication failure scenarios Test cases: - Primary crash with backup promotion (backup has valid token) - Backup crash with primary continuing - Both primary and backup crash (lock lost, re-acquirable) - Rolling restart across all nodes - Generation fencing token increments on new acquisition - Replication failure (primary still works independently) - Unlock replicates deletion to backup - Lock survives server addition (topology change) - Consistent hashing minimal disruption (only removed server's keys move) * dlm: address PR review findings 1. Causal replication ordering: Add per-lock sequence number (Seq) that increments on every mutation. Backup rejects incoming mutations with seq <= current seq, preventing stale async replications from overwriting newer state. Unlock replication also carries seq and is rejected if stale. 2. Demote-after-handoff: OnDlmChangeSnapshot now transfers the lock to the new primary first and only demotes to backup after a successful TransferLocks RPC. If the transfer fails, the lock stays as primary on this node. 3. SetSnapshot candidateServers leak: Replace the candidateServers map entirely instead of appending, so removed servers don't linger. 4. TransferLocks preserves Generation and Seq: InsertLock now accepts generation and seq parameters. After accepting a transferred lock, the receiving node re-replicates to its backup. 5. Rolling restart test: Add re-replication step after promotion and assert survivedCount > 0. Add TestDLM_StaleReplicationRejected. 6. Mixed-version upgrade note: Add comment on HashRing documenting that all filer nodes must be upgraded together. * dlm: serve renewals locally during transfer window on node join When a new node joins and steals hash ranges from surviving nodes, there's a window between ring update and lock transfer where the client gets redirected to a node that doesn't have the lock yet. Fix: if the ring says primary != self but we still hold the lock locally (non-backup, matching token), serve the renewal/unlock here rather than redirecting. The lock will be transferred by OnDlmChangeSnapshot, and subsequent requests will go to the new primary once the transfer completes. Add tests: - TestDLM_NodeDropAndJoin_OwnershipDisruption: measures disruption when a node drops and a new one joins (14/100 surviving-node locks disrupted, all handled by transfer logic) - TestDLM_RenewalDuringTransferWindow: verifies renewal succeeds on old primary during the transfer window * dlm: master-managed lock ring with stabilization batching The master now owns the lock ring membership. Instead of filers independently reacting to individual ClusterNodeUpdate add/remove events, the master: 1. Tracks filer membership in LockRingManager 2. Batches rapid changes with a 1-second stabilization timer (e.g., a node drop + join within 1 second → single ring update) 3. Broadcasts the complete ring snapshot atomically via the new LockRingUpdate message in KeepConnectedResponse Filers receive the ring as a complete snapshot and apply it via SetSnapshot, ensuring all filers converge to the same ring state without intermediate churn. This eliminates the double-churn problem where a rapid drop+join would fire two separate ring mutations, each triggering lock transfers and disrupting ownership on surviving nodes. * dlm: track ring version, reject stale updates, remove dead code SetSnapshot now takes a version parameter from the master. Stale updates (version < current) are rejected, preventing reordered messages from overwriting a newer ring state. Version 0 is always accepted for bootstrap. Remove AddServer/RemoveServer from LockRing — the ring is now exclusively managed by the master via SetSnapshot. Remove the candidateServers map that was only used by those methods. * dlm: fix SelectLocks data race, advance generation on backup insert - SelectLocks: change RLock to Lock since the function deletes map entries, which is a write operation and causes a data race under RLock. - InsertBackupLock: advance nextGeneration to at least the incoming generation so that after failover promotion, new lock acquisitions get a generation strictly greater than any replicated lock. - Bump replication failure log from V(1) to Warningf for production visibility. * dlm: fix SetSnapshot race, test reliability, timer edge cases - SetSnapshot: hold LockRing lock through both version update and Ring.SetServers() so they're atomic. Prevents a concurrent caller from seeing the new version but applying stale servers. - Transfer window test: search for a key that actually moves primary when filer4 joins, instead of relying on a fixed key that may not. - renewLock redirect: pass the existing token to the new primary instead of empty string, so redirected renewals work correctly. - scheduleBroadcast: check timer.Stop() return value. If the timer already fired, the callback picks up latest state. - FlushPending: only broadcast if timer.Stop() returns true (timer was still pending). If false, the callback is already running. - Fix test comment: "idempotent" → "accepted, state-changing". * dlm: use wall-clock nanoseconds for lock ring version The lock ring version was an in-memory counter that reset to 0 on master restart. A filer that had seen version 5 would reject version 1 from the restarted master. Fix: use time.Now().UnixNano() as the version. This survives master restarts without persistence — the restarted master produces a version greater than any pre-restart value. * dlm: treat expired lock owners as missing Co-authored-by: Copilot <223556219+Copilot@users.noreply.github.com> * dlm: reject stale lock transfers Co-authored-by: Copilot <223556219+Copilot@users.noreply.github.com> * dlm: order replication by generation Co-authored-by: Copilot <223556219+Copilot@users.noreply.github.com> * dlm: bootstrap lock ring on reconnect Co-authored-by: Copilot <223556219+Copilot@users.noreply.github.com> --------- Co-authored-by: Copilot <223556219+Copilot@users.noreply.github.com>
This commit is contained in:
@@ -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.
|
||||
|
||||
@@ -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,
|
||||
},
|
||||
|
||||
@@ -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{
|
||||
{
|
||||
|
||||
@@ -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 {
|
||||
|
||||
File diff suppressed because it is too large
Load Diff
Reference in New Issue
Block a user