diff --git a/pkg/internal/client/batch.go b/pkg/internal/client/batch.go index c851af32563c..01111793ca9d 100644 --- a/pkg/internal/client/batch.go +++ b/pkg/internal/client/batch.go @@ -243,14 +243,15 @@ func (b *Batch) fillResults(ctx context.Context) { reply, args) } - // Nothing to do for all methods below as they do not generate - // any rows. + // Nothing to do for all methods below as they do not generate + // any rows. case *roachpb.BeginTransactionRequest: case *roachpb.EndTransactionRequest: case *roachpb.AdminMergeRequest: case *roachpb.AdminSplitRequest: case *roachpb.AdminTransferLeaseRequest: case *roachpb.AdminChangeReplicasRequest: + case *roachpb.AdminRelocateRangeRequest: case *roachpb.HeartbeatTxnRequest: case *roachpb.GCRequest: case *roachpb.LeaseInfoRequest: @@ -636,6 +637,24 @@ func (b *Batch) adminChangeReplicas( b.initResult(1, 0, notRaw, nil) } +// adminRelocateRange is only exported on DB. It is here for symmetry with the +// other operations. +func (b *Batch) adminRelocateRange(key interface{}, targets []roachpb.ReplicationTarget) { + k, err := marshalKey(key) + if err != nil { + b.initResult(0, 0, notRaw, err) + return + } + req := &roachpb.AdminRelocateRangeRequest{ + RequestHeader: roachpb.RequestHeader{ + Key: k, + }, + Targets: targets, + } + b.appendReqs(req) + b.initResult(1, 0, notRaw, nil) +} + // writeBatch is only exported on DB. func (b *Batch) writeBatch(s, e interface{}, data []byte) { begin, err := marshalKey(s) diff --git a/pkg/internal/client/db.go b/pkg/internal/client/db.go index ae5ce2b69bbf..b812fabf4cb8 100644 --- a/pkg/internal/client/db.go +++ b/pkg/internal/client/db.go @@ -506,6 +506,16 @@ func (db *DB) AdminChangeReplicas( return getOneErr(db.Run(ctx, b), b) } +// AdminRelocateRange relocates the replicas for a range onto the specified +// list of stores. +func (db *DB) AdminRelocateRange( + ctx context.Context, key interface{}, targets []roachpb.ReplicationTarget, +) error { + b := &Batch{} + b.adminRelocateRange(key, targets) + return getOneErr(db.Run(ctx, b), b) +} + // WriteBatch applies the operations encoded in a BatchRepr, which is the // serialized form of a RocksDB Batch. The command cannot span Ranges and must // be run on an empty keyrange. diff --git a/pkg/roachpb/api.go b/pkg/roachpb/api.go index dc1b97ee6e27..e7853a01e6c4 100644 --- a/pkg/roachpb/api.go +++ b/pkg/roachpb/api.go @@ -509,6 +509,9 @@ func (*AdminTransferLeaseRequest) Method() Method { return AdminTransferLease } // Method implements the Request interface. func (*AdminChangeReplicasRequest) Method() Method { return AdminChangeReplicas } +// Method implements the Request interface. +func (*AdminRelocateRangeRequest) Method() Method { return AdminRelocateRange } + // Method implements the Request interface. func (*HeartbeatTxnRequest) Method() Method { return HeartbeatTxn } @@ -680,6 +683,12 @@ func (acrr *AdminChangeReplicasRequest) ShallowCopy() Request { return &shallowCopy } +// ShallowCopy implements the Request interface. +func (acrr *AdminRelocateRangeRequest) ShallowCopy() Request { + shallowCopy := *acrr + return &shallowCopy +} + // ShallowCopy implements the Request interface. func (htr *HeartbeatTxnRequest) ShallowCopy() Request { shallowCopy := *htr @@ -1020,6 +1029,7 @@ func (*AdminSplitRequest) flags() int { return isAdmin | isAlone } func (*AdminMergeRequest) flags() int { return isAdmin | isAlone } func (*AdminTransferLeaseRequest) flags() int { return isAdmin | isAlone } func (*AdminChangeReplicasRequest) flags() int { return isAdmin | isAlone } +func (*AdminRelocateRangeRequest) flags() int { return isAdmin | isAlone } func (*HeartbeatTxnRequest) flags() int { return isWrite | isTxn } func (*GCRequest) flags() int { return isWrite | isRange } func (*PushTxnRequest) flags() int { return isWrite | isAlone } diff --git a/pkg/roachpb/api.pb.go b/pkg/roachpb/api.pb.go index b8a662e3f8dc..3ad60a5da431 100644 --- a/pkg/roachpb/api.pb.go +++ b/pkg/roachpb/api.pb.go @@ -55,6 +55,8 @@ AdminTransferLeaseResponse AdminChangeReplicasRequest AdminChangeReplicasResponse + AdminRelocateRangeRequest + AdminRelocateRangeResponse HeartbeatTxnRequest HeartbeatTxnResponse GCRequest @@ -434,7 +436,7 @@ func (x QueryIntentRequest_IfMissingBehavior) String() string { return proto.EnumName(QueryIntentRequest_IfMissingBehavior_name, int32(x)) } func (QueryIntentRequest_IfMissingBehavior) EnumDescriptor() ([]byte, []int) { - return fileDescriptorApi, []int{48, 0} + return fileDescriptorApi, []int{50, 0} } // RangeInfo describes a range which executed a request. It contains @@ -1128,6 +1130,28 @@ func (m *AdminChangeReplicasResponse) String() string { return proto. func (*AdminChangeReplicasResponse) ProtoMessage() {} func (*AdminChangeReplicasResponse) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{39} } +// An AdminRelocateRangeRequest is the argument to the AdminRelocateRange() +// method. Relocates the replicas for a range to the specified target stores. +// The first store in the list of targets becomes the new leaseholder. +type AdminRelocateRangeRequest struct { + RequestHeader `protobuf:"bytes,1,opt,name=header,embedded=header" json:"header"` + Targets []ReplicationTarget `protobuf:"bytes,2,rep,name=targets" json:"targets"` +} + +func (m *AdminRelocateRangeRequest) Reset() { *m = AdminRelocateRangeRequest{} } +func (m *AdminRelocateRangeRequest) String() string { return proto.CompactTextString(m) } +func (*AdminRelocateRangeRequest) ProtoMessage() {} +func (*AdminRelocateRangeRequest) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{40} } + +type AdminRelocateRangeResponse struct { + ResponseHeader `protobuf:"bytes,1,opt,name=header,embedded=header" json:"header"` +} + +func (m *AdminRelocateRangeResponse) Reset() { *m = AdminRelocateRangeResponse{} } +func (m *AdminRelocateRangeResponse) String() string { return proto.CompactTextString(m) } +func (*AdminRelocateRangeResponse) ProtoMessage() {} +func (*AdminRelocateRangeResponse) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{41} } + // A HeartbeatTxnRequest is arguments to the HeartbeatTxn() // method. It's sent by transaction coordinators to let the system // know that the transaction is still ongoing. Note that this @@ -1141,7 +1165,7 @@ type HeartbeatTxnRequest struct { func (m *HeartbeatTxnRequest) Reset() { *m = HeartbeatTxnRequest{} } func (m *HeartbeatTxnRequest) String() string { return proto.CompactTextString(m) } func (*HeartbeatTxnRequest) ProtoMessage() {} -func (*HeartbeatTxnRequest) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{40} } +func (*HeartbeatTxnRequest) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{42} } // A HeartbeatTxnResponse is the return value from the HeartbeatTxn() // method. It returns the transaction info in the response header. The @@ -1154,7 +1178,7 @@ type HeartbeatTxnResponse struct { func (m *HeartbeatTxnResponse) Reset() { *m = HeartbeatTxnResponse{} } func (m *HeartbeatTxnResponse) String() string { return proto.CompactTextString(m) } func (*HeartbeatTxnResponse) ProtoMessage() {} -func (*HeartbeatTxnResponse) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{41} } +func (*HeartbeatTxnResponse) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{43} } // A GCRequest is arguments to the GC() method. It's sent by range // lease holders after scanning range data to find expired MVCC values. @@ -1171,7 +1195,7 @@ type GCRequest struct { func (m *GCRequest) Reset() { *m = GCRequest{} } func (m *GCRequest) String() string { return proto.CompactTextString(m) } func (*GCRequest) ProtoMessage() {} -func (*GCRequest) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{42} } +func (*GCRequest) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{44} } type GCRequest_GCKey struct { Key Key `protobuf:"bytes,1,opt,name=key,proto3,casttype=Key" json:"key,omitempty"` @@ -1181,7 +1205,7 @@ type GCRequest_GCKey struct { func (m *GCRequest_GCKey) Reset() { *m = GCRequest_GCKey{} } func (m *GCRequest_GCKey) String() string { return proto.CompactTextString(m) } func (*GCRequest_GCKey) ProtoMessage() {} -func (*GCRequest_GCKey) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{42, 0} } +func (*GCRequest_GCKey) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{44, 0} } // A GCResponse is the return value from the GC() method. type GCResponse struct { @@ -1191,7 +1215,7 @@ type GCResponse struct { func (m *GCResponse) Reset() { *m = GCResponse{} } func (m *GCResponse) String() string { return proto.CompactTextString(m) } func (*GCResponse) ProtoMessage() {} -func (*GCResponse) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{43} } +func (*GCResponse) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{45} } // A PushTxnRequest is arguments to the PushTxn() method. It's sent by // readers or writers which have encountered an "intent" laid down by @@ -1241,7 +1265,7 @@ type PushTxnRequest struct { func (m *PushTxnRequest) Reset() { *m = PushTxnRequest{} } func (m *PushTxnRequest) String() string { return proto.CompactTextString(m) } func (*PushTxnRequest) ProtoMessage() {} -func (*PushTxnRequest) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{44} } +func (*PushTxnRequest) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{46} } // A PushTxnResponse is the return value from the PushTxn() method. It // returns success and the resulting state of PusheeTxn if the @@ -1260,7 +1284,7 @@ type PushTxnResponse struct { func (m *PushTxnResponse) Reset() { *m = PushTxnResponse{} } func (m *PushTxnResponse) String() string { return proto.CompactTextString(m) } func (*PushTxnResponse) ProtoMessage() {} -func (*PushTxnResponse) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{45} } +func (*PushTxnResponse) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{47} } // A QueryTxnResponse is arguments to the QueryTxn() method. It's sent // by transactions which are waiting to push another transaction because @@ -1280,7 +1304,7 @@ type QueryTxnRequest struct { func (m *QueryTxnRequest) Reset() { *m = QueryTxnRequest{} } func (m *QueryTxnRequest) String() string { return proto.CompactTextString(m) } func (*QueryTxnRequest) ProtoMessage() {} -func (*QueryTxnRequest) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{46} } +func (*QueryTxnRequest) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{48} } // A QueryTxnResponse is the return value from the QueryTxn() method. type QueryTxnResponse struct { @@ -1295,7 +1319,7 @@ type QueryTxnResponse struct { func (m *QueryTxnResponse) Reset() { *m = QueryTxnResponse{} } func (m *QueryTxnResponse) String() string { return proto.CompactTextString(m) } func (*QueryTxnResponse) ProtoMessage() {} -func (*QueryTxnResponse) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{47} } +func (*QueryTxnResponse) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{49} } // A QueryIntentRequest is arguments to the QueryIntent() method. It visits // the specified key and checks whether an intent is present for the given @@ -1331,7 +1355,7 @@ type QueryIntentRequest struct { func (m *QueryIntentRequest) Reset() { *m = QueryIntentRequest{} } func (m *QueryIntentRequest) String() string { return proto.CompactTextString(m) } func (*QueryIntentRequest) ProtoMessage() {} -func (*QueryIntentRequest) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{48} } +func (*QueryIntentRequest) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{50} } // A QueryIntentResponse is the return value from the QueryIntent() method. type QueryIntentResponse struct { @@ -1343,7 +1367,7 @@ type QueryIntentResponse struct { func (m *QueryIntentResponse) Reset() { *m = QueryIntentResponse{} } func (m *QueryIntentResponse) String() string { return proto.CompactTextString(m) } func (*QueryIntentResponse) ProtoMessage() {} -func (*QueryIntentResponse) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{49} } +func (*QueryIntentResponse) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{51} } // A ResolveIntentRequest is arguments to the ResolveIntent() // method. It is sent by transaction coordinators after success @@ -1363,7 +1387,7 @@ type ResolveIntentRequest struct { func (m *ResolveIntentRequest) Reset() { *m = ResolveIntentRequest{} } func (m *ResolveIntentRequest) String() string { return proto.CompactTextString(m) } func (*ResolveIntentRequest) ProtoMessage() {} -func (*ResolveIntentRequest) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{50} } +func (*ResolveIntentRequest) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{52} } // A ResolveIntentResponse is the return value from the // ResolveIntent() method. @@ -1374,7 +1398,7 @@ type ResolveIntentResponse struct { func (m *ResolveIntentResponse) Reset() { *m = ResolveIntentResponse{} } func (m *ResolveIntentResponse) String() string { return proto.CompactTextString(m) } func (*ResolveIntentResponse) ProtoMessage() {} -func (*ResolveIntentResponse) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{51} } +func (*ResolveIntentResponse) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{53} } // A ResolveIntentRangeRequest is arguments to the ResolveIntentRange() method. // It is sent by transaction coordinators after success calling PushTxn to @@ -1398,7 +1422,7 @@ type ResolveIntentRangeRequest struct { func (m *ResolveIntentRangeRequest) Reset() { *m = ResolveIntentRangeRequest{} } func (m *ResolveIntentRangeRequest) String() string { return proto.CompactTextString(m) } func (*ResolveIntentRangeRequest) ProtoMessage() {} -func (*ResolveIntentRangeRequest) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{52} } +func (*ResolveIntentRangeRequest) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{54} } // A ResolveIntentRangeResponse is the return value from the // ResolveIntent() method. @@ -1409,7 +1433,7 @@ type ResolveIntentRangeResponse struct { func (m *ResolveIntentRangeResponse) Reset() { *m = ResolveIntentRangeResponse{} } func (m *ResolveIntentRangeResponse) String() string { return proto.CompactTextString(m) } func (*ResolveIntentRangeResponse) ProtoMessage() {} -func (*ResolveIntentRangeResponse) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{53} } +func (*ResolveIntentRangeResponse) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{55} } // A MergeRequest contains arguments to the Merge() method. It // specifies a key and a value which should be merged into the @@ -1422,7 +1446,7 @@ type MergeRequest struct { func (m *MergeRequest) Reset() { *m = MergeRequest{} } func (m *MergeRequest) String() string { return proto.CompactTextString(m) } func (*MergeRequest) ProtoMessage() {} -func (*MergeRequest) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{54} } +func (*MergeRequest) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{56} } // MergeResponse is the response to a Merge() operation. type MergeResponse struct { @@ -1432,7 +1456,7 @@ type MergeResponse struct { func (m *MergeResponse) Reset() { *m = MergeResponse{} } func (m *MergeResponse) String() string { return proto.CompactTextString(m) } func (*MergeResponse) ProtoMessage() {} -func (*MergeResponse) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{55} } +func (*MergeResponse) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{57} } // TruncateLogRequest is used to remove a prefix of the raft log. While there // is no requirement for correctness that the raft log truncation be synchronized across @@ -1453,7 +1477,7 @@ type TruncateLogRequest struct { func (m *TruncateLogRequest) Reset() { *m = TruncateLogRequest{} } func (m *TruncateLogRequest) String() string { return proto.CompactTextString(m) } func (*TruncateLogRequest) ProtoMessage() {} -func (*TruncateLogRequest) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{56} } +func (*TruncateLogRequest) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{58} } // TruncateLogResponse is the response to a TruncateLog() operation. type TruncateLogResponse struct { @@ -1463,7 +1487,7 @@ type TruncateLogResponse struct { func (m *TruncateLogResponse) Reset() { *m = TruncateLogResponse{} } func (m *TruncateLogResponse) String() string { return proto.CompactTextString(m) } func (*TruncateLogResponse) ProtoMessage() {} -func (*TruncateLogResponse) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{57} } +func (*TruncateLogResponse) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{59} } // A RequestLeaseRequest is arguments to the RequestLease() // method. It is sent by the store on behalf of one of its ranges upon receipt @@ -1483,7 +1507,7 @@ type RequestLeaseRequest struct { func (m *RequestLeaseRequest) Reset() { *m = RequestLeaseRequest{} } func (m *RequestLeaseRequest) String() string { return proto.CompactTextString(m) } func (*RequestLeaseRequest) ProtoMessage() {} -func (*RequestLeaseRequest) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{58} } +func (*RequestLeaseRequest) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{60} } // A TransferLeaseRequest represents the arguments to the TransferLease() // method. It is sent by a replica that currently holds the range lease and @@ -1508,7 +1532,7 @@ type TransferLeaseRequest struct { func (m *TransferLeaseRequest) Reset() { *m = TransferLeaseRequest{} } func (m *TransferLeaseRequest) String() string { return proto.CompactTextString(m) } func (*TransferLeaseRequest) ProtoMessage() {} -func (*TransferLeaseRequest) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{59} } +func (*TransferLeaseRequest) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{61} } // LeaseInfoRequest is the argument to the LeaseInfo() method, for getting // information about a range's lease. @@ -1521,7 +1545,7 @@ type LeaseInfoRequest struct { func (m *LeaseInfoRequest) Reset() { *m = LeaseInfoRequest{} } func (m *LeaseInfoRequest) String() string { return proto.CompactTextString(m) } func (*LeaseInfoRequest) ProtoMessage() {} -func (*LeaseInfoRequest) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{60} } +func (*LeaseInfoRequest) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{62} } // LeaseInfoResponse is the response to a LeaseInfo() operation. type LeaseInfoResponse struct { @@ -1534,7 +1558,7 @@ type LeaseInfoResponse struct { func (m *LeaseInfoResponse) Reset() { *m = LeaseInfoResponse{} } func (m *LeaseInfoResponse) String() string { return proto.CompactTextString(m) } func (*LeaseInfoResponse) ProtoMessage() {} -func (*LeaseInfoResponse) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{61} } +func (*LeaseInfoResponse) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{63} } // A RequestLeaseResponse is the response to a RequestLease() or TransferLease() // operation. @@ -1545,7 +1569,7 @@ type RequestLeaseResponse struct { func (m *RequestLeaseResponse) Reset() { *m = RequestLeaseResponse{} } func (m *RequestLeaseResponse) String() string { return proto.CompactTextString(m) } func (*RequestLeaseResponse) ProtoMessage() {} -func (*RequestLeaseResponse) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{62} } +func (*RequestLeaseResponse) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{64} } // A ComputeChecksumRequest is arguments to the ComputeChecksum() method, to // start computing the checksum for the specified range at the snapshot for this @@ -1564,7 +1588,7 @@ type ComputeChecksumRequest struct { func (m *ComputeChecksumRequest) Reset() { *m = ComputeChecksumRequest{} } func (m *ComputeChecksumRequest) String() string { return proto.CompactTextString(m) } func (*ComputeChecksumRequest) ProtoMessage() {} -func (*ComputeChecksumRequest) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{63} } +func (*ComputeChecksumRequest) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{65} } // A ComputeChecksumResponse is the response to a ComputeChecksum() operation. type ComputeChecksumResponse struct { @@ -1577,7 +1601,7 @@ type ComputeChecksumResponse struct { func (m *ComputeChecksumResponse) Reset() { *m = ComputeChecksumResponse{} } func (m *ComputeChecksumResponse) String() string { return proto.CompactTextString(m) } func (*ComputeChecksumResponse) ProtoMessage() {} -func (*ComputeChecksumResponse) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{64} } +func (*ComputeChecksumResponse) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{66} } type ExportStorage struct { Provider ExportStorageProvider `protobuf:"varint,1,opt,name=provider,proto3,enum=cockroach.roachpb.ExportStorageProvider" json:"provider,omitempty"` @@ -1591,7 +1615,7 @@ type ExportStorage struct { func (m *ExportStorage) Reset() { *m = ExportStorage{} } func (m *ExportStorage) String() string { return proto.CompactTextString(m) } func (*ExportStorage) ProtoMessage() {} -func (*ExportStorage) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{65} } +func (*ExportStorage) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{67} } type ExportStorage_LocalFilePath struct { Path string `protobuf:"bytes,1,opt,name=path,proto3" json:"path,omitempty"` @@ -1601,7 +1625,7 @@ func (m *ExportStorage_LocalFilePath) Reset() { *m = ExportStorage_Local func (m *ExportStorage_LocalFilePath) String() string { return proto.CompactTextString(m) } func (*ExportStorage_LocalFilePath) ProtoMessage() {} func (*ExportStorage_LocalFilePath) Descriptor() ([]byte, []int) { - return fileDescriptorApi, []int{65, 0} + return fileDescriptorApi, []int{67, 0} } type ExportStorage_Http struct { @@ -1611,7 +1635,7 @@ type ExportStorage_Http struct { func (m *ExportStorage_Http) Reset() { *m = ExportStorage_Http{} } func (m *ExportStorage_Http) String() string { return proto.CompactTextString(m) } func (*ExportStorage_Http) ProtoMessage() {} -func (*ExportStorage_Http) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{65, 1} } +func (*ExportStorage_Http) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{67, 1} } type ExportStorage_S3 struct { Bucket string `protobuf:"bytes,1,opt,name=bucket,proto3" json:"bucket,omitempty"` @@ -1626,7 +1650,7 @@ type ExportStorage_S3 struct { func (m *ExportStorage_S3) Reset() { *m = ExportStorage_S3{} } func (m *ExportStorage_S3) String() string { return proto.CompactTextString(m) } func (*ExportStorage_S3) ProtoMessage() {} -func (*ExportStorage_S3) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{65, 2} } +func (*ExportStorage_S3) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{67, 2} } type ExportStorage_GCS struct { Bucket string `protobuf:"bytes,1,opt,name=bucket,proto3" json:"bucket,omitempty"` @@ -1640,7 +1664,7 @@ type ExportStorage_GCS struct { func (m *ExportStorage_GCS) Reset() { *m = ExportStorage_GCS{} } func (m *ExportStorage_GCS) String() string { return proto.CompactTextString(m) } func (*ExportStorage_GCS) ProtoMessage() {} -func (*ExportStorage_GCS) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{65, 3} } +func (*ExportStorage_GCS) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{67, 3} } type ExportStorage_Azure struct { Container string `protobuf:"bytes,1,opt,name=container,proto3" json:"container,omitempty"` @@ -1652,7 +1676,7 @@ type ExportStorage_Azure struct { func (m *ExportStorage_Azure) Reset() { *m = ExportStorage_Azure{} } func (m *ExportStorage_Azure) String() string { return proto.CompactTextString(m) } func (*ExportStorage_Azure) ProtoMessage() {} -func (*ExportStorage_Azure) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{65, 4} } +func (*ExportStorage_Azure) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{67, 4} } // WriteBatchRequest is arguments to the WriteBatch() method, to apply the // operations encoded in a BatchRepr. @@ -1668,7 +1692,7 @@ type WriteBatchRequest struct { func (m *WriteBatchRequest) Reset() { *m = WriteBatchRequest{} } func (m *WriteBatchRequest) String() string { return proto.CompactTextString(m) } func (*WriteBatchRequest) ProtoMessage() {} -func (*WriteBatchRequest) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{66} } +func (*WriteBatchRequest) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{68} } // WriteBatchResponse is the response to a WriteBatch() operation. type WriteBatchResponse struct { @@ -1678,7 +1702,7 @@ type WriteBatchResponse struct { func (m *WriteBatchResponse) Reset() { *m = WriteBatchResponse{} } func (m *WriteBatchResponse) String() string { return proto.CompactTextString(m) } func (*WriteBatchResponse) ProtoMessage() {} -func (*WriteBatchResponse) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{67} } +func (*WriteBatchResponse) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{69} } // ExportRequest is the argument to the Export() method, to dump a keyrange into // files under a basepath. @@ -1699,7 +1723,7 @@ type ExportRequest struct { func (m *ExportRequest) Reset() { *m = ExportRequest{} } func (m *ExportRequest) String() string { return proto.CompactTextString(m) } func (*ExportRequest) ProtoMessage() {} -func (*ExportRequest) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{68} } +func (*ExportRequest) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{70} } type BulkOpSummary struct { DataSize int64 `protobuf:"varint,1,opt,name=data_size,json=dataSize,proto3" json:"data_size,omitempty"` @@ -1711,7 +1735,7 @@ type BulkOpSummary struct { func (m *BulkOpSummary) Reset() { *m = BulkOpSummary{} } func (m *BulkOpSummary) String() string { return proto.CompactTextString(m) } func (*BulkOpSummary) ProtoMessage() {} -func (*BulkOpSummary) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{69} } +func (*BulkOpSummary) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{71} } // ExportResponse is the response to an Export() operation. type ExportResponse struct { @@ -1723,7 +1747,7 @@ type ExportResponse struct { func (m *ExportResponse) Reset() { *m = ExportResponse{} } func (m *ExportResponse) String() string { return proto.CompactTextString(m) } func (*ExportResponse) ProtoMessage() {} -func (*ExportResponse) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{70} } +func (*ExportResponse) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{72} } // File describes a keyrange that has been dumped to a file at the given // path. @@ -1738,7 +1762,7 @@ type ExportResponse_File struct { func (m *ExportResponse_File) Reset() { *m = ExportResponse_File{} } func (m *ExportResponse_File) String() string { return proto.CompactTextString(m) } func (*ExportResponse_File) ProtoMessage() {} -func (*ExportResponse_File) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{70, 0} } +func (*ExportResponse_File) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{72, 0} } // ImportRequest is the argument to the Import() method, to bulk load key/value // entries. @@ -1764,7 +1788,7 @@ type ImportRequest struct { func (m *ImportRequest) Reset() { *m = ImportRequest{} } func (m *ImportRequest) String() string { return proto.CompactTextString(m) } func (*ImportRequest) ProtoMessage() {} -func (*ImportRequest) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{71} } +func (*ImportRequest) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{73} } type ImportRequest_File struct { Dir ExportStorage `protobuf:"bytes,1,opt,name=dir" json:"dir"` @@ -1775,7 +1799,7 @@ type ImportRequest_File struct { func (m *ImportRequest_File) Reset() { *m = ImportRequest_File{} } func (m *ImportRequest_File) String() string { return proto.CompactTextString(m) } func (*ImportRequest_File) ProtoMessage() {} -func (*ImportRequest_File) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{71, 0} } +func (*ImportRequest_File) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{73, 0} } type ImportRequest_TableRekey struct { // OldID is the previous ID of `new_desc`. @@ -1787,7 +1811,7 @@ type ImportRequest_TableRekey struct { func (m *ImportRequest_TableRekey) Reset() { *m = ImportRequest_TableRekey{} } func (m *ImportRequest_TableRekey) String() string { return proto.CompactTextString(m) } func (*ImportRequest_TableRekey) ProtoMessage() {} -func (*ImportRequest_TableRekey) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{71, 1} } +func (*ImportRequest_TableRekey) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{73, 1} } // ImportResponse is the response to a Import() operation. type ImportResponse struct { @@ -1798,7 +1822,7 @@ type ImportResponse struct { func (m *ImportResponse) Reset() { *m = ImportResponse{} } func (m *ImportResponse) String() string { return proto.CompactTextString(m) } func (*ImportResponse) ProtoMessage() {} -func (*ImportResponse) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{72} } +func (*ImportResponse) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{74} } // AdminScatterRequest is the argument to the AdminScatter() method, which moves // replicas and leaseholders for a selection of ranges. Scatter is best-effort; @@ -1812,7 +1836,7 @@ type AdminScatterRequest struct { func (m *AdminScatterRequest) Reset() { *m = AdminScatterRequest{} } func (m *AdminScatterRequest) String() string { return proto.CompactTextString(m) } func (*AdminScatterRequest) ProtoMessage() {} -func (*AdminScatterRequest) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{73} } +func (*AdminScatterRequest) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{75} } // ScatterResponse is the response to a Scatter() operation. type AdminScatterResponse struct { @@ -1823,7 +1847,7 @@ type AdminScatterResponse struct { func (m *AdminScatterResponse) Reset() { *m = AdminScatterResponse{} } func (m *AdminScatterResponse) String() string { return proto.CompactTextString(m) } func (*AdminScatterResponse) ProtoMessage() {} -func (*AdminScatterResponse) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{74} } +func (*AdminScatterResponse) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{76} } type AdminScatterResponse_Range struct { Span Span `protobuf:"bytes,1,opt,name=span" json:"span"` @@ -1833,7 +1857,7 @@ func (m *AdminScatterResponse_Range) Reset() { *m = AdminScatterResponse func (m *AdminScatterResponse_Range) String() string { return proto.CompactTextString(m) } func (*AdminScatterResponse_Range) ProtoMessage() {} func (*AdminScatterResponse_Range) Descriptor() ([]byte, []int) { - return fileDescriptorApi, []int{74, 0} + return fileDescriptorApi, []int{76, 0} } // AddSSTableRequest is arguments to the AddSSTable() method, to link a file @@ -1846,7 +1870,7 @@ type AddSSTableRequest struct { func (m *AddSSTableRequest) Reset() { *m = AddSSTableRequest{} } func (m *AddSSTableRequest) String() string { return proto.CompactTextString(m) } func (*AddSSTableRequest) ProtoMessage() {} -func (*AddSSTableRequest) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{75} } +func (*AddSSTableRequest) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{77} } // AddSSTableResponse is the response to a AddSSTable() operation. type AddSSTableResponse struct { @@ -1856,7 +1880,7 @@ type AddSSTableResponse struct { func (m *AddSSTableResponse) Reset() { *m = AddSSTableResponse{} } func (m *AddSSTableResponse) String() string { return proto.CompactTextString(m) } func (*AddSSTableResponse) ProtoMessage() {} -func (*AddSSTableResponse) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{76} } +func (*AddSSTableResponse) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{78} } // RefreshRequest is arguments to the Refresh() method, which verifies // that no write has occurred since the transaction's orig timestamp @@ -1876,7 +1900,7 @@ type RefreshRequest struct { func (m *RefreshRequest) Reset() { *m = RefreshRequest{} } func (m *RefreshRequest) String() string { return proto.CompactTextString(m) } func (*RefreshRequest) ProtoMessage() {} -func (*RefreshRequest) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{77} } +func (*RefreshRequest) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{79} } // RefreshResponse is the response to a Refresh() operation. type RefreshResponse struct { @@ -1886,7 +1910,7 @@ type RefreshResponse struct { func (m *RefreshResponse) Reset() { *m = RefreshResponse{} } func (m *RefreshResponse) String() string { return proto.CompactTextString(m) } func (*RefreshResponse) ProtoMessage() {} -func (*RefreshResponse) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{78} } +func (*RefreshResponse) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{80} } // RefreshRangeRequest is arguments to the RefreshRange() method, which // is similar to RefreshRequest (see comments above), but operates on @@ -1902,7 +1926,7 @@ type RefreshRangeRequest struct { func (m *RefreshRangeRequest) Reset() { *m = RefreshRangeRequest{} } func (m *RefreshRangeRequest) String() string { return proto.CompactTextString(m) } func (*RefreshRangeRequest) ProtoMessage() {} -func (*RefreshRangeRequest) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{79} } +func (*RefreshRangeRequest) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{81} } // RefreshRangeResponse is the response to a RefreshRange() operation. type RefreshRangeResponse struct { @@ -1912,7 +1936,7 @@ type RefreshRangeResponse struct { func (m *RefreshRangeResponse) Reset() { *m = RefreshRangeResponse{} } func (m *RefreshRangeResponse) String() string { return proto.CompactTextString(m) } func (*RefreshRangeResponse) ProtoMessage() {} -func (*RefreshRangeResponse) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{80} } +func (*RefreshRangeResponse) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{82} } // SubsumeRequest is the argument to the Subsume() method, which freezes a range // for merging with its left-hand neighbor. @@ -1934,7 +1958,7 @@ type SubsumeRequest struct { func (m *SubsumeRequest) Reset() { *m = SubsumeRequest{} } func (m *SubsumeRequest) String() string { return proto.CompactTextString(m) } func (*SubsumeRequest) ProtoMessage() {} -func (*SubsumeRequest) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{81} } +func (*SubsumeRequest) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{83} } // SubsumeResponse is the response to a SubsumeRequest. type SubsumeResponse struct { @@ -1955,7 +1979,7 @@ type SubsumeResponse struct { func (m *SubsumeResponse) Reset() { *m = SubsumeResponse{} } func (m *SubsumeResponse) String() string { return proto.CompactTextString(m) } func (*SubsumeResponse) ProtoMessage() {} -func (*SubsumeResponse) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{82} } +func (*SubsumeResponse) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{84} } // RangeStatsRequest is the argument to the RangeStats() method. It requests the // MVCC statistics of the receiving range. @@ -1966,7 +1990,7 @@ type RangeStatsRequest struct { func (m *RangeStatsRequest) Reset() { *m = RangeStatsRequest{} } func (m *RangeStatsRequest) String() string { return proto.CompactTextString(m) } func (*RangeStatsRequest) ProtoMessage() {} -func (*RangeStatsRequest) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{83} } +func (*RangeStatsRequest) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{85} } // RangeStatsResponse is the response to a RangeStatsRequest. type RangeStatsResponse struct { @@ -1979,7 +2003,7 @@ type RangeStatsResponse struct { func (m *RangeStatsResponse) Reset() { *m = RangeStatsResponse{} } func (m *RangeStatsResponse) String() string { return proto.CompactTextString(m) } func (*RangeStatsResponse) ProtoMessage() {} -func (*RangeStatsResponse) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{84} } +func (*RangeStatsResponse) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{86} } // A RequestUnion contains exactly one of the requests. // The values added here must match those in ResponseUnion. @@ -2002,6 +2026,7 @@ type RequestUnion struct { // *RequestUnion_AdminMerge // *RequestUnion_AdminTransferLease // *RequestUnion_AdminChangeReplicas + // *RequestUnion_AdminRelocateRange // *RequestUnion_HeartbeatTxn // *RequestUnion_Gc // *RequestUnion_PushTxn @@ -2034,7 +2059,7 @@ type RequestUnion struct { func (m *RequestUnion) Reset() { *m = RequestUnion{} } func (m *RequestUnion) String() string { return proto.CompactTextString(m) } func (*RequestUnion) ProtoMessage() {} -func (*RequestUnion) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{85} } +func (*RequestUnion) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{87} } type isRequestUnion_Value interface { isRequestUnion_Value() @@ -2084,6 +2109,9 @@ type RequestUnion_AdminTransferLease struct { type RequestUnion_AdminChangeReplicas struct { AdminChangeReplicas *AdminChangeReplicasRequest `protobuf:"bytes,35,opt,name=admin_change_replicas,json=adminChangeReplicas,oneof"` } +type RequestUnion_AdminRelocateRange struct { + AdminRelocateRange *AdminRelocateRangeRequest `protobuf:"bytes,45,opt,name=admin_relocate_range,json=adminRelocateRange,oneof"` +} type RequestUnion_HeartbeatTxn struct { HeartbeatTxn *HeartbeatTxnRequest `protobuf:"bytes,12,opt,name=heartbeat_txn,json=heartbeatTxn,oneof"` } @@ -2177,6 +2205,7 @@ func (*RequestUnion_AdminSplit) isRequestUnion_Value() {} func (*RequestUnion_AdminMerge) isRequestUnion_Value() {} func (*RequestUnion_AdminTransferLease) isRequestUnion_Value() {} func (*RequestUnion_AdminChangeReplicas) isRequestUnion_Value() {} +func (*RequestUnion_AdminRelocateRange) isRequestUnion_Value() {} func (*RequestUnion_HeartbeatTxn) isRequestUnion_Value() {} func (*RequestUnion_Gc) isRequestUnion_Value() {} func (*RequestUnion_PushTxn) isRequestUnion_Value() {} @@ -2309,6 +2338,13 @@ func (m *RequestUnion) GetAdminChangeReplicas() *AdminChangeReplicasRequest { return nil } +func (m *RequestUnion) GetAdminRelocateRange() *AdminRelocateRangeRequest { + if x, ok := m.GetValue().(*RequestUnion_AdminRelocateRange); ok { + return x.AdminRelocateRange + } + return nil +} + func (m *RequestUnion) GetHeartbeatTxn() *HeartbeatTxnRequest { if x, ok := m.GetValue().(*RequestUnion_HeartbeatTxn); ok { return x.HeartbeatTxn @@ -2508,6 +2544,7 @@ func (*RequestUnion) XXX_OneofFuncs() (func(msg proto.Message, b *proto.Buffer) (*RequestUnion_AdminMerge)(nil), (*RequestUnion_AdminTransferLease)(nil), (*RequestUnion_AdminChangeReplicas)(nil), + (*RequestUnion_AdminRelocateRange)(nil), (*RequestUnion_HeartbeatTxn)(nil), (*RequestUnion_Gc)(nil), (*RequestUnion_PushTxn)(nil), @@ -2611,6 +2648,11 @@ func _RequestUnion_OneofMarshaler(msg proto.Message, b *proto.Buffer) error { if err := b.EncodeMessage(x.AdminChangeReplicas); err != nil { return err } + case *RequestUnion_AdminRelocateRange: + _ = b.EncodeVarint(45<<3 | proto.WireBytes) + if err := b.EncodeMessage(x.AdminRelocateRange); err != nil { + return err + } case *RequestUnion_HeartbeatTxn: _ = b.EncodeVarint(12<<3 | proto.WireBytes) if err := b.EncodeMessage(x.HeartbeatTxn); err != nil { @@ -2863,6 +2905,14 @@ func _RequestUnion_OneofUnmarshaler(msg proto.Message, tag, wire int, b *proto.B err := b.DecodeMessage(msg) m.Value = &RequestUnion_AdminChangeReplicas{msg} return true, err + case 45: // value.admin_relocate_range + if wire != proto.WireBytes { + return true, proto.ErrInternalBadWireType + } + msg := new(AdminRelocateRangeRequest) + err := b.DecodeMessage(msg) + m.Value = &RequestUnion_AdminRelocateRange{msg} + return true, err case 12: // value.heartbeat_txn if wire != proto.WireBytes { return true, proto.ErrInternalBadWireType @@ -3150,6 +3200,11 @@ func _RequestUnion_OneofSizer(msg proto.Message) (n int) { n += proto.SizeVarint(35<<3 | proto.WireBytes) n += proto.SizeVarint(uint64(s)) n += s + case *RequestUnion_AdminRelocateRange: + s := proto.Size(x.AdminRelocateRange) + n += proto.SizeVarint(45<<3 | proto.WireBytes) + n += proto.SizeVarint(uint64(s)) + n += s case *RequestUnion_HeartbeatTxn: s := proto.Size(x.HeartbeatTxn) n += proto.SizeVarint(12<<3 | proto.WireBytes) @@ -3305,6 +3360,7 @@ type ResponseUnion struct { // *ResponseUnion_AdminMerge // *ResponseUnion_AdminTransferLease // *ResponseUnion_AdminChangeReplicas + // *ResponseUnion_AdminRelocateRange // *ResponseUnion_HeartbeatTxn // *ResponseUnion_Gc // *ResponseUnion_PushTxn @@ -3336,7 +3392,7 @@ type ResponseUnion struct { func (m *ResponseUnion) Reset() { *m = ResponseUnion{} } func (m *ResponseUnion) String() string { return proto.CompactTextString(m) } func (*ResponseUnion) ProtoMessage() {} -func (*ResponseUnion) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{86} } +func (*ResponseUnion) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{88} } type isResponseUnion_Value interface { isResponseUnion_Value() @@ -3386,6 +3442,9 @@ type ResponseUnion_AdminTransferLease struct { type ResponseUnion_AdminChangeReplicas struct { AdminChangeReplicas *AdminChangeReplicasResponse `protobuf:"bytes,35,opt,name=admin_change_replicas,json=adminChangeReplicas,oneof"` } +type ResponseUnion_AdminRelocateRange struct { + AdminRelocateRange *AdminRelocateRangeResponse `protobuf:"bytes,45,opt,name=admin_relocate_range,json=adminRelocateRange,oneof"` +} type ResponseUnion_HeartbeatTxn struct { HeartbeatTxn *HeartbeatTxnResponse `protobuf:"bytes,12,opt,name=heartbeat_txn,json=heartbeatTxn,oneof"` } @@ -3476,6 +3535,7 @@ func (*ResponseUnion_AdminSplit) isResponseUnion_Value() {} func (*ResponseUnion_AdminMerge) isResponseUnion_Value() {} func (*ResponseUnion_AdminTransferLease) isResponseUnion_Value() {} func (*ResponseUnion_AdminChangeReplicas) isResponseUnion_Value() {} +func (*ResponseUnion_AdminRelocateRange) isResponseUnion_Value() {} func (*ResponseUnion_HeartbeatTxn) isResponseUnion_Value() {} func (*ResponseUnion_Gc) isResponseUnion_Value() {} func (*ResponseUnion_PushTxn) isResponseUnion_Value() {} @@ -3607,6 +3667,13 @@ func (m *ResponseUnion) GetAdminChangeReplicas() *AdminChangeReplicasResponse { return nil } +func (m *ResponseUnion) GetAdminRelocateRange() *AdminRelocateRangeResponse { + if x, ok := m.GetValue().(*ResponseUnion_AdminRelocateRange); ok { + return x.AdminRelocateRange + } + return nil +} + func (m *ResponseUnion) GetHeartbeatTxn() *HeartbeatTxnResponse { if x, ok := m.GetValue().(*ResponseUnion_HeartbeatTxn); ok { return x.HeartbeatTxn @@ -3799,6 +3866,7 @@ func (*ResponseUnion) XXX_OneofFuncs() (func(msg proto.Message, b *proto.Buffer) (*ResponseUnion_AdminMerge)(nil), (*ResponseUnion_AdminTransferLease)(nil), (*ResponseUnion_AdminChangeReplicas)(nil), + (*ResponseUnion_AdminRelocateRange)(nil), (*ResponseUnion_HeartbeatTxn)(nil), (*ResponseUnion_Gc)(nil), (*ResponseUnion_PushTxn)(nil), @@ -3901,6 +3969,11 @@ func _ResponseUnion_OneofMarshaler(msg proto.Message, b *proto.Buffer) error { if err := b.EncodeMessage(x.AdminChangeReplicas); err != nil { return err } + case *ResponseUnion_AdminRelocateRange: + _ = b.EncodeVarint(45<<3 | proto.WireBytes) + if err := b.EncodeMessage(x.AdminRelocateRange); err != nil { + return err + } case *ResponseUnion_HeartbeatTxn: _ = b.EncodeVarint(12<<3 | proto.WireBytes) if err := b.EncodeMessage(x.HeartbeatTxn); err != nil { @@ -4148,6 +4221,14 @@ func _ResponseUnion_OneofUnmarshaler(msg proto.Message, tag, wire int, b *proto. err := b.DecodeMessage(msg) m.Value = &ResponseUnion_AdminChangeReplicas{msg} return true, err + case 45: // value.admin_relocate_range + if wire != proto.WireBytes { + return true, proto.ErrInternalBadWireType + } + msg := new(AdminRelocateRangeResponse) + err := b.DecodeMessage(msg) + m.Value = &ResponseUnion_AdminRelocateRange{msg} + return true, err case 12: // value.heartbeat_txn if wire != proto.WireBytes { return true, proto.ErrInternalBadWireType @@ -4427,6 +4508,11 @@ func _ResponseUnion_OneofSizer(msg proto.Message) (n int) { n += proto.SizeVarint(35<<3 | proto.WireBytes) n += proto.SizeVarint(uint64(s)) n += s + case *ResponseUnion_AdminRelocateRange: + s := proto.Size(x.AdminRelocateRange) + n += proto.SizeVarint(45<<3 | proto.WireBytes) + n += proto.SizeVarint(uint64(s)) + n += s case *ResponseUnion_HeartbeatTxn: s := proto.Size(x.HeartbeatTxn) n += proto.SizeVarint(12<<3 | proto.WireBytes) @@ -4629,7 +4715,7 @@ type Header struct { func (m *Header) Reset() { *m = Header{} } func (m *Header) String() string { return proto.CompactTextString(m) } func (*Header) ProtoMessage() {} -func (*Header) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{87} } +func (*Header) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{89} } // A BatchRequest contains one or more requests to be executed in // parallel, or if applicable (based on write-only commands and @@ -4641,7 +4727,7 @@ type BatchRequest struct { func (m *BatchRequest) Reset() { *m = BatchRequest{} } func (*BatchRequest) ProtoMessage() {} -func (*BatchRequest) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{88} } +func (*BatchRequest) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{90} } // A BatchResponse contains one or more responses, one per request // corresponding to the requests in the matching BatchRequest. The @@ -4654,7 +4740,7 @@ type BatchResponse struct { func (m *BatchResponse) Reset() { *m = BatchResponse{} } func (*BatchResponse) ProtoMessage() {} -func (*BatchResponse) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{89} } +func (*BatchResponse) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{91} } type BatchResponse_Header struct { // error communicates a structured error (i.e. one originating from a Node) @@ -4689,7 +4775,7 @@ type BatchResponse_Header struct { func (m *BatchResponse_Header) Reset() { *m = BatchResponse_Header{} } func (m *BatchResponse_Header) String() string { return proto.CompactTextString(m) } func (*BatchResponse_Header) ProtoMessage() {} -func (*BatchResponse_Header) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{89, 0} } +func (*BatchResponse_Header) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{91, 0} } // RangeFeedRequest is a request that expresses the intention to establish a // RangeFeed stream over the provided span, starting at the specified timestamp. @@ -4701,7 +4787,7 @@ type RangeFeedRequest struct { func (m *RangeFeedRequest) Reset() { *m = RangeFeedRequest{} } func (m *RangeFeedRequest) String() string { return proto.CompactTextString(m) } func (*RangeFeedRequest) ProtoMessage() {} -func (*RangeFeedRequest) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{90} } +func (*RangeFeedRequest) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{92} } // RangeFeedValue is a variant of RangeFeedEvent that represents an update to // the specified key with the provided value. @@ -4713,7 +4799,7 @@ type RangeFeedValue struct { func (m *RangeFeedValue) Reset() { *m = RangeFeedValue{} } func (m *RangeFeedValue) String() string { return proto.CompactTextString(m) } func (*RangeFeedValue) ProtoMessage() {} -func (*RangeFeedValue) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{91} } +func (*RangeFeedValue) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{93} } // RangeFeedCheckpoint is a variant of RangeFeedEvent that represents the // promise that no more RangeFeedValue events with keys in the specified span @@ -4727,7 +4813,7 @@ type RangeFeedCheckpoint struct { func (m *RangeFeedCheckpoint) Reset() { *m = RangeFeedCheckpoint{} } func (m *RangeFeedCheckpoint) String() string { return proto.CompactTextString(m) } func (*RangeFeedCheckpoint) ProtoMessage() {} -func (*RangeFeedCheckpoint) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{92} } +func (*RangeFeedCheckpoint) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{94} } // RangeFeedError is a variant of RangeFeedEvent that indicates that an error // occurred during the processing of the RangeFeed. If emitted, a RangeFeedError @@ -4740,7 +4826,7 @@ type RangeFeedError struct { func (m *RangeFeedError) Reset() { *m = RangeFeedError{} } func (m *RangeFeedError) String() string { return proto.CompactTextString(m) } func (*RangeFeedError) ProtoMessage() {} -func (*RangeFeedError) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{93} } +func (*RangeFeedError) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{95} } // RangeFeedEvent is a union of all event types that may be returned on a // RangeFeed response stream. @@ -4753,7 +4839,7 @@ type RangeFeedEvent struct { func (m *RangeFeedEvent) Reset() { *m = RangeFeedEvent{} } func (m *RangeFeedEvent) String() string { return proto.CompactTextString(m) } func (*RangeFeedEvent) ProtoMessage() {} -func (*RangeFeedEvent) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{94} } +func (*RangeFeedEvent) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{96} } func init() { proto.RegisterType((*RangeInfo)(nil), "cockroach.roachpb.RangeInfo") @@ -4796,6 +4882,8 @@ func init() { proto.RegisterType((*AdminTransferLeaseResponse)(nil), "cockroach.roachpb.AdminTransferLeaseResponse") proto.RegisterType((*AdminChangeReplicasRequest)(nil), "cockroach.roachpb.AdminChangeReplicasRequest") proto.RegisterType((*AdminChangeReplicasResponse)(nil), "cockroach.roachpb.AdminChangeReplicasResponse") + proto.RegisterType((*AdminRelocateRangeRequest)(nil), "cockroach.roachpb.AdminRelocateRangeRequest") + proto.RegisterType((*AdminRelocateRangeResponse)(nil), "cockroach.roachpb.AdminRelocateRangeResponse") proto.RegisterType((*HeartbeatTxnRequest)(nil), "cockroach.roachpb.HeartbeatTxnRequest") proto.RegisterType((*HeartbeatTxnResponse)(nil), "cockroach.roachpb.HeartbeatTxnResponse") proto.RegisterType((*GCRequest)(nil), "cockroach.roachpb.GCRequest") @@ -5450,6 +5538,38 @@ func (this *AdminChangeReplicasRequest) Equal(that interface{}) bool { } return true } +func (this *AdminRelocateRangeRequest) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + that1, ok := that.(*AdminRelocateRangeRequest) + if !ok { + that2, ok := that.(AdminRelocateRangeRequest) + if ok { + that1 = &that2 + } else { + return false + } + } + if that1 == nil { + return this == nil + } else if this == nil { + return false + } + if !this.RequestHeader.Equal(&that1.RequestHeader) { + return false + } + if len(this.Targets) != len(that1.Targets) { + return false + } + for i := range this.Targets { + if !this.Targets[i].Equal(&that1.Targets[i]) { + return false + } + } + return true +} func (this *HeartbeatTxnRequest) Equal(that interface{}) bool { if that == nil { return this == nil @@ -8003,7 +8123,7 @@ func (m *AdminChangeReplicasResponse) MarshalTo(dAtA []byte) (int, error) { return i, nil } -func (m *HeartbeatTxnRequest) Marshal() (dAtA []byte, err error) { +func (m *AdminRelocateRangeRequest) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) n, err := m.MarshalTo(dAtA) @@ -8013,7 +8133,7 @@ func (m *HeartbeatTxnRequest) Marshal() (dAtA []byte, err error) { return dAtA[:n], nil } -func (m *HeartbeatTxnRequest) MarshalTo(dAtA []byte) (int, error) { +func (m *AdminRelocateRangeRequest) MarshalTo(dAtA []byte) (int, error) { var i int _ = i var l int @@ -8026,14 +8146,78 @@ func (m *HeartbeatTxnRequest) MarshalTo(dAtA []byte) (int, error) { return 0, err } i += n50 + if len(m.Targets) > 0 { + for _, msg := range m.Targets { + dAtA[i] = 0x12 + i++ + i = encodeVarintApi(dAtA, i, uint64(msg.Size())) + n, err := msg.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n + } + } + return i, nil +} + +func (m *AdminRelocateRangeResponse) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalTo(dAtA) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *AdminRelocateRangeResponse) MarshalTo(dAtA []byte) (int, error) { + var i int + _ = i + var l int + _ = l + dAtA[i] = 0xa + i++ + i = encodeVarintApi(dAtA, i, uint64(m.ResponseHeader.Size())) + n51, err := m.ResponseHeader.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n51 + return i, nil +} + +func (m *HeartbeatTxnRequest) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalTo(dAtA) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *HeartbeatTxnRequest) MarshalTo(dAtA []byte) (int, error) { + var i int + _ = i + var l int + _ = l + dAtA[i] = 0xa + i++ + i = encodeVarintApi(dAtA, i, uint64(m.RequestHeader.Size())) + n52, err := m.RequestHeader.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n52 dAtA[i] = 0x12 i++ i = encodeVarintApi(dAtA, i, uint64(m.Now.Size())) - n51, err := m.Now.MarshalTo(dAtA[i:]) + n53, err := m.Now.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n51 + i += n53 return i, nil } @@ -8055,11 +8239,11 @@ func (m *HeartbeatTxnResponse) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.ResponseHeader.Size())) - n52, err := m.ResponseHeader.MarshalTo(dAtA[i:]) + n54, err := m.ResponseHeader.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n52 + i += n54 return i, nil } @@ -8081,11 +8265,11 @@ func (m *GCRequest) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.RequestHeader.Size())) - n53, err := m.RequestHeader.MarshalTo(dAtA[i:]) + n55, err := m.RequestHeader.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n53 + i += n55 if len(m.Keys) > 0 { for _, msg := range m.Keys { dAtA[i] = 0x1a @@ -8101,19 +8285,19 @@ func (m *GCRequest) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x22 i++ i = encodeVarintApi(dAtA, i, uint64(m.Threshold.Size())) - n54, err := m.Threshold.MarshalTo(dAtA[i:]) + n56, err := m.Threshold.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n54 + i += n56 dAtA[i] = 0x2a i++ i = encodeVarintApi(dAtA, i, uint64(m.TxnSpanGCThreshold.Size())) - n55, err := m.TxnSpanGCThreshold.MarshalTo(dAtA[i:]) + n57, err := m.TxnSpanGCThreshold.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n55 + i += n57 return i, nil } @@ -8141,11 +8325,11 @@ func (m *GCRequest_GCKey) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x12 i++ i = encodeVarintApi(dAtA, i, uint64(m.Timestamp.Size())) - n56, err := m.Timestamp.MarshalTo(dAtA[i:]) + n58, err := m.Timestamp.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n56 + i += n58 return i, nil } @@ -8167,11 +8351,11 @@ func (m *GCResponse) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.ResponseHeader.Size())) - n57, err := m.ResponseHeader.MarshalTo(dAtA[i:]) + n59, err := m.ResponseHeader.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n57 + i += n59 return i, nil } @@ -8193,43 +8377,43 @@ func (m *PushTxnRequest) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.RequestHeader.Size())) - n58, err := m.RequestHeader.MarshalTo(dAtA[i:]) + n60, err := m.RequestHeader.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n58 + i += n60 dAtA[i] = 0x12 i++ i = encodeVarintApi(dAtA, i, uint64(m.PusherTxn.Size())) - n59, err := m.PusherTxn.MarshalTo(dAtA[i:]) + n61, err := m.PusherTxn.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n59 + i += n61 dAtA[i] = 0x1a i++ i = encodeVarintApi(dAtA, i, uint64(m.PusheeTxn.Size())) - n60, err := m.PusheeTxn.MarshalTo(dAtA[i:]) + n62, err := m.PusheeTxn.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n60 + i += n62 dAtA[i] = 0x22 i++ i = encodeVarintApi(dAtA, i, uint64(m.PushTo.Size())) - n61, err := m.PushTo.MarshalTo(dAtA[i:]) + n63, err := m.PushTo.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n61 + i += n63 dAtA[i] = 0x2a i++ i = encodeVarintApi(dAtA, i, uint64(m.Now.Size())) - n62, err := m.Now.MarshalTo(dAtA[i:]) + n64, err := m.Now.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n62 + i += n64 if m.PushType != 0 { dAtA[i] = 0x30 i++ @@ -8266,19 +8450,19 @@ func (m *PushTxnResponse) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.ResponseHeader.Size())) - n63, err := m.ResponseHeader.MarshalTo(dAtA[i:]) + n65, err := m.ResponseHeader.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n63 + i += n65 dAtA[i] = 0x12 i++ i = encodeVarintApi(dAtA, i, uint64(m.PusheeTxn.Size())) - n64, err := m.PusheeTxn.MarshalTo(dAtA[i:]) + n66, err := m.PusheeTxn.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n64 + i += n66 return i, nil } @@ -8300,19 +8484,19 @@ func (m *QueryTxnRequest) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.RequestHeader.Size())) - n65, err := m.RequestHeader.MarshalTo(dAtA[i:]) + n67, err := m.RequestHeader.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n65 + i += n67 dAtA[i] = 0x12 i++ i = encodeVarintApi(dAtA, i, uint64(m.Txn.Size())) - n66, err := m.Txn.MarshalTo(dAtA[i:]) + n68, err := m.Txn.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n66 + i += n68 if m.WaitForUpdate { dAtA[i] = 0x18 i++ @@ -8356,19 +8540,19 @@ func (m *QueryTxnResponse) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.ResponseHeader.Size())) - n67, err := m.ResponseHeader.MarshalTo(dAtA[i:]) + n69, err := m.ResponseHeader.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n67 + i += n69 dAtA[i] = 0x12 i++ i = encodeVarintApi(dAtA, i, uint64(m.QueriedTxn.Size())) - n68, err := m.QueriedTxn.MarshalTo(dAtA[i:]) + n70, err := m.QueriedTxn.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n68 + i += n70 if len(m.WaitingTxns) > 0 { for _, msg := range m.WaitingTxns { dAtA[i] = 0x1a @@ -8402,19 +8586,19 @@ func (m *QueryIntentRequest) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.RequestHeader.Size())) - n69, err := m.RequestHeader.MarshalTo(dAtA[i:]) + n71, err := m.RequestHeader.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n69 + i += n71 dAtA[i] = 0x12 i++ i = encodeVarintApi(dAtA, i, uint64(m.Txn.Size())) - n70, err := m.Txn.MarshalTo(dAtA[i:]) + n72, err := m.Txn.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n70 + i += n72 if m.IfMissing != 0 { dAtA[i] = 0x18 i++ @@ -8441,11 +8625,11 @@ func (m *QueryIntentResponse) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.ResponseHeader.Size())) - n71, err := m.ResponseHeader.MarshalTo(dAtA[i:]) + n73, err := m.ResponseHeader.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n71 + i += n73 if m.FoundIntent { dAtA[i] = 0x10 i++ @@ -8477,19 +8661,19 @@ func (m *ResolveIntentRequest) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.RequestHeader.Size())) - n72, err := m.RequestHeader.MarshalTo(dAtA[i:]) + n74, err := m.RequestHeader.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n72 + i += n74 dAtA[i] = 0x12 i++ i = encodeVarintApi(dAtA, i, uint64(m.IntentTxn.Size())) - n73, err := m.IntentTxn.MarshalTo(dAtA[i:]) + n75, err := m.IntentTxn.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n73 + i += n75 if m.Status != 0 { dAtA[i] = 0x18 i++ @@ -8526,11 +8710,11 @@ func (m *ResolveIntentResponse) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.ResponseHeader.Size())) - n74, err := m.ResponseHeader.MarshalTo(dAtA[i:]) + n76, err := m.ResponseHeader.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n74 + i += n76 return i, nil } @@ -8552,19 +8736,19 @@ func (m *ResolveIntentRangeRequest) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.RequestHeader.Size())) - n75, err := m.RequestHeader.MarshalTo(dAtA[i:]) + n77, err := m.RequestHeader.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n75 + i += n77 dAtA[i] = 0x12 i++ i = encodeVarintApi(dAtA, i, uint64(m.IntentTxn.Size())) - n76, err := m.IntentTxn.MarshalTo(dAtA[i:]) + n78, err := m.IntentTxn.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n76 + i += n78 if m.Status != 0 { dAtA[i] = 0x18 i++ @@ -8583,11 +8767,11 @@ func (m *ResolveIntentRangeRequest) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2a i++ i = encodeVarintApi(dAtA, i, uint64(m.MinTimestamp.Size())) - n77, err := m.MinTimestamp.MarshalTo(dAtA[i:]) + n79, err := m.MinTimestamp.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n77 + i += n79 return i, nil } @@ -8609,11 +8793,11 @@ func (m *ResolveIntentRangeResponse) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.ResponseHeader.Size())) - n78, err := m.ResponseHeader.MarshalTo(dAtA[i:]) + n80, err := m.ResponseHeader.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n78 + i += n80 return i, nil } @@ -8635,19 +8819,19 @@ func (m *MergeRequest) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.RequestHeader.Size())) - n79, err := m.RequestHeader.MarshalTo(dAtA[i:]) + n81, err := m.RequestHeader.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n79 + i += n81 dAtA[i] = 0x12 i++ i = encodeVarintApi(dAtA, i, uint64(m.Value.Size())) - n80, err := m.Value.MarshalTo(dAtA[i:]) + n82, err := m.Value.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n80 + i += n82 return i, nil } @@ -8669,11 +8853,11 @@ func (m *MergeResponse) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.ResponseHeader.Size())) - n81, err := m.ResponseHeader.MarshalTo(dAtA[i:]) + n83, err := m.ResponseHeader.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n81 + i += n83 return i, nil } @@ -8695,11 +8879,11 @@ func (m *TruncateLogRequest) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.RequestHeader.Size())) - n82, err := m.RequestHeader.MarshalTo(dAtA[i:]) + n84, err := m.RequestHeader.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n82 + i += n84 if m.Index != 0 { dAtA[i] = 0x10 i++ @@ -8731,11 +8915,11 @@ func (m *TruncateLogResponse) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.ResponseHeader.Size())) - n83, err := m.ResponseHeader.MarshalTo(dAtA[i:]) + n85, err := m.ResponseHeader.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n83 + i += n85 return i, nil } @@ -8757,36 +8941,36 @@ func (m *RequestLeaseRequest) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.RequestHeader.Size())) - n84, err := m.RequestHeader.MarshalTo(dAtA[i:]) + n86, err := m.RequestHeader.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n84 + i += n86 dAtA[i] = 0x12 i++ i = encodeVarintApi(dAtA, i, uint64(m.Lease.Size())) - n85, err := m.Lease.MarshalTo(dAtA[i:]) + n87, err := m.Lease.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n85 + i += n87 dAtA[i] = 0x1a i++ i = encodeVarintApi(dAtA, i, uint64(m.PrevLease.Size())) - n86, err := m.PrevLease.MarshalTo(dAtA[i:]) + n88, err := m.PrevLease.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n86 + i += n88 if m.MinProposedTS != nil { dAtA[i] = 0x22 i++ i = encodeVarintApi(dAtA, i, uint64(m.MinProposedTS.Size())) - n87, err := m.MinProposedTS.MarshalTo(dAtA[i:]) + n89, err := m.MinProposedTS.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n87 + i += n89 } return i, nil } @@ -8809,27 +8993,27 @@ func (m *TransferLeaseRequest) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.RequestHeader.Size())) - n88, err := m.RequestHeader.MarshalTo(dAtA[i:]) + n90, err := m.RequestHeader.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n88 + i += n90 dAtA[i] = 0x12 i++ i = encodeVarintApi(dAtA, i, uint64(m.Lease.Size())) - n89, err := m.Lease.MarshalTo(dAtA[i:]) + n91, err := m.Lease.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n89 + i += n91 dAtA[i] = 0x1a i++ i = encodeVarintApi(dAtA, i, uint64(m.PrevLease.Size())) - n90, err := m.PrevLease.MarshalTo(dAtA[i:]) + n92, err := m.PrevLease.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n90 + i += n92 return i, nil } @@ -8851,11 +9035,11 @@ func (m *LeaseInfoRequest) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.RequestHeader.Size())) - n91, err := m.RequestHeader.MarshalTo(dAtA[i:]) + n93, err := m.RequestHeader.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n91 + i += n93 return i, nil } @@ -8877,19 +9061,19 @@ func (m *LeaseInfoResponse) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.ResponseHeader.Size())) - n92, err := m.ResponseHeader.MarshalTo(dAtA[i:]) + n94, err := m.ResponseHeader.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n92 + i += n94 dAtA[i] = 0x12 i++ i = encodeVarintApi(dAtA, i, uint64(m.Lease.Size())) - n93, err := m.Lease.MarshalTo(dAtA[i:]) + n95, err := m.Lease.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n93 + i += n95 return i, nil } @@ -8911,11 +9095,11 @@ func (m *RequestLeaseResponse) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.ResponseHeader.Size())) - n94, err := m.ResponseHeader.MarshalTo(dAtA[i:]) + n96, err := m.ResponseHeader.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n94 + i += n96 return i, nil } @@ -8937,11 +9121,11 @@ func (m *ComputeChecksumRequest) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.RequestHeader.Size())) - n95, err := m.RequestHeader.MarshalTo(dAtA[i:]) + n97, err := m.RequestHeader.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n95 + i += n97 if m.Version != 0 { dAtA[i] = 0x10 i++ @@ -8978,19 +9162,19 @@ func (m *ComputeChecksumResponse) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.ResponseHeader.Size())) - n96, err := m.ResponseHeader.MarshalTo(dAtA[i:]) + n98, err := m.ResponseHeader.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n96 + i += n98 dAtA[i] = 0x12 i++ i = encodeVarintApi(dAtA, i, uint64(m.ChecksumID.Size())) - n97, err := m.ChecksumID.MarshalTo(dAtA[i:]) + n99, err := m.ChecksumID.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n97 + i += n99 return i, nil } @@ -9017,48 +9201,48 @@ func (m *ExportStorage) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x12 i++ i = encodeVarintApi(dAtA, i, uint64(m.LocalFile.Size())) - n98, err := m.LocalFile.MarshalTo(dAtA[i:]) + n100, err := m.LocalFile.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n98 + i += n100 dAtA[i] = 0x1a i++ i = encodeVarintApi(dAtA, i, uint64(m.HttpPath.Size())) - n99, err := m.HttpPath.MarshalTo(dAtA[i:]) + n101, err := m.HttpPath.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n99 + i += n101 if m.GoogleCloudConfig != nil { dAtA[i] = 0x22 i++ i = encodeVarintApi(dAtA, i, uint64(m.GoogleCloudConfig.Size())) - n100, err := m.GoogleCloudConfig.MarshalTo(dAtA[i:]) + n102, err := m.GoogleCloudConfig.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n100 + i += n102 } if m.S3Config != nil { dAtA[i] = 0x2a i++ i = encodeVarintApi(dAtA, i, uint64(m.S3Config.Size())) - n101, err := m.S3Config.MarshalTo(dAtA[i:]) + n103, err := m.S3Config.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n101 + i += n103 } if m.AzureConfig != nil { dAtA[i] = 0x32 i++ i = encodeVarintApi(dAtA, i, uint64(m.AzureConfig.Size())) - n102, err := m.AzureConfig.MarshalTo(dAtA[i:]) + n104, err := m.AzureConfig.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n102 + i += n104 } return i, nil } @@ -9273,19 +9457,19 @@ func (m *WriteBatchRequest) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.RequestHeader.Size())) - n103, err := m.RequestHeader.MarshalTo(dAtA[i:]) + n105, err := m.RequestHeader.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n103 + i += n105 dAtA[i] = 0x12 i++ i = encodeVarintApi(dAtA, i, uint64(m.DataSpan.Size())) - n104, err := m.DataSpan.MarshalTo(dAtA[i:]) + n106, err := m.DataSpan.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n104 + i += n106 if len(m.Data) > 0 { dAtA[i] = 0x1a i++ @@ -9313,11 +9497,11 @@ func (m *WriteBatchResponse) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.ResponseHeader.Size())) - n105, err := m.ResponseHeader.MarshalTo(dAtA[i:]) + n107, err := m.ResponseHeader.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n105 + i += n107 return i, nil } @@ -9339,27 +9523,27 @@ func (m *ExportRequest) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.RequestHeader.Size())) - n106, err := m.RequestHeader.MarshalTo(dAtA[i:]) + n108, err := m.RequestHeader.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n106 + i += n108 dAtA[i] = 0x12 i++ i = encodeVarintApi(dAtA, i, uint64(m.Storage.Size())) - n107, err := m.Storage.MarshalTo(dAtA[i:]) + n109, err := m.Storage.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n107 + i += n109 dAtA[i] = 0x1a i++ i = encodeVarintApi(dAtA, i, uint64(m.StartTime.Size())) - n108, err := m.StartTime.MarshalTo(dAtA[i:]) + n110, err := m.StartTime.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n108 + i += n110 if m.MVCCFilter != 0 { dAtA[i] = 0x20 i++ @@ -9444,11 +9628,11 @@ func (m *ExportResponse) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.ResponseHeader.Size())) - n109, err := m.ResponseHeader.MarshalTo(dAtA[i:]) + n111, err := m.ResponseHeader.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n109 + i += n111 if len(m.Files) > 0 { for _, msg := range m.Files { dAtA[i] = 0x12 @@ -9464,11 +9648,11 @@ func (m *ExportResponse) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1a i++ i = encodeVarintApi(dAtA, i, uint64(m.StartTime.Size())) - n110, err := m.StartTime.MarshalTo(dAtA[i:]) + n112, err := m.StartTime.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n110 + i += n112 return i, nil } @@ -9490,11 +9674,11 @@ func (m *ExportResponse_File) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.Span.Size())) - n111, err := m.Span.MarshalTo(dAtA[i:]) + n113, err := m.Span.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n111 + i += n113 if len(m.Path) > 0 { dAtA[i] = 0x12 i++ @@ -9510,11 +9694,11 @@ func (m *ExportResponse_File) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x32 i++ i = encodeVarintApi(dAtA, i, uint64(m.Exported.Size())) - n112, err := m.Exported.MarshalTo(dAtA[i:]) + n114, err := m.Exported.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n112 + i += n114 if len(m.SST) > 0 { dAtA[i] = 0x3a i++ @@ -9542,11 +9726,11 @@ func (m *ImportRequest) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.RequestHeader.Size())) - n113, err := m.RequestHeader.MarshalTo(dAtA[i:]) + n115, err := m.RequestHeader.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n113 + i += n115 if len(m.Files) > 0 { for _, msg := range m.Files { dAtA[i] = 0x12 @@ -9562,11 +9746,11 @@ func (m *ImportRequest) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1a i++ i = encodeVarintApi(dAtA, i, uint64(m.DataSpan.Size())) - n114, err := m.DataSpan.MarshalTo(dAtA[i:]) + n116, err := m.DataSpan.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n114 + i += n116 if len(m.Rekeys) > 0 { for _, msg := range m.Rekeys { dAtA[i] = 0x2a @@ -9582,11 +9766,11 @@ func (m *ImportRequest) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x32 i++ i = encodeVarintApi(dAtA, i, uint64(m.EndTime.Size())) - n115, err := m.EndTime.MarshalTo(dAtA[i:]) + n117, err := m.EndTime.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n115 + i += n117 return i, nil } @@ -9608,11 +9792,11 @@ func (m *ImportRequest_File) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.Dir.Size())) - n116, err := m.Dir.MarshalTo(dAtA[i:]) + n118, err := m.Dir.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n116 + i += n118 if len(m.Path) > 0 { dAtA[i] = 0x12 i++ @@ -9675,19 +9859,19 @@ func (m *ImportResponse) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.ResponseHeader.Size())) - n117, err := m.ResponseHeader.MarshalTo(dAtA[i:]) + n119, err := m.ResponseHeader.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n117 + i += n119 dAtA[i] = 0x1a i++ i = encodeVarintApi(dAtA, i, uint64(m.Imported.Size())) - n118, err := m.Imported.MarshalTo(dAtA[i:]) + n120, err := m.Imported.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n118 + i += n120 return i, nil } @@ -9709,11 +9893,11 @@ func (m *AdminScatterRequest) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.RequestHeader.Size())) - n119, err := m.RequestHeader.MarshalTo(dAtA[i:]) + n121, err := m.RequestHeader.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n119 + i += n121 if m.RandomizeLeases { dAtA[i] = 0x10 i++ @@ -9745,11 +9929,11 @@ func (m *AdminScatterResponse) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.ResponseHeader.Size())) - n120, err := m.ResponseHeader.MarshalTo(dAtA[i:]) + n122, err := m.ResponseHeader.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n120 + i += n122 if len(m.Ranges) > 0 { for _, msg := range m.Ranges { dAtA[i] = 0x12 @@ -9783,11 +9967,11 @@ func (m *AdminScatterResponse_Range) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.Span.Size())) - n121, err := m.Span.MarshalTo(dAtA[i:]) + n123, err := m.Span.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n121 + i += n123 return i, nil } @@ -9809,11 +9993,11 @@ func (m *AddSSTableRequest) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.RequestHeader.Size())) - n122, err := m.RequestHeader.MarshalTo(dAtA[i:]) + n124, err := m.RequestHeader.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n122 + i += n124 if len(m.Data) > 0 { dAtA[i] = 0x12 i++ @@ -9841,11 +10025,11 @@ func (m *AddSSTableResponse) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.ResponseHeader.Size())) - n123, err := m.ResponseHeader.MarshalTo(dAtA[i:]) + n125, err := m.ResponseHeader.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n123 + i += n125 return i, nil } @@ -9867,11 +10051,11 @@ func (m *RefreshRequest) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.RequestHeader.Size())) - n124, err := m.RequestHeader.MarshalTo(dAtA[i:]) + n126, err := m.RequestHeader.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n124 + i += n126 if m.Write { dAtA[i] = 0x10 i++ @@ -9903,11 +10087,11 @@ func (m *RefreshResponse) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.ResponseHeader.Size())) - n125, err := m.ResponseHeader.MarshalTo(dAtA[i:]) + n127, err := m.ResponseHeader.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n125 + i += n127 return i, nil } @@ -9929,11 +10113,11 @@ func (m *RefreshRangeRequest) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.RequestHeader.Size())) - n126, err := m.RequestHeader.MarshalTo(dAtA[i:]) + n128, err := m.RequestHeader.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n126 + i += n128 if m.Write { dAtA[i] = 0x10 i++ @@ -9965,11 +10149,11 @@ func (m *RefreshRangeResponse) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.ResponseHeader.Size())) - n127, err := m.ResponseHeader.MarshalTo(dAtA[i:]) + n129, err := m.ResponseHeader.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n127 + i += n129 return i, nil } @@ -9991,19 +10175,19 @@ func (m *SubsumeRequest) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.RequestHeader.Size())) - n128, err := m.RequestHeader.MarshalTo(dAtA[i:]) + n130, err := m.RequestHeader.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n128 + i += n130 dAtA[i] = 0x12 i++ i = encodeVarintApi(dAtA, i, uint64(m.LeftRange.Size())) - n129, err := m.LeftRange.MarshalTo(dAtA[i:]) + n131, err := m.LeftRange.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n129 + i += n131 return i, nil } @@ -10025,19 +10209,19 @@ func (m *SubsumeResponse) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.ResponseHeader.Size())) - n130, err := m.ResponseHeader.MarshalTo(dAtA[i:]) + n132, err := m.ResponseHeader.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n130 + i += n132 dAtA[i] = 0x1a i++ i = encodeVarintApi(dAtA, i, uint64(m.MVCCStats.Size())) - n131, err := m.MVCCStats.MarshalTo(dAtA[i:]) + n133, err := m.MVCCStats.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n131 + i += n133 if m.LeaseAppliedIndex != 0 { dAtA[i] = 0x20 i++ @@ -10046,11 +10230,11 @@ func (m *SubsumeResponse) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2a i++ i = encodeVarintApi(dAtA, i, uint64(m.FreezeStart.Size())) - n132, err := m.FreezeStart.MarshalTo(dAtA[i:]) + n134, err := m.FreezeStart.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n132 + i += n134 return i, nil } @@ -10072,11 +10256,11 @@ func (m *RangeStatsRequest) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.RequestHeader.Size())) - n133, err := m.RequestHeader.MarshalTo(dAtA[i:]) + n135, err := m.RequestHeader.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n133 + i += n135 return i, nil } @@ -10098,19 +10282,19 @@ func (m *RangeStatsResponse) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.ResponseHeader.Size())) - n134, err := m.ResponseHeader.MarshalTo(dAtA[i:]) + n136, err := m.ResponseHeader.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n134 + i += n136 dAtA[i] = 0x12 i++ i = encodeVarintApi(dAtA, i, uint64(m.MVCCStats.Size())) - n135, err := m.MVCCStats.MarshalTo(dAtA[i:]) + n137, err := m.MVCCStats.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n135 + i += n137 return i, nil } @@ -10130,11 +10314,11 @@ func (m *RequestUnion) MarshalTo(dAtA []byte) (int, error) { var l int _ = l if m.Value != nil { - nn136, err := m.Value.MarshalTo(dAtA[i:]) + nn138, err := m.Value.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += nn136 + i += nn138 } return i, nil } @@ -10145,11 +10329,11 @@ func (m *RequestUnion_Get) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.Get.Size())) - n137, err := m.Get.MarshalTo(dAtA[i:]) + n139, err := m.Get.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n137 + i += n139 } return i, nil } @@ -10159,11 +10343,11 @@ func (m *RequestUnion_Put) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x12 i++ i = encodeVarintApi(dAtA, i, uint64(m.Put.Size())) - n138, err := m.Put.MarshalTo(dAtA[i:]) + n140, err := m.Put.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n138 + i += n140 } return i, nil } @@ -10173,11 +10357,11 @@ func (m *RequestUnion_ConditionalPut) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1a i++ i = encodeVarintApi(dAtA, i, uint64(m.ConditionalPut.Size())) - n139, err := m.ConditionalPut.MarshalTo(dAtA[i:]) + n141, err := m.ConditionalPut.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n139 + i += n141 } return i, nil } @@ -10187,11 +10371,11 @@ func (m *RequestUnion_Increment) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x22 i++ i = encodeVarintApi(dAtA, i, uint64(m.Increment.Size())) - n140, err := m.Increment.MarshalTo(dAtA[i:]) + n142, err := m.Increment.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n140 + i += n142 } return i, nil } @@ -10201,11 +10385,11 @@ func (m *RequestUnion_Delete) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2a i++ i = encodeVarintApi(dAtA, i, uint64(m.Delete.Size())) - n141, err := m.Delete.MarshalTo(dAtA[i:]) + n143, err := m.Delete.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n141 + i += n143 } return i, nil } @@ -10215,11 +10399,11 @@ func (m *RequestUnion_DeleteRange) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x32 i++ i = encodeVarintApi(dAtA, i, uint64(m.DeleteRange.Size())) - n142, err := m.DeleteRange.MarshalTo(dAtA[i:]) + n144, err := m.DeleteRange.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n142 + i += n144 } return i, nil } @@ -10229,11 +10413,11 @@ func (m *RequestUnion_Scan) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x3a i++ i = encodeVarintApi(dAtA, i, uint64(m.Scan.Size())) - n143, err := m.Scan.MarshalTo(dAtA[i:]) + n145, err := m.Scan.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n143 + i += n145 } return i, nil } @@ -10243,11 +10427,11 @@ func (m *RequestUnion_BeginTransaction) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x42 i++ i = encodeVarintApi(dAtA, i, uint64(m.BeginTransaction.Size())) - n144, err := m.BeginTransaction.MarshalTo(dAtA[i:]) + n146, err := m.BeginTransaction.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n144 + i += n146 } return i, nil } @@ -10257,11 +10441,11 @@ func (m *RequestUnion_EndTransaction) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x4a i++ i = encodeVarintApi(dAtA, i, uint64(m.EndTransaction.Size())) - n145, err := m.EndTransaction.MarshalTo(dAtA[i:]) + n147, err := m.EndTransaction.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n145 + i += n147 } return i, nil } @@ -10271,11 +10455,11 @@ func (m *RequestUnion_AdminSplit) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x52 i++ i = encodeVarintApi(dAtA, i, uint64(m.AdminSplit.Size())) - n146, err := m.AdminSplit.MarshalTo(dAtA[i:]) + n148, err := m.AdminSplit.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n146 + i += n148 } return i, nil } @@ -10285,11 +10469,11 @@ func (m *RequestUnion_AdminMerge) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x5a i++ i = encodeVarintApi(dAtA, i, uint64(m.AdminMerge.Size())) - n147, err := m.AdminMerge.MarshalTo(dAtA[i:]) + n149, err := m.AdminMerge.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n147 + i += n149 } return i, nil } @@ -10299,11 +10483,11 @@ func (m *RequestUnion_HeartbeatTxn) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x62 i++ i = encodeVarintApi(dAtA, i, uint64(m.HeartbeatTxn.Size())) - n148, err := m.HeartbeatTxn.MarshalTo(dAtA[i:]) + n150, err := m.HeartbeatTxn.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n148 + i += n150 } return i, nil } @@ -10313,11 +10497,11 @@ func (m *RequestUnion_Gc) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x6a i++ i = encodeVarintApi(dAtA, i, uint64(m.Gc.Size())) - n149, err := m.Gc.MarshalTo(dAtA[i:]) + n151, err := m.Gc.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n149 + i += n151 } return i, nil } @@ -10327,11 +10511,11 @@ func (m *RequestUnion_PushTxn) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x72 i++ i = encodeVarintApi(dAtA, i, uint64(m.PushTxn.Size())) - n150, err := m.PushTxn.MarshalTo(dAtA[i:]) + n152, err := m.PushTxn.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n150 + i += n152 } return i, nil } @@ -10343,11 +10527,11 @@ func (m *RequestUnion_ResolveIntent) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.ResolveIntent.Size())) - n151, err := m.ResolveIntent.MarshalTo(dAtA[i:]) + n153, err := m.ResolveIntent.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n151 + i += n153 } return i, nil } @@ -10359,11 +10543,11 @@ func (m *RequestUnion_ResolveIntentRange) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.ResolveIntentRange.Size())) - n152, err := m.ResolveIntentRange.MarshalTo(dAtA[i:]) + n154, err := m.ResolveIntentRange.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n152 + i += n154 } return i, nil } @@ -10375,11 +10559,11 @@ func (m *RequestUnion_Merge) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.Merge.Size())) - n153, err := m.Merge.MarshalTo(dAtA[i:]) + n155, err := m.Merge.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n153 + i += n155 } return i, nil } @@ -10391,11 +10575,11 @@ func (m *RequestUnion_TruncateLog) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.TruncateLog.Size())) - n154, err := m.TruncateLog.MarshalTo(dAtA[i:]) + n156, err := m.TruncateLog.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n154 + i += n156 } return i, nil } @@ -10407,11 +10591,11 @@ func (m *RequestUnion_RequestLease) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.RequestLease.Size())) - n155, err := m.RequestLease.MarshalTo(dAtA[i:]) + n157, err := m.RequestLease.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n155 + i += n157 } return i, nil } @@ -10423,11 +10607,11 @@ func (m *RequestUnion_ReverseScan) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.ReverseScan.Size())) - n156, err := m.ReverseScan.MarshalTo(dAtA[i:]) + n158, err := m.ReverseScan.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n156 + i += n158 } return i, nil } @@ -10439,11 +10623,11 @@ func (m *RequestUnion_ComputeChecksum) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.ComputeChecksum.Size())) - n157, err := m.ComputeChecksum.MarshalTo(dAtA[i:]) + n159, err := m.ComputeChecksum.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n157 + i += n159 } return i, nil } @@ -10455,11 +10639,11 @@ func (m *RequestUnion_CheckConsistency) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.CheckConsistency.Size())) - n158, err := m.CheckConsistency.MarshalTo(dAtA[i:]) + n160, err := m.CheckConsistency.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n158 + i += n160 } return i, nil } @@ -10471,11 +10655,11 @@ func (m *RequestUnion_InitPut) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.InitPut.Size())) - n159, err := m.InitPut.MarshalTo(dAtA[i:]) + n161, err := m.InitPut.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n159 + i += n161 } return i, nil } @@ -10487,11 +10671,11 @@ func (m *RequestUnion_TransferLease) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.TransferLease.Size())) - n160, err := m.TransferLease.MarshalTo(dAtA[i:]) + n162, err := m.TransferLease.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n160 + i += n162 } return i, nil } @@ -10503,11 +10687,11 @@ func (m *RequestUnion_AdminTransferLease) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.AdminTransferLease.Size())) - n161, err := m.AdminTransferLease.MarshalTo(dAtA[i:]) + n163, err := m.AdminTransferLease.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n161 + i += n163 } return i, nil } @@ -10519,11 +10703,11 @@ func (m *RequestUnion_LeaseInfo) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.LeaseInfo.Size())) - n162, err := m.LeaseInfo.MarshalTo(dAtA[i:]) + n164, err := m.LeaseInfo.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n162 + i += n164 } return i, nil } @@ -10535,11 +10719,11 @@ func (m *RequestUnion_WriteBatch) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.WriteBatch.Size())) - n163, err := m.WriteBatch.MarshalTo(dAtA[i:]) + n165, err := m.WriteBatch.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n163 + i += n165 } return i, nil } @@ -10551,11 +10735,11 @@ func (m *RequestUnion_Export) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.Export.Size())) - n164, err := m.Export.MarshalTo(dAtA[i:]) + n166, err := m.Export.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n164 + i += n166 } return i, nil } @@ -10567,11 +10751,11 @@ func (m *RequestUnion_QueryTxn) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.QueryTxn.Size())) - n165, err := m.QueryTxn.MarshalTo(dAtA[i:]) + n167, err := m.QueryTxn.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n165 + i += n167 } return i, nil } @@ -10583,11 +10767,11 @@ func (m *RequestUnion_Import) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.Import.Size())) - n166, err := m.Import.MarshalTo(dAtA[i:]) + n168, err := m.Import.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n166 + i += n168 } return i, nil } @@ -10599,11 +10783,11 @@ func (m *RequestUnion_AdminChangeReplicas) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.AdminChangeReplicas.Size())) - n167, err := m.AdminChangeReplicas.MarshalTo(dAtA[i:]) + n169, err := m.AdminChangeReplicas.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n167 + i += n169 } return i, nil } @@ -10615,11 +10799,11 @@ func (m *RequestUnion_AdminScatter) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.AdminScatter.Size())) - n168, err := m.AdminScatter.MarshalTo(dAtA[i:]) + n170, err := m.AdminScatter.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n168 + i += n170 } return i, nil } @@ -10631,11 +10815,11 @@ func (m *RequestUnion_AddSstable) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.AddSstable.Size())) - n169, err := m.AddSstable.MarshalTo(dAtA[i:]) + n171, err := m.AddSstable.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n169 + i += n171 } return i, nil } @@ -10647,11 +10831,11 @@ func (m *RequestUnion_ClearRange) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.ClearRange.Size())) - n170, err := m.ClearRange.MarshalTo(dAtA[i:]) + n172, err := m.ClearRange.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n170 + i += n172 } return i, nil } @@ -10663,11 +10847,11 @@ func (m *RequestUnion_RecomputeStats) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.RecomputeStats.Size())) - n171, err := m.RecomputeStats.MarshalTo(dAtA[i:]) + n173, err := m.RecomputeStats.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n171 + i += n173 } return i, nil } @@ -10679,11 +10863,11 @@ func (m *RequestUnion_Refresh) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.Refresh.Size())) - n172, err := m.Refresh.MarshalTo(dAtA[i:]) + n174, err := m.Refresh.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n172 + i += n174 } return i, nil } @@ -10695,11 +10879,11 @@ func (m *RequestUnion_RefreshRange) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.RefreshRange.Size())) - n173, err := m.RefreshRange.MarshalTo(dAtA[i:]) + n175, err := m.RefreshRange.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n173 + i += n175 } return i, nil } @@ -10711,11 +10895,11 @@ func (m *RequestUnion_QueryIntent) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.QueryIntent.Size())) - n174, err := m.QueryIntent.MarshalTo(dAtA[i:]) + n176, err := m.QueryIntent.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n174 + i += n176 } return i, nil } @@ -10727,11 +10911,11 @@ func (m *RequestUnion_Subsume) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.Subsume.Size())) - n175, err := m.Subsume.MarshalTo(dAtA[i:]) + n177, err := m.Subsume.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n175 + i += n177 } return i, nil } @@ -10743,11 +10927,27 @@ func (m *RequestUnion_RangeStats) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.RangeStats.Size())) - n176, err := m.RangeStats.MarshalTo(dAtA[i:]) + n178, err := m.RangeStats.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n176 + i += n178 + } + return i, nil +} +func (m *RequestUnion_AdminRelocateRange) MarshalTo(dAtA []byte) (int, error) { + i := 0 + if m.AdminRelocateRange != nil { + dAtA[i] = 0xea + i++ + dAtA[i] = 0x2 + i++ + i = encodeVarintApi(dAtA, i, uint64(m.AdminRelocateRange.Size())) + n179, err := m.AdminRelocateRange.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n179 } return i, nil } @@ -10767,11 +10967,11 @@ func (m *ResponseUnion) MarshalTo(dAtA []byte) (int, error) { var l int _ = l if m.Value != nil { - nn177, err := m.Value.MarshalTo(dAtA[i:]) + nn180, err := m.Value.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += nn177 + i += nn180 } return i, nil } @@ -10782,11 +10982,11 @@ func (m *ResponseUnion_Get) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.Get.Size())) - n178, err := m.Get.MarshalTo(dAtA[i:]) + n181, err := m.Get.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n178 + i += n181 } return i, nil } @@ -10796,11 +10996,11 @@ func (m *ResponseUnion_Put) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x12 i++ i = encodeVarintApi(dAtA, i, uint64(m.Put.Size())) - n179, err := m.Put.MarshalTo(dAtA[i:]) + n182, err := m.Put.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n179 + i += n182 } return i, nil } @@ -10810,11 +11010,11 @@ func (m *ResponseUnion_ConditionalPut) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1a i++ i = encodeVarintApi(dAtA, i, uint64(m.ConditionalPut.Size())) - n180, err := m.ConditionalPut.MarshalTo(dAtA[i:]) + n183, err := m.ConditionalPut.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n180 + i += n183 } return i, nil } @@ -10824,11 +11024,11 @@ func (m *ResponseUnion_Increment) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x22 i++ i = encodeVarintApi(dAtA, i, uint64(m.Increment.Size())) - n181, err := m.Increment.MarshalTo(dAtA[i:]) + n184, err := m.Increment.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n181 + i += n184 } return i, nil } @@ -10838,11 +11038,11 @@ func (m *ResponseUnion_Delete) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2a i++ i = encodeVarintApi(dAtA, i, uint64(m.Delete.Size())) - n182, err := m.Delete.MarshalTo(dAtA[i:]) + n185, err := m.Delete.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n182 + i += n185 } return i, nil } @@ -10852,11 +11052,11 @@ func (m *ResponseUnion_DeleteRange) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x32 i++ i = encodeVarintApi(dAtA, i, uint64(m.DeleteRange.Size())) - n183, err := m.DeleteRange.MarshalTo(dAtA[i:]) + n186, err := m.DeleteRange.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n183 + i += n186 } return i, nil } @@ -10866,11 +11066,11 @@ func (m *ResponseUnion_Scan) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x3a i++ i = encodeVarintApi(dAtA, i, uint64(m.Scan.Size())) - n184, err := m.Scan.MarshalTo(dAtA[i:]) + n187, err := m.Scan.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n184 + i += n187 } return i, nil } @@ -10880,11 +11080,11 @@ func (m *ResponseUnion_BeginTransaction) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x42 i++ i = encodeVarintApi(dAtA, i, uint64(m.BeginTransaction.Size())) - n185, err := m.BeginTransaction.MarshalTo(dAtA[i:]) + n188, err := m.BeginTransaction.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n185 + i += n188 } return i, nil } @@ -10894,11 +11094,11 @@ func (m *ResponseUnion_EndTransaction) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x4a i++ i = encodeVarintApi(dAtA, i, uint64(m.EndTransaction.Size())) - n186, err := m.EndTransaction.MarshalTo(dAtA[i:]) + n189, err := m.EndTransaction.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n186 + i += n189 } return i, nil } @@ -10908,11 +11108,11 @@ func (m *ResponseUnion_AdminSplit) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x52 i++ i = encodeVarintApi(dAtA, i, uint64(m.AdminSplit.Size())) - n187, err := m.AdminSplit.MarshalTo(dAtA[i:]) + n190, err := m.AdminSplit.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n187 + i += n190 } return i, nil } @@ -10922,11 +11122,11 @@ func (m *ResponseUnion_AdminMerge) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x5a i++ i = encodeVarintApi(dAtA, i, uint64(m.AdminMerge.Size())) - n188, err := m.AdminMerge.MarshalTo(dAtA[i:]) + n191, err := m.AdminMerge.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n188 + i += n191 } return i, nil } @@ -10936,11 +11136,11 @@ func (m *ResponseUnion_HeartbeatTxn) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x62 i++ i = encodeVarintApi(dAtA, i, uint64(m.HeartbeatTxn.Size())) - n189, err := m.HeartbeatTxn.MarshalTo(dAtA[i:]) + n192, err := m.HeartbeatTxn.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n189 + i += n192 } return i, nil } @@ -10950,11 +11150,11 @@ func (m *ResponseUnion_Gc) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x6a i++ i = encodeVarintApi(dAtA, i, uint64(m.Gc.Size())) - n190, err := m.Gc.MarshalTo(dAtA[i:]) + n193, err := m.Gc.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n190 + i += n193 } return i, nil } @@ -10964,11 +11164,11 @@ func (m *ResponseUnion_PushTxn) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x72 i++ i = encodeVarintApi(dAtA, i, uint64(m.PushTxn.Size())) - n191, err := m.PushTxn.MarshalTo(dAtA[i:]) + n194, err := m.PushTxn.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n191 + i += n194 } return i, nil } @@ -10980,11 +11180,11 @@ func (m *ResponseUnion_ResolveIntent) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.ResolveIntent.Size())) - n192, err := m.ResolveIntent.MarshalTo(dAtA[i:]) + n195, err := m.ResolveIntent.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n192 + i += n195 } return i, nil } @@ -10996,11 +11196,11 @@ func (m *ResponseUnion_ResolveIntentRange) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.ResolveIntentRange.Size())) - n193, err := m.ResolveIntentRange.MarshalTo(dAtA[i:]) + n196, err := m.ResolveIntentRange.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n193 + i += n196 } return i, nil } @@ -11012,11 +11212,11 @@ func (m *ResponseUnion_Merge) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.Merge.Size())) - n194, err := m.Merge.MarshalTo(dAtA[i:]) + n197, err := m.Merge.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n194 + i += n197 } return i, nil } @@ -11028,11 +11228,11 @@ func (m *ResponseUnion_TruncateLog) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.TruncateLog.Size())) - n195, err := m.TruncateLog.MarshalTo(dAtA[i:]) + n198, err := m.TruncateLog.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n195 + i += n198 } return i, nil } @@ -11044,11 +11244,11 @@ func (m *ResponseUnion_RequestLease) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.RequestLease.Size())) - n196, err := m.RequestLease.MarshalTo(dAtA[i:]) + n199, err := m.RequestLease.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n196 + i += n199 } return i, nil } @@ -11060,11 +11260,11 @@ func (m *ResponseUnion_ReverseScan) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.ReverseScan.Size())) - n197, err := m.ReverseScan.MarshalTo(dAtA[i:]) + n200, err := m.ReverseScan.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n197 + i += n200 } return i, nil } @@ -11076,11 +11276,11 @@ func (m *ResponseUnion_ComputeChecksum) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.ComputeChecksum.Size())) - n198, err := m.ComputeChecksum.MarshalTo(dAtA[i:]) + n201, err := m.ComputeChecksum.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n198 + i += n201 } return i, nil } @@ -11092,11 +11292,11 @@ func (m *ResponseUnion_CheckConsistency) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.CheckConsistency.Size())) - n199, err := m.CheckConsistency.MarshalTo(dAtA[i:]) + n202, err := m.CheckConsistency.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n199 + i += n202 } return i, nil } @@ -11108,11 +11308,11 @@ func (m *ResponseUnion_InitPut) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.InitPut.Size())) - n200, err := m.InitPut.MarshalTo(dAtA[i:]) + n203, err := m.InitPut.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n200 + i += n203 } return i, nil } @@ -11124,11 +11324,11 @@ func (m *ResponseUnion_AdminTransferLease) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.AdminTransferLease.Size())) - n201, err := m.AdminTransferLease.MarshalTo(dAtA[i:]) + n204, err := m.AdminTransferLease.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n201 + i += n204 } return i, nil } @@ -11140,11 +11340,11 @@ func (m *ResponseUnion_LeaseInfo) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.LeaseInfo.Size())) - n202, err := m.LeaseInfo.MarshalTo(dAtA[i:]) + n205, err := m.LeaseInfo.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n202 + i += n205 } return i, nil } @@ -11156,11 +11356,11 @@ func (m *ResponseUnion_WriteBatch) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.WriteBatch.Size())) - n203, err := m.WriteBatch.MarshalTo(dAtA[i:]) + n206, err := m.WriteBatch.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n203 + i += n206 } return i, nil } @@ -11172,11 +11372,11 @@ func (m *ResponseUnion_Export) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.Export.Size())) - n204, err := m.Export.MarshalTo(dAtA[i:]) + n207, err := m.Export.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n204 + i += n207 } return i, nil } @@ -11188,11 +11388,11 @@ func (m *ResponseUnion_QueryTxn) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.QueryTxn.Size())) - n205, err := m.QueryTxn.MarshalTo(dAtA[i:]) + n208, err := m.QueryTxn.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n205 + i += n208 } return i, nil } @@ -11204,11 +11404,11 @@ func (m *ResponseUnion_Import) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.Import.Size())) - n206, err := m.Import.MarshalTo(dAtA[i:]) + n209, err := m.Import.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n206 + i += n209 } return i, nil } @@ -11220,11 +11420,11 @@ func (m *ResponseUnion_AdminChangeReplicas) MarshalTo(dAtA []byte) (int, error) dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.AdminChangeReplicas.Size())) - n207, err := m.AdminChangeReplicas.MarshalTo(dAtA[i:]) + n210, err := m.AdminChangeReplicas.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n207 + i += n210 } return i, nil } @@ -11236,11 +11436,11 @@ func (m *ResponseUnion_AdminScatter) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.AdminScatter.Size())) - n208, err := m.AdminScatter.MarshalTo(dAtA[i:]) + n211, err := m.AdminScatter.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n208 + i += n211 } return i, nil } @@ -11252,11 +11452,11 @@ func (m *ResponseUnion_AddSstable) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.AddSstable.Size())) - n209, err := m.AddSstable.MarshalTo(dAtA[i:]) + n212, err := m.AddSstable.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n209 + i += n212 } return i, nil } @@ -11268,11 +11468,11 @@ func (m *ResponseUnion_ClearRange) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.ClearRange.Size())) - n210, err := m.ClearRange.MarshalTo(dAtA[i:]) + n213, err := m.ClearRange.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n210 + i += n213 } return i, nil } @@ -11284,11 +11484,11 @@ func (m *ResponseUnion_RecomputeStats) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.RecomputeStats.Size())) - n211, err := m.RecomputeStats.MarshalTo(dAtA[i:]) + n214, err := m.RecomputeStats.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n211 + i += n214 } return i, nil } @@ -11300,11 +11500,11 @@ func (m *ResponseUnion_Refresh) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.Refresh.Size())) - n212, err := m.Refresh.MarshalTo(dAtA[i:]) + n215, err := m.Refresh.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n212 + i += n215 } return i, nil } @@ -11316,11 +11516,11 @@ func (m *ResponseUnion_RefreshRange) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.RefreshRange.Size())) - n213, err := m.RefreshRange.MarshalTo(dAtA[i:]) + n216, err := m.RefreshRange.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n213 + i += n216 } return i, nil } @@ -11332,11 +11532,11 @@ func (m *ResponseUnion_QueryIntent) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.QueryIntent.Size())) - n214, err := m.QueryIntent.MarshalTo(dAtA[i:]) + n217, err := m.QueryIntent.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n214 + i += n217 } return i, nil } @@ -11348,11 +11548,11 @@ func (m *ResponseUnion_Subsume) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.Subsume.Size())) - n215, err := m.Subsume.MarshalTo(dAtA[i:]) + n218, err := m.Subsume.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n215 + i += n218 } return i, nil } @@ -11364,11 +11564,27 @@ func (m *ResponseUnion_RangeStats) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.RangeStats.Size())) - n216, err := m.RangeStats.MarshalTo(dAtA[i:]) + n219, err := m.RangeStats.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n216 + i += n219 + } + return i, nil +} +func (m *ResponseUnion_AdminRelocateRange) MarshalTo(dAtA []byte) (int, error) { + i := 0 + if m.AdminRelocateRange != nil { + dAtA[i] = 0xea + i++ + dAtA[i] = 0x2 + i++ + i = encodeVarintApi(dAtA, i, uint64(m.AdminRelocateRange.Size())) + n220, err := m.AdminRelocateRange.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n220 } return i, nil } @@ -11390,19 +11606,19 @@ func (m *Header) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.Timestamp.Size())) - n217, err := m.Timestamp.MarshalTo(dAtA[i:]) + n221, err := m.Timestamp.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n217 + i += n221 dAtA[i] = 0x12 i++ i = encodeVarintApi(dAtA, i, uint64(m.Replica.Size())) - n218, err := m.Replica.MarshalTo(dAtA[i:]) + n222, err := m.Replica.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n218 + i += n222 if m.RangeID != 0 { dAtA[i] = 0x18 i++ @@ -11418,11 +11634,11 @@ func (m *Header) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2a i++ i = encodeVarintApi(dAtA, i, uint64(m.Txn.Size())) - n219, err := m.Txn.MarshalTo(dAtA[i:]) + n223, err := m.Txn.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n219 + i += n223 } if m.ReadConsistency != 0 { dAtA[i] = 0x30 @@ -11463,11 +11679,11 @@ func (m *Header) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x62 i++ i = encodeVarintApi(dAtA, i, uint64(m.ScanOptions.Size())) - n220, err := m.ScanOptions.MarshalTo(dAtA[i:]) + n224, err := m.ScanOptions.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n220 + i += n224 } if m.AsyncConsensus { dAtA[i] = 0x68 @@ -11500,11 +11716,11 @@ func (m *BatchRequest) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.Header.Size())) - n221, err := m.Header.MarshalTo(dAtA[i:]) + n225, err := m.Header.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n221 + i += n225 if len(m.Requests) > 0 { for _, msg := range m.Requests { dAtA[i] = 0x12 @@ -11538,11 +11754,11 @@ func (m *BatchResponse) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.BatchResponse_Header.Size())) - n222, err := m.BatchResponse_Header.MarshalTo(dAtA[i:]) + n226, err := m.BatchResponse_Header.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n222 + i += n226 if len(m.Responses) > 0 { for _, msg := range m.Responses { dAtA[i] = 0x12 @@ -11577,38 +11793,38 @@ func (m *BatchResponse_Header) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.Error.Size())) - n223, err := m.Error.MarshalTo(dAtA[i:]) + n227, err := m.Error.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n223 + i += n227 } dAtA[i] = 0x12 i++ i = encodeVarintApi(dAtA, i, uint64(m.Timestamp.Size())) - n224, err := m.Timestamp.MarshalTo(dAtA[i:]) + n228, err := m.Timestamp.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n224 + i += n228 if m.Txn != nil { dAtA[i] = 0x1a i++ i = encodeVarintApi(dAtA, i, uint64(m.Txn.Size())) - n225, err := m.Txn.MarshalTo(dAtA[i:]) + n229, err := m.Txn.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n225 + i += n229 } dAtA[i] = 0x2a i++ i = encodeVarintApi(dAtA, i, uint64(m.Now.Size())) - n226, err := m.Now.MarshalTo(dAtA[i:]) + n230, err := m.Now.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n226 + i += n230 if len(m.CollectedSpans) > 0 { for _, msg := range m.CollectedSpans { dAtA[i] = 0x32 @@ -11642,19 +11858,19 @@ func (m *RangeFeedRequest) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.Header.Size())) - n227, err := m.Header.MarshalTo(dAtA[i:]) + n231, err := m.Header.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n227 + i += n231 dAtA[i] = 0x12 i++ i = encodeVarintApi(dAtA, i, uint64(m.Span.Size())) - n228, err := m.Span.MarshalTo(dAtA[i:]) + n232, err := m.Span.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n228 + i += n232 return i, nil } @@ -11682,11 +11898,11 @@ func (m *RangeFeedValue) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x12 i++ i = encodeVarintApi(dAtA, i, uint64(m.Value.Size())) - n229, err := m.Value.MarshalTo(dAtA[i:]) + n233, err := m.Value.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n229 + i += n233 return i, nil } @@ -11708,19 +11924,19 @@ func (m *RangeFeedCheckpoint) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.Span.Size())) - n230, err := m.Span.MarshalTo(dAtA[i:]) + n234, err := m.Span.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n230 + i += n234 dAtA[i] = 0x12 i++ i = encodeVarintApi(dAtA, i, uint64(m.ResolvedTS.Size())) - n231, err := m.ResolvedTS.MarshalTo(dAtA[i:]) + n235, err := m.ResolvedTS.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n231 + i += n235 return i, nil } @@ -11742,11 +11958,11 @@ func (m *RangeFeedError) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.Error.Size())) - n232, err := m.Error.MarshalTo(dAtA[i:]) + n236, err := m.Error.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n232 + i += n236 return i, nil } @@ -11769,31 +11985,31 @@ func (m *RangeFeedEvent) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.Val.Size())) - n233, err := m.Val.MarshalTo(dAtA[i:]) + n237, err := m.Val.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n233 + i += n237 } if m.Checkpoint != nil { dAtA[i] = 0x12 i++ i = encodeVarintApi(dAtA, i, uint64(m.Checkpoint.Size())) - n234, err := m.Checkpoint.MarshalTo(dAtA[i:]) + n238, err := m.Checkpoint.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n234 + i += n238 } if m.Error != nil { dAtA[i] = 0x1a i++ i = encodeVarintApi(dAtA, i, uint64(m.Error.Size())) - n235, err := m.Error.MarshalTo(dAtA[i:]) + n239, err := m.Error.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n235 + i += n239 } return i, nil } @@ -12294,7 +12510,29 @@ func (m *AdminChangeReplicasRequest) Size() (n int) { return n } -func (m *AdminChangeReplicasResponse) Size() (n int) { +func (m *AdminChangeReplicasResponse) Size() (n int) { + var l int + _ = l + l = m.ResponseHeader.Size() + n += 1 + l + sovApi(uint64(l)) + return n +} + +func (m *AdminRelocateRangeRequest) Size() (n int) { + var l int + _ = l + l = m.RequestHeader.Size() + n += 1 + l + sovApi(uint64(l)) + if len(m.Targets) > 0 { + for _, e := range m.Targets { + l = e.Size() + n += 1 + l + sovApi(uint64(l)) + } + } + return n +} + +func (m *AdminRelocateRangeResponse) Size() (n int) { var l int _ = l l = m.ResponseHeader.Size() @@ -13405,6 +13643,15 @@ func (m *RequestUnion_RangeStats) Size() (n int) { } return n } +func (m *RequestUnion_AdminRelocateRange) Size() (n int) { + var l int + _ = l + if m.AdminRelocateRange != nil { + l = m.AdminRelocateRange.Size() + n += 2 + l + sovApi(uint64(l)) + } + return n +} func (m *ResponseUnion) Size() (n int) { var l int _ = l @@ -13765,6 +14012,15 @@ func (m *ResponseUnion_RangeStats) Size() (n int) { } return n } +func (m *ResponseUnion_AdminRelocateRange) Size() (n int) { + var l int + _ = l + if m.AdminRelocateRange != nil { + l = m.AdminRelocateRange.Size() + n += 2 + l + sovApi(uint64(l)) + } + return n +} func (m *Header) Size() (n int) { var l int _ = l @@ -18338,6 +18594,197 @@ func (m *AdminChangeReplicasResponse) Unmarshal(dAtA []byte) error { } return nil } +func (m *AdminRelocateRangeRequest) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowApi + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: AdminRelocateRangeRequest: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: AdminRelocateRangeRequest: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field RequestHeader", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowApi + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthApi + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if err := m.RequestHeader.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Targets", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowApi + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthApi + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Targets = append(m.Targets, ReplicationTarget{}) + if err := m.Targets[len(m.Targets)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipApi(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthApi + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *AdminRelocateRangeResponse) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowApi + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: AdminRelocateRangeResponse: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: AdminRelocateRangeResponse: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ResponseHeader", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowApi + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthApi + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if err := m.ResponseHeader.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipApi(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthApi + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} func (m *HeartbeatTxnRequest) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 @@ -26481,6 +26928,38 @@ func (m *RequestUnion) Unmarshal(dAtA []byte) error { } m.Value = &RequestUnion_RangeStats{v} iNdEx = postIndex + case 45: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field AdminRelocateRange", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowApi + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthApi + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + v := &AdminRelocateRangeRequest{} + if err := v.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.Value = &RequestUnion_AdminRelocateRange{v} + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipApi(dAtA[iNdEx:]) @@ -27779,6 +28258,38 @@ func (m *ResponseUnion) Unmarshal(dAtA []byte) error { } m.Value = &ResponseUnion_RangeStats{v} iNdEx = postIndex + case 45: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field AdminRelocateRange", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowApi + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthApi + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + v := &AdminRelocateRangeResponse{} + if err := v.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.Value = &ResponseUnion_AdminRelocateRange{v} + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipApi(dAtA[iNdEx:]) @@ -29220,376 +29731,381 @@ var ( func init() { proto.RegisterFile("roachpb/api.proto", fileDescriptorApi) } var fileDescriptorApi = []byte{ - // 5936 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xe4, 0x5c, 0x4d, 0x6c, 0x23, 0xc9, - 0x75, 0x66, 0x93, 0x94, 0x44, 0x3e, 0x8a, 0x54, 0xab, 0x34, 0x3f, 0x1c, 0xcd, 0xec, 0x48, 0xc3, - 0xf9, 0x9f, 0xb5, 0x35, 0x9e, 0x19, 0x6f, 0xd6, 0xd9, 0xb5, 0xd7, 0x16, 0x29, 0x8d, 0xa8, 0xd1, - 0xe8, 0x67, 0x8b, 0xd4, 0xac, 0x67, 0x9d, 0x4d, 0xbb, 0xd5, 0x5d, 0xa2, 0xda, 0x43, 0x76, 0x73, - 0xba, 0x9b, 0x23, 0x69, 0x80, 0x20, 0x81, 0x73, 0x70, 0x60, 0x18, 0x46, 0x80, 0x04, 0x41, 0x80, - 0xfc, 0x19, 0xf0, 0x21, 0x40, 0x80, 0x18, 0x36, 0x60, 0x20, 0x48, 0x80, 0xd8, 0x97, 0x1c, 0x16, - 0x41, 0x0e, 0x9b, 0x00, 0x09, 0x82, 0x1c, 0x84, 0x44, 0xc9, 0xc1, 0xc8, 0x35, 0xf0, 0x65, 0x91, - 0x43, 0x50, 0x3f, 0xfd, 0x43, 0xb1, 0xf9, 0x33, 0x72, 0x2f, 0xb2, 0x41, 0x4e, 0x64, 0xbf, 0xaa, - 0xf7, 0xaa, 0xea, 0xd5, 0xab, 0x57, 0x5f, 0x55, 0xbd, 0x2a, 0x98, 0xb6, 0x2d, 0x55, 0xdb, 0x6b, - 0xef, 0xdc, 0x55, 0xdb, 0xc6, 0x42, 0xdb, 0xb6, 0x5c, 0x0b, 0x4d, 0x6b, 0x96, 0xf6, 0x8c, 0x91, - 0x17, 0x44, 0xe2, 0x2c, 0xf2, 0x72, 0xe9, 0xaa, 0xab, 0xf2, 0x6c, 0xb3, 0x67, 0x3c, 0x1a, 0xb1, - 0x6d, 0xcb, 0x76, 0x04, 0xf5, 0x9c, 0x47, 0x6d, 0x11, 0x57, 0x0d, 0xe5, 0x2e, 0x39, 0xae, 0x65, - 0xab, 0x0d, 0x72, 0x97, 0x98, 0x0d, 0xc3, 0xf4, 0x7e, 0x68, 0xbe, 0x17, 0x9a, 0x26, 0xf2, 0x5c, - 0x1d, 0x94, 0xe7, 0x81, 0xc8, 0x54, 0xec, 0xb8, 0x46, 0xf3, 0xee, 0x5e, 0x53, 0xbb, 0xeb, 0x1a, - 0x2d, 0xe2, 0xb8, 0x6a, 0xab, 0x2d, 0x52, 0xe6, 0x59, 0x8a, 0x6b, 0xab, 0x9a, 0x61, 0x36, 0xee, - 0xda, 0x44, 0xb3, 0x6c, 0x9d, 0xe8, 0x8a, 0xd3, 0x56, 0x4d, 0xaf, 0xca, 0x0d, 0xab, 0x61, 0xb1, - 0xbf, 0x77, 0xe9, 0x3f, 0x4e, 0x2d, 0xfd, 0x3a, 0x64, 0xb1, 0x6a, 0x36, 0xc8, 0xaa, 0xb9, 0x6b, - 0xa1, 0x2f, 0x42, 0x5a, 0x27, 0x8e, 0x56, 0x94, 0xe6, 0xa5, 0x5b, 0xb9, 0xfb, 0xa5, 0x85, 0x1e, - 0x5d, 0x2c, 0xb0, 0xbc, 0x4b, 0xc4, 0xd1, 0x6c, 0xa3, 0xed, 0x5a, 0x76, 0x39, 0xfd, 0xe1, 0xd1, - 0x5c, 0x02, 0x33, 0x2e, 0xf4, 0x79, 0x18, 0x6b, 0x12, 0xd5, 0x21, 0xc5, 0x24, 0x63, 0x2f, 0x46, - 0xb0, 0x3f, 0xa6, 0xe9, 0x82, 0x89, 0x67, 0x2e, 0xbd, 0x84, 0x3c, 0x26, 0xcf, 0x3b, 0xc4, 0x71, - 0xab, 0x44, 0xd5, 0x89, 0x8d, 0x2e, 0x40, 0xea, 0x19, 0x39, 0x2c, 0xa6, 0xe6, 0xa5, 0x5b, 0x93, - 0xe5, 0x89, 0x8f, 0x8f, 0xe6, 0x52, 0x6b, 0xe4, 0x10, 0x53, 0x1a, 0x9a, 0x87, 0x09, 0x62, 0xea, - 0x0a, 0x4d, 0x4e, 0x77, 0x27, 0x8f, 0x13, 0x53, 0x5f, 0x23, 0x87, 0x68, 0x16, 0x32, 0x0e, 0x95, - 0x66, 0x6a, 0xa4, 0x38, 0x36, 0x2f, 0xdd, 0x1a, 0xc3, 0xfe, 0xf7, 0x5b, 0xe9, 0x9f, 0x7d, 0x6f, - 0x4e, 0x7a, 0x94, 0xce, 0x48, 0x72, 0xf2, 0x51, 0x3a, 0x93, 0x94, 0x53, 0xa5, 0xef, 0xa4, 0xa0, - 0x80, 0x89, 0xd3, 0xb6, 0x4c, 0x87, 0x88, 0xd2, 0x3f, 0x07, 0x29, 0xf7, 0xc0, 0x64, 0xa5, 0xe7, - 0xee, 0x5f, 0x8e, 0x68, 0x42, 0xdd, 0x56, 0x4d, 0x47, 0xd5, 0x5c, 0xc3, 0x32, 0x31, 0xcd, 0x8a, - 0xbe, 0x00, 0x39, 0x9b, 0x38, 0x9d, 0x16, 0x61, 0xca, 0x66, 0x15, 0xcb, 0xdd, 0x3f, 0x1f, 0xc1, - 0x59, 0x6b, 0xab, 0x26, 0x06, 0x9e, 0x97, 0xfe, 0x47, 0x17, 0x20, 0x63, 0x76, 0x5a, 0xb4, 0x39, - 0x0e, 0xab, 0x6c, 0x0a, 0x4f, 0x98, 0x9d, 0xd6, 0x1a, 0x39, 0x74, 0x50, 0x05, 0x72, 0x36, 0x55, - 0xb5, 0x62, 0x98, 0xbb, 0x96, 0x53, 0x1c, 0x9f, 0x4f, 0xdd, 0xca, 0xdd, 0xbf, 0xd4, 0xaf, 0x43, - 0x68, 0xe7, 0x09, 0xad, 0x82, 0xed, 0x11, 0x1c, 0x54, 0x83, 0xbc, 0xa8, 0x99, 0x4d, 0x54, 0xc7, - 0x32, 0x8b, 0x13, 0xf3, 0xd2, 0xad, 0xc2, 0xfd, 0x85, 0x28, 0x31, 0x5d, 0x5a, 0xa0, 0x9f, 0x9d, - 0x16, 0xc1, 0x8c, 0x0b, 0x4f, 0xda, 0xa1, 0xaf, 0xd2, 0x53, 0x98, 0x0c, 0xa7, 0x22, 0x04, 0x05, - 0xbc, 0x5c, 0xdb, 0x5e, 0x5f, 0x56, 0xb6, 0x37, 0xd6, 0x36, 0x36, 0xdf, 0xdb, 0x90, 0x13, 0xe8, - 0x0c, 0xc8, 0x82, 0xb6, 0xb6, 0xfc, 0x54, 0x79, 0xbc, 0xba, 0xbe, 0x5a, 0x97, 0x25, 0x74, 0x01, - 0xce, 0x0a, 0x2a, 0x5e, 0xdc, 0x58, 0x59, 0x56, 0xca, 0x9b, 0xdb, 0x1b, 0x4b, 0x8b, 0xf8, 0xa9, - 0x9c, 0x9c, 0x4d, 0xff, 0xd6, 0xf7, 0x2f, 0x27, 0x4a, 0x4f, 0x00, 0x56, 0x88, 0x2b, 0xac, 0x01, - 0x95, 0x61, 0x7c, 0x8f, 0xd5, 0x46, 0x98, 0xe3, 0x7c, 0x64, 0xb5, 0x43, 0x96, 0x53, 0xce, 0x50, - 0x0d, 0x7c, 0x74, 0x34, 0x27, 0x61, 0xc1, 0xc9, 0xbb, 0xbc, 0xf4, 0x53, 0x09, 0x72, 0x4c, 0x30, - 0x6f, 0x23, 0xaa, 0x9c, 0x90, 0x7c, 0x65, 0xa8, 0x42, 0x7a, 0x45, 0xa3, 0x05, 0x18, 0x7b, 0xa1, - 0x36, 0x3b, 0x83, 0xac, 0xfd, 0x09, 0x4d, 0xc7, 0x3c, 0x1b, 0x7a, 0x1b, 0x26, 0x0d, 0xd3, 0x25, - 0xa6, 0xab, 0x70, 0xb6, 0xd4, 0x10, 0xb6, 0x1c, 0xcf, 0xcd, 0x3e, 0x4a, 0x7f, 0x25, 0x01, 0x6c, - 0x75, 0xe2, 0x54, 0x0d, 0x1d, 0xad, 0x23, 0xd5, 0xdf, 0x1b, 0xad, 0xbc, 0x15, 0xe7, 0x60, 0xdc, - 0x30, 0x9b, 0x86, 0xc9, 0xeb, 0x9f, 0xc1, 0xe2, 0x0b, 0x9d, 0x81, 0xb1, 0x9d, 0xa6, 0x61, 0xea, - 0xcc, 0xfc, 0x33, 0x98, 0x7f, 0x08, 0xf5, 0x63, 0xc8, 0xb1, 0xba, 0xc7, 0xa8, 0xfd, 0xd2, 0x7f, - 0x48, 0x70, 0xb6, 0x62, 0x99, 0xba, 0x41, 0xc7, 0xa1, 0xda, 0xfc, 0x54, 0xe8, 0xe6, 0x0d, 0xc8, - 0x92, 0x83, 0xf6, 0x88, 0xdd, 0x9b, 0x21, 0x07, 0x6d, 0xf6, 0x6f, 0xa0, 0xea, 0x3e, 0x80, 0x73, - 0x27, 0x5b, 0x19, 0xa7, 0x16, 0xff, 0x41, 0x82, 0xc2, 0xaa, 0x69, 0xb8, 0x9f, 0x0a, 0xf5, 0xf9, - 0x7a, 0x48, 0x85, 0xf4, 0x80, 0xee, 0x80, 0xbc, 0xab, 0x1a, 0xcd, 0x4d, 0xb3, 0x6e, 0xb5, 0x76, - 0x1c, 0xd7, 0x32, 0x89, 0x23, 0x14, 0xd5, 0x43, 0x17, 0x3a, 0x7b, 0x02, 0x53, 0x7e, 0x9b, 0xe2, - 0x54, 0xd6, 0x4b, 0x90, 0x57, 0x4d, 0xcd, 0x26, 0x2d, 0x62, 0xc6, 0xaa, 0xad, 0x4b, 0x90, 0x35, - 0x3c, 0xb9, 0x4c, 0x63, 0x29, 0x1c, 0x10, 0x44, 0x9b, 0x3a, 0x30, 0x1d, 0x2a, 0x3b, 0x4e, 0x37, - 0x76, 0x11, 0xb2, 0x26, 0xd9, 0x57, 0x82, 0xfe, 0x4a, 0xe1, 0x8c, 0x49, 0xf6, 0xb9, 0xdb, 0x79, - 0x0a, 0xf9, 0x25, 0xd2, 0x24, 0x2e, 0x89, 0xdf, 0x27, 0x6f, 0x43, 0xc1, 0x13, 0x1d, 0x67, 0x27, - 0xfd, 0x91, 0x04, 0x48, 0xc8, 0xa5, 0xf3, 0x60, 0x9c, 0xfd, 0x34, 0x47, 0xe7, 0x79, 0xb7, 0x63, - 0x9b, 0x7c, 0xc2, 0xe6, 0x56, 0x0a, 0x9c, 0xc4, 0xe6, 0xec, 0xc0, 0x37, 0xa6, 0xc3, 0xbe, 0xd1, - 0xc7, 0x1d, 0x14, 0x71, 0xec, 0xc3, 0x4c, 0x57, 0xf5, 0xe2, 0xed, 0xca, 0x34, 0xab, 0x59, 0x72, - 0x3e, 0x15, 0x86, 0x46, 0x8c, 0x58, 0xfa, 0x00, 0xa6, 0x2b, 0x4d, 0xa2, 0xda, 0x71, 0xab, 0x45, - 0x74, 0xe7, 0x53, 0x40, 0x61, 0xf1, 0x71, 0x76, 0xa9, 0x01, 0xb9, 0x9a, 0xa6, 0x9a, 0x9b, 0x6d, - 0xea, 0x04, 0x1d, 0xf4, 0x00, 0xce, 0x39, 0xae, 0xd5, 0x56, 0x54, 0x57, 0xe1, 0x08, 0x69, 0xc7, - 0xea, 0x98, 0xba, 0x6a, 0x1f, 0xb2, 0x32, 0x32, 0x78, 0x86, 0xa6, 0x2e, 0xba, 0xac, 0x22, 0x65, - 0x91, 0x44, 0xfb, 0xae, 0x65, 0x98, 0x0a, 0x05, 0x32, 0x4d, 0xd7, 0x11, 0x76, 0x0e, 0x2d, 0xc3, - 0xc4, 0x9c, 0x22, 0x5a, 0xf1, 0x7d, 0x89, 0x97, 0x15, 0xa7, 0xd9, 0xbc, 0x03, 0x39, 0x47, 0x53, - 0x4d, 0x65, 0xd7, 0xb2, 0x5b, 0xaa, 0xcb, 0x4c, 0xa3, 0x70, 0xff, 0xb5, 0x28, 0x78, 0xa8, 0xa9, - 0xe6, 0x43, 0x96, 0x09, 0x83, 0xe3, 0xff, 0x0f, 0x5b, 0xcf, 0xa3, 0x74, 0x26, 0x25, 0xa7, 0x4b, - 0x3f, 0x97, 0x60, 0x92, 0xd7, 0x32, 0x4e, 0xeb, 0x79, 0x03, 0xd2, 0xb6, 0xb5, 0xcf, 0xad, 0x27, - 0x77, 0xff, 0x62, 0x84, 0x88, 0x35, 0x72, 0x18, 0x76, 0xdb, 0x2c, 0x3b, 0x2a, 0x83, 0x00, 0x2a, - 0x0a, 0xe3, 0x4e, 0x8d, 0xca, 0x0d, 0x9c, 0x0b, 0x53, 0x19, 0xd7, 0xa1, 0xb0, 0xa3, 0xba, 0xda, - 0x1e, 0xed, 0x1f, 0x56, 0x49, 0x8e, 0xee, 0x71, 0x9e, 0x51, 0xbd, 0x9a, 0x97, 0xfe, 0x4c, 0x02, - 0x84, 0xc9, 0x0b, 0x62, 0x3b, 0xe4, 0xd3, 0xdf, 0x49, 0xff, 0x2d, 0xc1, 0x4c, 0x57, 0x65, 0xff, - 0x7f, 0xf5, 0xd5, 0x6f, 0x48, 0x70, 0xbe, 0xb2, 0x47, 0xb4, 0x67, 0x15, 0xcb, 0x74, 0x0c, 0xc7, - 0x25, 0xa6, 0x76, 0x18, 0x67, 0x87, 0x5d, 0x84, 0xec, 0xbe, 0xe1, 0xee, 0x29, 0xba, 0xb1, 0xbb, - 0xcb, 0x86, 0x73, 0x06, 0x67, 0x28, 0x61, 0xc9, 0xd8, 0xdd, 0x15, 0x83, 0x59, 0x81, 0x62, 0x6f, - 0x0d, 0xe2, 0x05, 0x04, 0x67, 0x31, 0xd1, 0xac, 0x56, 0xbb, 0xe3, 0x92, 0x9a, 0xab, 0xba, 0x4e, - 0x9c, 0x0d, 0x3c, 0x0f, 0x13, 0xba, 0x7d, 0xa8, 0xd8, 0x1d, 0x53, 0x34, 0x6f, 0x5c, 0xb7, 0x0f, - 0x71, 0xc7, 0x14, 0x8d, 0xfb, 0x0b, 0x09, 0xce, 0x9d, 0x2c, 0x3c, 0x4e, 0x0b, 0xfb, 0x2a, 0xe4, - 0x54, 0x5d, 0x27, 0xba, 0xa2, 0x93, 0xa6, 0xab, 0x0a, 0x20, 0x77, 0x2f, 0x24, 0x49, 0xec, 0x56, - 0x2c, 0xf0, 0x6d, 0x8a, 0x05, 0x6f, 0xb7, 0x62, 0x61, 0xfd, 0x49, 0xa5, 0xc2, 0xea, 0xb3, 0x44, - 0x19, 0x3d, 0x03, 0x62, 0xb2, 0x18, 0xa5, 0xa4, 0xc1, 0xf9, 0x32, 0x69, 0x18, 0x66, 0x78, 0x1d, - 0x1d, 0xfb, 0x74, 0xa4, 0x40, 0xb1, 0xb7, 0x90, 0x38, 0xfb, 0xfe, 0xef, 0x53, 0x70, 0x76, 0xd9, - 0xd4, 0x3f, 0x99, 0x46, 0x50, 0x24, 0xa1, 0x59, 0xad, 0x96, 0xe1, 0x7a, 0x7d, 0xcf, 0xbf, 0xd0, - 0x2f, 0x43, 0x46, 0x27, 0xaa, 0xee, 0xaf, 0xbf, 0x72, 0x5d, 0x3e, 0xaa, 0xe3, 0x1a, 0xcd, 0x85, - 0xbd, 0xa6, 0xb6, 0x50, 0xf7, 0xf6, 0x86, 0xb0, 0x9f, 0x1d, 0x7d, 0x1d, 0xce, 0xd3, 0x51, 0x6c, - 0x9b, 0x6a, 0x53, 0xe1, 0xd2, 0x14, 0xd7, 0x36, 0x1a, 0x0d, 0x62, 0x8b, 0x1d, 0x8b, 0x5b, 0x11, - 0xf5, 0x5c, 0x15, 0x1c, 0x15, 0xc6, 0x50, 0xe7, 0xf9, 0xf1, 0x59, 0x23, 0x8a, 0x8c, 0xbe, 0xe2, - 0x2f, 0x70, 0x9d, 0xb6, 0x6a, 0x3a, 0xc5, 0x31, 0xe6, 0x5e, 0xfa, 0x6d, 0x84, 0x08, 0xcb, 0x10, - 0x0e, 0x89, 0x52, 0x1c, 0x74, 0x97, 0x22, 0xac, 0xe7, 0x1d, 0xc3, 0x26, 0xca, 0xbd, 0xb6, 0x56, - 0x1c, 0xa7, 0x6d, 0x2f, 0x17, 0x8e, 0x8f, 0xe6, 0x00, 0x73, 0xf2, 0xbd, 0xad, 0x0a, 0x45, 0x5c, - 0xfc, 0x7f, 0x5b, 0x43, 0xb7, 0x40, 0x36, 0x2d, 0xc5, 0x26, 0xbb, 0x36, 0x71, 0xf6, 0x44, 0xb1, - 0x19, 0xa6, 0xb1, 0x82, 0x69, 0x61, 0x4e, 0xe6, 0xa2, 0xcf, 0xc1, 0x78, 0xdb, 0x32, 0x1c, 0xcb, - 0x2c, 0x66, 0xb9, 0x46, 0xf9, 0x97, 0xef, 0xb8, 0x27, 0xe4, 0x4c, 0xe9, 0x77, 0x24, 0x38, 0x77, - 0xb2, 0x4f, 0xe3, 0x1c, 0x53, 0xb7, 0x40, 0xb6, 0x4c, 0xa2, 0xb4, 0xf7, 0x54, 0x87, 0x88, 0x3e, - 0x10, 0x48, 0xb1, 0x60, 0x99, 0x64, 0x8b, 0x92, 0xb9, 0x46, 0xbb, 0x26, 0x92, 0xdf, 0x94, 0x60, - 0x7a, 0x51, 0x6f, 0x19, 0x66, 0xad, 0xdd, 0x34, 0x62, 0x5d, 0x78, 0x5c, 0x83, 0xac, 0x43, 0x65, - 0xb2, 0xfd, 0xb4, 0x64, 0xf7, 0x7e, 0x5a, 0x86, 0xa5, 0xac, 0x91, 0xc3, 0x00, 0xdf, 0x85, 0x2b, - 0x11, 0xe7, 0x50, 0xfa, 0x40, 0xb4, 0x6f, 0x9d, 0xd8, 0x9f, 0x10, 0x32, 0x0d, 0x8b, 0x8f, 0xb3, - 0xe6, 0xdf, 0x96, 0xe0, 0x02, 0x93, 0xcd, 0x4c, 0x66, 0x97, 0xd8, 0x6c, 0x7b, 0x33, 0xce, 0x2e, - 0xba, 0x0a, 0xe3, 0xae, 0x6a, 0x37, 0x08, 0x77, 0x04, 0x63, 0xe5, 0xdc, 0xc7, 0x47, 0x73, 0x13, - 0x35, 0xd7, 0xb2, 0xc9, 0xea, 0x12, 0x16, 0x49, 0xa2, 0x9d, 0x2a, 0xcc, 0x46, 0xd5, 0x25, 0xce, - 0xf6, 0xfe, 0x97, 0x24, 0xca, 0xa8, 0xec, 0x71, 0x98, 0xdf, 0x6e, 0x1a, 0x9a, 0x1a, 0xeb, 0xb4, - 0xb7, 0x0c, 0x39, 0x8d, 0x09, 0x57, 0xdc, 0xc3, 0x36, 0x5f, 0x90, 0x16, 0xee, 0x5f, 0x8b, 0x14, - 0xc4, 0x0a, 0xe7, 0x35, 0xa9, 0x1f, 0xb6, 0x09, 0x06, 0xcd, 0xff, 0x8f, 0x96, 0x60, 0x82, 0x2b, - 0xc7, 0x43, 0x39, 0x03, 0x44, 0xd0, 0x81, 0x5e, 0x67, 0x99, 0x85, 0x4f, 0xf2, 0x58, 0x85, 0x62, - 0x77, 0xe0, 0x62, 0x64, 0xa3, 0xe3, 0xd4, 0xec, 0xef, 0x49, 0x30, 0x53, 0x25, 0xaa, 0xed, 0xee, - 0x10, 0xd5, 0xad, 0x1f, 0xc4, 0x3a, 0x99, 0xbc, 0x01, 0x29, 0xd3, 0xda, 0x17, 0x53, 0xf8, 0xe0, - 0xf9, 0x42, 0x28, 0x80, 0xe6, 0x17, 0x8d, 0xff, 0x1a, 0x9c, 0xe9, 0xae, 0x57, 0x9c, 0xad, 0xfe, - 0x51, 0x0a, 0xb2, 0x2b, 0x95, 0x38, 0xdb, 0xfa, 0x45, 0xb1, 0x04, 0xe6, 0x9d, 0x1e, 0x75, 0x80, - 0xe1, 0x97, 0xb7, 0xb0, 0x52, 0x59, 0x23, 0x87, 0x1e, 0x3e, 0xa6, 0x5c, 0x68, 0x11, 0xb2, 0xee, - 0x1e, 0x9d, 0x33, 0xac, 0xa6, 0x2e, 0x66, 0xc5, 0x91, 0xf4, 0x15, 0x70, 0xa1, 0x26, 0x9c, 0x75, - 0x0f, 0x4c, 0x36, 0x15, 0x29, 0x0d, 0x4d, 0x09, 0xc4, 0x8d, 0x8d, 0x22, 0x6e, 0x96, 0x8a, 0x3b, - 0x3e, 0x9a, 0x43, 0xf5, 0x03, 0x93, 0xce, 0x5b, 0x2b, 0x95, 0xba, 0x27, 0x00, 0x23, 0x57, 0xd0, - 0x34, 0x9f, 0x36, 0xfb, 0x0c, 0xc6, 0x58, 0x2b, 0xbc, 0x33, 0x13, 0x29, 0xe2, 0xcc, 0x84, 0x36, - 0xca, 0x2b, 0xe0, 0x55, 0x8c, 0x20, 0xe0, 0xe2, 0xa6, 0x20, 0x0c, 0xe2, 0x5d, 0x00, 0xaa, 0xc2, - 0x38, 0xcd, 0xe0, 0x27, 0x29, 0x28, 0x6c, 0x75, 0x9c, 0xbd, 0x98, 0xed, 0xbe, 0x02, 0xd0, 0xee, - 0x38, 0x7b, 0xc4, 0x56, 0xdc, 0x03, 0x53, 0xb4, 0x7c, 0xc8, 0x81, 0x8e, 0xd7, 0x74, 0xce, 0x57, - 0x3f, 0x30, 0xd1, 0xa6, 0x10, 0x42, 0x94, 0xe0, 0x54, 0xe8, 0xce, 0x08, 0x30, 0xb8, 0x7e, 0x60, - 0xae, 0x13, 0x1f, 0xff, 0x72, 0x81, 0x84, 0x0a, 0xfc, 0x22, 0x4c, 0xd0, 0x0f, 0xc5, 0xb5, 0x5e, - 0xc5, 0xc2, 0xc6, 0x29, 0x4f, 0xdd, 0xf2, 0xc6, 0xf2, 0xd8, 0xab, 0x8d, 0x65, 0xf4, 0x36, 0x64, - 0x79, 0xa1, 0xd4, 0xa7, 0x8e, 0x33, 0x9f, 0x1a, 0xa5, 0x09, 0xd1, 0x09, 0xcc, 0x9b, 0x66, 0x58, - 0x89, 0xd4, 0x97, 0x9e, 0x81, 0xb1, 0x5d, 0xcb, 0xd6, 0x08, 0x3b, 0x3d, 0xca, 0x60, 0xfe, 0xe1, - 0x03, 0xa7, 0x8c, 0x9c, 0x2d, 0xfd, 0x81, 0x04, 0x53, 0x7e, 0x07, 0xc6, 0x89, 0x98, 0x2a, 0x5d, - 0xda, 0x7f, 0xf5, 0x2e, 0xa4, 0x1a, 0x2f, 0xfd, 0x20, 0x09, 0x53, 0xef, 0x76, 0x88, 0x7d, 0x18, - 0xb3, 0x7d, 0x95, 0xf9, 0x49, 0x61, 0xf2, 0x94, 0x36, 0xc1, 0xce, 0x0e, 0x6f, 0xc0, 0xd4, 0xbe, - 0x6a, 0xb8, 0xca, 0xae, 0x65, 0x2b, 0x9d, 0xb6, 0xae, 0xba, 0xde, 0xb9, 0x4a, 0x9e, 0x92, 0x1f, - 0x5a, 0xf6, 0x36, 0x23, 0x22, 0x02, 0xe8, 0x99, 0x69, 0xed, 0x9b, 0x0a, 0x25, 0x1b, 0x66, 0x83, - 0xea, 0xc3, 0x29, 0xa6, 0xd9, 0x46, 0xdf, 0x9b, 0xff, 0x72, 0x34, 0xf7, 0xa0, 0x61, 0xb8, 0x7b, - 0x9d, 0x9d, 0x05, 0xcd, 0x6a, 0xdd, 0xf5, 0x2b, 0xa2, 0xef, 0x04, 0xff, 0xef, 0xb6, 0x9f, 0x35, - 0xee, 0xb2, 0x23, 0xe2, 0x4e, 0xc7, 0xd0, 0x17, 0xb6, 0xb7, 0x57, 0x97, 0xb0, 0xcc, 0x44, 0xbe, - 0xc7, 0x25, 0xd6, 0x0f, 0x4c, 0x6f, 0xc2, 0xfb, 0x58, 0x02, 0x39, 0x50, 0x58, 0x9c, 0xfd, 0xb9, - 0x0c, 0xb9, 0xe7, 0x1d, 0x62, 0x1b, 0x44, 0x7f, 0xe5, 0x0e, 0x05, 0xc1, 0x48, 0xc7, 0xd0, 0xfb, - 0x30, 0xd9, 0xa5, 0x87, 0xd4, 0x2f, 0xa6, 0x87, 0xdc, 0x7e, 0xa0, 0x82, 0xd2, 0x4f, 0x92, 0x80, - 0x58, 0xe3, 0x57, 0xf9, 0x9e, 0xc7, 0xa7, 0xcc, 0x60, 0x9e, 0x00, 0x18, 0xbb, 0x4a, 0xcb, 0x70, - 0x1c, 0xc3, 0x6c, 0x30, 0x5b, 0x29, 0xdc, 0x7f, 0x33, 0xa2, 0x2e, 0xbd, 0x4d, 0x58, 0x58, 0xdd, - 0x5d, 0xe7, 0x6c, 0x65, 0xb2, 0xa7, 0xbe, 0x30, 0x2c, 0x1b, 0x67, 0x0d, 0x8f, 0x54, 0x2a, 0xc3, - 0x74, 0x4f, 0x3a, 0x2a, 0x00, 0x2c, 0x6d, 0x2a, 0x1b, 0x9b, 0xf5, 0xea, 0xea, 0xc6, 0x8a, 0x9c, - 0x40, 0x32, 0x4c, 0xe2, 0xe5, 0xfa, 0x36, 0xde, 0x50, 0x96, 0x31, 0xde, 0xc4, 0xb2, 0x84, 0x72, - 0x30, 0xb1, 0x85, 0x97, 0x9f, 0x2c, 0x6f, 0xd4, 0xe5, 0xa4, 0xb0, 0x9e, 0x5f, 0x83, 0x99, 0xae, - 0xc2, 0xe3, 0xb4, 0x9f, 0x2b, 0x30, 0xb9, 0x6b, 0x75, 0x4c, 0x5d, 0xe1, 0xab, 0x46, 0xb1, 0x3a, - 0xce, 0x31, 0x1a, 0x2f, 0xaf, 0xf4, 0xad, 0x24, 0x9c, 0xc1, 0xc4, 0xb1, 0x9a, 0x2f, 0x48, 0xfc, - 0x3d, 0xb8, 0x09, 0x62, 0x2b, 0x4c, 0xf9, 0x45, 0x3a, 0x32, 0xcb, 0x65, 0xf0, 0xd9, 0x60, 0xdc, - 0x71, 0x55, 0xb7, 0xe3, 0x88, 0xae, 0xbc, 0x36, 0x78, 0x2c, 0xd4, 0x58, 0x5e, 0x2c, 0x78, 0x42, - 0x8b, 0xda, 0x74, 0xef, 0xa2, 0xb6, 0xf4, 0x2b, 0x70, 0xf6, 0x84, 0x22, 0xe2, 0x9c, 0xb4, 0xff, - 0x29, 0x09, 0x17, 0xba, 0xc5, 0xc7, 0x7d, 0xde, 0xf2, 0x7f, 0x43, 0xd9, 0xa8, 0x0a, 0xf9, 0x96, - 0x61, 0x2a, 0x01, 0xc6, 0x7a, 0x85, 0xc9, 0x79, 0x92, 0xae, 0xd7, 0xba, 0x61, 0x16, 0x5d, 0xc7, - 0x45, 0xe9, 0x35, 0xce, 0xbe, 0xfb, 0xae, 0x04, 0x93, 0x71, 0xaf, 0xb6, 0x4f, 0x77, 0xe8, 0x2b, - 0xda, 0x5c, 0x87, 0xfc, 0x27, 0xb0, 0x3c, 0xff, 0x53, 0x09, 0x50, 0xdd, 0xee, 0x98, 0x9a, 0xea, - 0x92, 0xc7, 0x56, 0x23, 0xce, 0xc6, 0x9e, 0x81, 0x31, 0xc3, 0xd4, 0xc9, 0x01, 0x6b, 0x6c, 0x1a, - 0xf3, 0x0f, 0x74, 0x0f, 0x32, 0x22, 0x68, 0x87, 0x1f, 0x62, 0xa7, 0xca, 0xe7, 0x8e, 0x8f, 0xe6, - 0x26, 0x78, 0x88, 0xce, 0xd2, 0xc7, 0xc1, 0x5f, 0x3c, 0xc1, 0xa3, 0x74, 0xbc, 0x63, 0xfe, 0xf7, - 0x61, 0xa6, 0xab, 0xa2, 0x71, 0x6a, 0xe1, 0x07, 0x49, 0x98, 0x11, 0xcd, 0x89, 0x7d, 0x7b, 0xe2, - 0x54, 0x11, 0x5f, 0xe8, 0x4b, 0x00, 0x6d, 0x9b, 0xbc, 0x50, 0x38, 0x6b, 0x6a, 0x24, 0xd6, 0x2c, - 0xe5, 0x60, 0x04, 0xf4, 0x55, 0x98, 0xa2, 0x03, 0xae, 0x6d, 0x5b, 0x6d, 0xcb, 0xa1, 0x48, 0xc2, - 0x19, 0x0d, 0x49, 0x4f, 0x1f, 0x1f, 0xcd, 0xe5, 0xd7, 0x0d, 0x73, 0x4b, 0x30, 0xd6, 0x6b, 0x98, - 0x8e, 0x5c, 0xff, 0xd3, 0x83, 0x3f, 0xff, 0x28, 0xc1, 0x99, 0x4f, 0x6c, 0x43, 0xe7, 0x7f, 0x43, - 0x63, 0xfe, 0x7c, 0x20, 0xb3, 0xcf, 0x55, 0x73, 0xd7, 0x8a, 0x7f, 0x9b, 0xed, 0xbb, 0x12, 0x4c, - 0x87, 0xc4, 0xc7, 0x39, 0xeb, 0x9f, 0x2e, 0xae, 0xf0, 0x6b, 0x14, 0x07, 0x84, 0xcd, 0x3e, 0xce, - 0x41, 0xf5, 0xc7, 0x12, 0x9c, 0xab, 0xf0, 0xc3, 0x17, 0x76, 0xc6, 0xe4, 0x74, 0x5a, 0x71, 0x5a, - 0x49, 0x11, 0x26, 0x5e, 0x10, 0xdb, 0x31, 0x2c, 0x3e, 0xef, 0xe5, 0xb1, 0xf7, 0xc9, 0xe2, 0x1b, - 0x4d, 0xb5, 0xed, 0xec, 0x59, 0xde, 0xde, 0xb1, 0xff, 0xed, 0x2f, 0xc9, 0x52, 0x72, 0xba, 0xf4, - 0xb7, 0x12, 0x9c, 0xef, 0xa9, 0x60, 0x9c, 0x9d, 0xf2, 0x0d, 0xc8, 0x69, 0x42, 0x30, 0x75, 0x77, - 0x7c, 0xfb, 0x78, 0x95, 0x66, 0x3b, 0x25, 0x0c, 0x3f, 0x3e, 0x9a, 0x03, 0xaf, 0xaa, 0xab, 0x4b, - 0x18, 0x3c, 0xe9, 0xab, 0x7a, 0xe9, 0xe7, 0x13, 0x90, 0x5f, 0x3e, 0x68, 0x5b, 0xb6, 0x5b, 0xe3, - 0x73, 0x3e, 0x5a, 0x82, 0x4c, 0xdb, 0xb6, 0x5e, 0x18, 0x5e, 0x23, 0x0a, 0x91, 0xc7, 0x17, 0x5d, - 0x3c, 0x5b, 0x22, 0x3f, 0xf6, 0x39, 0x11, 0x86, 0xec, 0x63, 0x4b, 0x53, 0x9b, 0x0f, 0x8d, 0xa6, - 0x67, 0x5c, 0x0b, 0xc3, 0xc4, 0x2c, 0xf8, 0x1c, 0x5b, 0xaa, 0xbb, 0xe7, 0x8d, 0x35, 0x9f, 0x88, - 0x56, 0x20, 0x53, 0x75, 0xdd, 0x36, 0x4d, 0x14, 0x03, 0xf5, 0xfa, 0x50, 0x91, 0x94, 0x41, 0x48, - 0xf2, 0x99, 0x11, 0x86, 0xe9, 0x15, 0xcb, 0x6a, 0x34, 0x49, 0xa5, 0x69, 0x75, 0xf4, 0x8a, 0x65, - 0xee, 0x1a, 0x0d, 0xe1, 0xe8, 0xae, 0x0d, 0x95, 0xb8, 0x52, 0xa9, 0xe1, 0x5e, 0x76, 0xf4, 0x65, - 0xc8, 0xd4, 0x1e, 0x08, 0x51, 0x1c, 0xa6, 0x5c, 0x1d, 0x2a, 0xaa, 0xf6, 0x00, 0xfb, 0x4c, 0xa8, - 0x0a, 0xb9, 0xc5, 0x97, 0x1d, 0x9b, 0x08, 0x19, 0xe3, 0x4c, 0xc6, 0x8d, 0xa1, 0x32, 0x18, 0x0f, - 0x0e, 0xb3, 0xce, 0xde, 0x86, 0x7c, 0x97, 0x26, 0x11, 0x82, 0x74, 0x9b, 0x2a, 0x8d, 0x76, 0x67, - 0x16, 0xb3, 0xff, 0xdc, 0xa2, 0x67, 0x6f, 0x40, 0x9a, 0x6a, 0x85, 0x8e, 0x8a, 0x1d, 0xd5, 0x21, - 0xdb, 0xb6, 0x21, 0x32, 0x79, 0x9f, 0x22, 0xdf, 0xdf, 0x48, 0x90, 0xac, 0x3d, 0xa0, 0x40, 0x6d, - 0xa7, 0xa3, 0x3d, 0x23, 0xae, 0xc8, 0x25, 0xbe, 0x18, 0x80, 0xb3, 0xc9, 0xae, 0xc1, 0x27, 0xed, - 0x2c, 0x16, 0x5f, 0xe8, 0x35, 0x00, 0x55, 0xd3, 0x88, 0xe3, 0x28, 0x5e, 0xd8, 0x71, 0x16, 0x67, - 0x39, 0x65, 0x8d, 0x1c, 0x52, 0x36, 0x87, 0x68, 0x36, 0xe1, 0xe3, 0x2d, 0x8b, 0xc5, 0x17, 0x65, - 0x73, 0x49, 0xab, 0xad, 0xb8, 0xd6, 0x33, 0x62, 0x32, 0x6d, 0x66, 0x71, 0x96, 0x52, 0xea, 0x94, - 0x40, 0x07, 0x2a, 0x31, 0xf5, 0xb6, 0x65, 0x98, 0x2e, 0x53, 0x53, 0x16, 0xfb, 0xdf, 0x54, 0xa4, - 0x4d, 0x1a, 0x86, 0x08, 0xc8, 0xcd, 0x62, 0xf1, 0x25, 0x9a, 0x71, 0x00, 0xa9, 0x95, 0x4a, 0xed, - 0x95, 0x9b, 0x81, 0x20, 0xad, 0x76, 0x84, 0xd1, 0x65, 0x31, 0xfb, 0x8f, 0x6e, 0xc2, 0xd4, 0x8e, - 0xd1, 0x6c, 0xd2, 0x85, 0x72, 0xdb, 0xb6, 0xbe, 0x41, 0x34, 0xaf, 0x11, 0x05, 0x41, 0xde, 0xe2, - 0x54, 0x51, 0xf2, 0xb7, 0x24, 0x18, 0x63, 0x7d, 0x84, 0x2e, 0x41, 0x56, 0xb3, 0x4c, 0x57, 0x35, - 0x4c, 0x31, 0xc0, 0xb2, 0x38, 0x20, 0xf4, 0xad, 0xc2, 0x15, 0x98, 0x54, 0x35, 0xcd, 0xea, 0x98, - 0xae, 0x62, 0xaa, 0x2d, 0x22, 0xaa, 0x92, 0x13, 0xb4, 0x0d, 0xb5, 0x45, 0xd0, 0x1c, 0x78, 0x9f, - 0x7e, 0x14, 0x77, 0x16, 0x83, 0x20, 0xf9, 0xc7, 0x4d, 0x62, 0x4e, 0xf9, 0x73, 0x09, 0xa6, 0xdf, - 0xb3, 0x0d, 0x97, 0x94, 0x79, 0x6c, 0x41, 0x7c, 0x0e, 0xf6, 0x2d, 0xc8, 0xea, 0xaa, 0xab, 0xf2, - 0x88, 0xed, 0xe4, 0xc0, 0x88, 0x6d, 0x6f, 0x60, 0xd2, 0xfc, 0x2c, 0x6a, 0x1b, 0x41, 0x9a, 0xfe, - 0xe7, 0x01, 0xea, 0x98, 0xfd, 0x0f, 0x8e, 0x9a, 0xc2, 0xd5, 0x8d, 0x73, 0xc2, 0xf9, 0x66, 0xca, - 0x73, 0x81, 0x71, 0xaa, 0xe1, 0x2b, 0x30, 0x21, 0x56, 0x51, 0x42, 0x09, 0xf3, 0xc3, 0x86, 0xb2, - 0x77, 0x44, 0x22, 0xd8, 0x50, 0x19, 0xc0, 0x71, 0x55, 0xdb, 0x65, 0xeb, 0x9f, 0x91, 0xce, 0xa4, - 0x3d, 0x97, 0xc9, 0xd8, 0x28, 0x15, 0x6d, 0x40, 0xae, 0xf5, 0x42, 0xd3, 0x94, 0x5d, 0xa3, 0xe9, - 0x8a, 0xe3, 0xe8, 0xe8, 0xe0, 0x9b, 0xf5, 0x27, 0x95, 0xca, 0x43, 0x96, 0x89, 0x9f, 0x0a, 0x07, - 0xdf, 0x18, 0xa8, 0x04, 0xfe, 0x1f, 0x7d, 0x06, 0x44, 0x54, 0x9e, 0xe2, 0x38, 0x2e, 0x1b, 0x99, - 0x99, 0x72, 0xfe, 0xf8, 0x68, 0x2e, 0x8b, 0x19, 0xb5, 0x56, 0xab, 0xe3, 0x2c, 0xcf, 0x50, 0x73, - 0x5c, 0x74, 0x15, 0xf2, 0x56, 0xcb, 0x70, 0x15, 0x6f, 0xbe, 0xe1, 0xc7, 0xce, 0x78, 0x92, 0x12, - 0xbd, 0xf9, 0x48, 0xf4, 0xef, 0x77, 0x24, 0xc8, 0x97, 0x3b, 0xcd, 0x67, 0x9b, 0xed, 0x5a, 0xa7, - 0xd5, 0x52, 0xed, 0x43, 0x74, 0xd1, 0xb3, 0x23, 0xe3, 0x25, 0x61, 0xfd, 0x90, 0x12, 0x86, 0x62, - 0xbc, 0x24, 0xd4, 0x50, 0x44, 0x94, 0x0e, 0xa5, 0xf3, 0x10, 0x9c, 0xab, 0x90, 0x67, 0x6b, 0x05, - 0x85, 0x98, 0xae, 0x6d, 0x10, 0xbe, 0x14, 0x4d, 0xe1, 0x49, 0x46, 0x5c, 0xe6, 0x34, 0x74, 0x1d, - 0x0a, 0xce, 0xa1, 0xe3, 0x92, 0x96, 0xc2, 0xef, 0x71, 0x70, 0x80, 0x9b, 0xc2, 0x79, 0x4e, 0xc5, - 0x9c, 0x58, 0xfa, 0x61, 0x0a, 0x0a, 0x9e, 0x4d, 0xc4, 0x39, 0xb5, 0x97, 0x61, 0x6c, 0xd7, 0x68, - 0x12, 0x2f, 0xbc, 0xa8, 0xbf, 0x7b, 0xf7, 0x24, 0x2d, 0x50, 0x27, 0xee, 0xa1, 0x2f, 0xc6, 0x1a, - 0x87, 0x5d, 0xcc, 0x7e, 0x24, 0x41, 0x9a, 0xcd, 0xa9, 0xf7, 0x20, 0xcd, 0x06, 0xaa, 0x34, 0xca, - 0x40, 0x65, 0x59, 0xfd, 0xd9, 0x24, 0x19, 0xcc, 0x26, 0xcc, 0x93, 0xef, 0xa9, 0x6f, 0xdc, 0xbb, - 0xcf, 0x6c, 0x62, 0x12, 0x8b, 0x2f, 0x54, 0x86, 0x0c, 0x61, 0xed, 0x21, 0xba, 0x98, 0xd1, 0xa2, - 0x86, 0x41, 0x57, 0xc7, 0x7b, 0x4e, 0xc1, 0xe3, 0x43, 0x17, 0x20, 0x45, 0x8d, 0x6d, 0x82, 0x1f, - 0xc0, 0x1c, 0x1f, 0xcd, 0xa5, 0xa8, 0x99, 0x51, 0x1a, 0x07, 0x64, 0x8f, 0xd2, 0x99, 0xb4, 0x3c, - 0x56, 0xfa, 0x71, 0x1a, 0xf2, 0xab, 0xad, 0xb8, 0x87, 0xf1, 0x62, 0x77, 0x87, 0x45, 0x01, 0x8e, - 0xae, 0x42, 0x23, 0xfa, 0xab, 0xcb, 0x21, 0xa6, 0x5e, 0xcd, 0x21, 0xae, 0xd2, 0xe9, 0x4c, 0x5c, - 0x62, 0xa1, 0xe5, 0xbf, 0x3e, 0xb4, 0xfc, 0xba, 0xba, 0xd3, 0x24, 0x98, 0xf2, 0x78, 0xe7, 0x1b, - 0x5c, 0x00, 0x7a, 0x87, 0xcd, 0x9a, 0xdc, 0x68, 0xc6, 0x47, 0x37, 0x9a, 0x09, 0x62, 0xea, 0xcc, - 0x64, 0x0e, 0x84, 0xc5, 0x7c, 0x01, 0x52, 0xba, 0x31, 0x48, 0xa5, 0x51, 0x4e, 0x8d, 0xb2, 0x0c, - 0x31, 0x9c, 0x74, 0xd8, 0x70, 0xc2, 0x80, 0x7b, 0x76, 0x13, 0x20, 0x68, 0x15, 0x9a, 0x87, 0x71, - 0xab, 0xa9, 0x53, 0x60, 0x4c, 0xab, 0x90, 0x2f, 0x67, 0x8f, 0x8f, 0xe6, 0xc6, 0x36, 0x9b, 0xfa, - 0xea, 0x12, 0x1e, 0xb3, 0x9a, 0xfa, 0xaa, 0xce, 0xee, 0xfe, 0x90, 0x7d, 0x85, 0x5d, 0xb7, 0x62, - 0xe0, 0x19, 0x4f, 0x98, 0x64, 0x7f, 0x89, 0x38, 0x5a, 0x78, 0x0a, 0x14, 0x66, 0xf3, 0x27, 0x12, - 0x14, 0x3c, 0x0d, 0xc6, 0x3b, 0xd2, 0x33, 0x46, 0x4b, 0x58, 0x7e, 0xea, 0xd5, 0x2c, 0xdf, 0xe3, - 0x13, 0x71, 0xcd, 0xdf, 0x96, 0x60, 0x86, 0x07, 0x88, 0x68, 0xaa, 0x4b, 0x1d, 0x72, 0x8c, 0xe6, - 0x7d, 0x1b, 0x64, 0x5b, 0x35, 0x75, 0xab, 0x65, 0xbc, 0x24, 0x7c, 0x09, 0xec, 0x88, 0x9d, 0xdf, - 0x29, 0x9f, 0xce, 0xd6, 0x78, 0xde, 0x0a, 0xfe, 0x3f, 0x25, 0x38, 0xd3, 0x5d, 0x99, 0x38, 0x95, - 0xb6, 0x06, 0xe3, 0x6c, 0xf7, 0xc6, 0x1b, 0x6e, 0x9f, 0x8d, 0x10, 0x12, 0x55, 0x3a, 0xbf, 0xaa, - 0xe5, 0x1b, 0x3c, 0x13, 0x31, 0xfb, 0x15, 0x18, 0x63, 0xe4, 0x53, 0xf8, 0x38, 0xa1, 0xf9, 0xe7, - 0x30, 0xbd, 0xa8, 0xeb, 0xb5, 0x9a, 0xb0, 0xbe, 0xf8, 0xd4, 0xee, 0xe1, 0x9c, 0x64, 0x14, 0xce, - 0x09, 0x17, 0x19, 0x27, 0xce, 0x69, 0x43, 0x41, 0xc4, 0x6d, 0xc5, 0xbc, 0x5d, 0xb7, 0x4f, 0x81, - 0x99, 0x30, 0x1b, 0xfe, 0x11, 0x5c, 0x17, 0xf1, 0x4b, 0x8c, 0xb3, 0x25, 0x1d, 0x98, 0xf1, 0xe4, - 0xc6, 0xbd, 0x33, 0x3e, 0xa8, 0x39, 0x6c, 0xdb, 0x23, 0x5c, 0x6c, 0x9c, 0x6d, 0xfa, 0xbe, 0x04, - 0x85, 0x5a, 0x67, 0x87, 0xdf, 0xfe, 0x8b, 0xaf, 0x3d, 0x2b, 0x00, 0x4d, 0xb2, 0x2b, 0xe2, 0xf9, - 0x05, 0x12, 0x1d, 0xfd, 0xf2, 0x69, 0x96, 0xf2, 0xb2, 0x24, 0xa1, 0x82, 0x1f, 0x27, 0x61, 0xca, - 0xaf, 0x65, 0x9c, 0x23, 0xff, 0x57, 0x81, 0xc1, 0x4c, 0xc5, 0x71, 0x55, 0xd7, 0x11, 0x0e, 0xf3, - 0x33, 0xaf, 0x12, 0x13, 0x5b, 0x9e, 0x16, 0x01, 0x1e, 0x59, 0x9f, 0x84, 0xb3, 0x54, 0x24, 0xfb, - 0x8b, 0x16, 0x60, 0x86, 0xb9, 0x37, 0x45, 0x6d, 0xb7, 0x9b, 0x06, 0xd1, 0x15, 0xbe, 0xc7, 0x9c, - 0x66, 0x7b, 0xcc, 0xd3, 0x2c, 0x69, 0x91, 0xa7, 0xac, 0xb2, 0xfd, 0xe6, 0x87, 0x30, 0xb9, 0x6b, - 0x13, 0xf2, 0x92, 0x28, 0x0c, 0x34, 0xbd, 0xca, 0xc9, 0x43, 0x8e, 0x33, 0xd6, 0x28, 0x9f, 0x70, - 0x24, 0x1f, 0xc0, 0x34, 0xd3, 0x62, 0xdc, 0xa1, 0xcc, 0xa2, 0x57, 0xfe, 0x52, 0x02, 0x14, 0x96, - 0xff, 0xc9, 0x75, 0x4c, 0x32, 0xee, 0x8e, 0x29, 0xfd, 0xf0, 0x3c, 0x4c, 0x8a, 0x56, 0x6e, 0x9b, - 0x86, 0x65, 0xa2, 0x7b, 0x90, 0x6a, 0x88, 0x95, 0x79, 0x2e, 0x72, 0xa9, 0x12, 0xdc, 0x63, 0xad, - 0x26, 0x30, 0xcd, 0x4b, 0x59, 0xda, 0x1d, 0x37, 0x22, 0x02, 0x27, 0x88, 0xbe, 0x08, 0xb3, 0xb4, - 0x3b, 0x2e, 0xaa, 0xc1, 0x94, 0x16, 0xdc, 0xfe, 0x53, 0x28, 0x7b, 0xaa, 0x6f, 0xac, 0x6e, 0xe4, - 0x6d, 0xc8, 0x6a, 0x02, 0x17, 0xb4, 0xae, 0x04, 0x54, 0x09, 0x5f, 0x37, 0x4b, 0xf7, 0xdd, 0x04, - 0x3a, 0x79, 0xd5, 0xad, 0x9a, 0x08, 0xdd, 0x4a, 0x43, 0x6f, 0xc1, 0xb8, 0xce, 0xae, 0x31, 0x09, - 0x9b, 0x8b, 0x32, 0x8b, 0xae, 0x9b, 0x63, 0xd5, 0x04, 0x16, 0x1c, 0xe8, 0x11, 0x4c, 0xf2, 0x7f, - 0x62, 0xbc, 0x8f, 0xf7, 0xdd, 0x25, 0xeb, 0xbd, 0xc8, 0x55, 0x4d, 0xe0, 0x9c, 0x1e, 0x50, 0xd1, - 0xe7, 0x21, 0xed, 0x68, 0x2a, 0xdf, 0x47, 0x89, 0x8e, 0x24, 0x08, 0xdd, 0x14, 0xa9, 0xd2, 0x89, - 0x53, 0x53, 0x4d, 0xf4, 0x14, 0xa6, 0x77, 0x48, 0xc3, 0x30, 0x15, 0x37, 0x38, 0xed, 0x63, 0x71, - 0xc3, 0xdd, 0x07, 0x8c, 0x3e, 0xfe, 0x89, 0x0e, 0x57, 0xaf, 0x26, 0xb0, 0xbc, 0x73, 0x22, 0x89, - 0x76, 0x19, 0x03, 0xb0, 0x21, 0xc1, 0xd9, 0xbe, 0x5d, 0x16, 0x19, 0x40, 0x4e, 0xbb, 0x8c, 0x74, - 0x25, 0xa0, 0x15, 0xc8, 0xa9, 0x14, 0x50, 0x28, 0x2c, 0x28, 0xb7, 0x08, 0x7d, 0x37, 0x01, 0x7b, - 0xe2, 0x84, 0xab, 0x2c, 0xf6, 0xde, 0x23, 0x06, 0x82, 0x5a, 0xc4, 0x6e, 0x90, 0x62, 0x6e, 0xb0, - 0xa0, 0xf0, 0x11, 0xa1, 0x2f, 0x88, 0x11, 0xd1, 0x3a, 0xe4, 0xf7, 0xbc, 0xb0, 0x40, 0x76, 0x3c, - 0x3b, 0xd9, 0x77, 0x27, 0x30, 0x22, 0xac, 0xb1, 0x9a, 0xc0, 0x93, 0x7b, 0x21, 0x32, 0x5a, 0x80, - 0x64, 0x43, 0x2b, 0xe6, 0x99, 0x8c, 0x4b, 0x83, 0x82, 0xf6, 0xaa, 0x09, 0x9c, 0x6c, 0x68, 0x74, - 0x99, 0xc0, 0xe3, 0x99, 0x0e, 0xcc, 0x62, 0xa1, 0xaf, 0xdb, 0xe8, 0x8e, 0x29, 0xab, 0x26, 0x30, - 0x8b, 0xbc, 0xa2, 0xe5, 0x6d, 0x41, 0xc1, 0xe6, 0x67, 0xac, 0x5e, 0x24, 0x81, 0xcc, 0xa4, 0xdc, - 0x8c, 0x76, 0x3e, 0x3d, 0xc1, 0x04, 0xd5, 0x04, 0xce, 0xdb, 0x61, 0x3a, 0xfa, 0x3a, 0x9c, 0xe9, - 0x96, 0x28, 0x8c, 0x7b, 0xba, 0xc7, 0x17, 0x45, 0xcb, 0xed, 0xb6, 0x71, 0x64, 0xf7, 0x24, 0xa2, - 0x37, 0x61, 0x8c, 0xf7, 0x1a, 0x62, 0x22, 0xe7, 0xa2, 0xf6, 0x47, 0xba, 0x3b, 0x8c, 0xe7, 0xa7, - 0xe3, 0xcd, 0x15, 0x87, 0x8b, 0x4a, 0xd3, 0x6a, 0x14, 0x67, 0xfa, 0x8e, 0xb7, 0xde, 0xc3, 0x52, - 0x3a, 0xde, 0xdc, 0x80, 0x4a, 0xfb, 0xdd, 0xe6, 0x29, 0xe2, 0x2c, 0xea, 0x4c, 0xdf, 0x7e, 0x8f, - 0x38, 0x73, 0xa4, 0xfd, 0x6e, 0x87, 0xc8, 0xb4, 0x6a, 0x36, 0xbf, 0x23, 0xa5, 0xb0, 0x61, 0x7c, - 0xb6, 0x6f, 0xd5, 0x7a, 0xef, 0x7d, 0xd1, 0xaa, 0xd9, 0x01, 0x15, 0x3d, 0x01, 0x59, 0x5c, 0x87, - 0x09, 0xb6, 0x72, 0xce, 0x31, 0x79, 0xb7, 0x23, 0xbd, 0x65, 0xd4, 0xe1, 0x4d, 0x35, 0x81, 0xa7, - 0xb4, 0xee, 0x14, 0xea, 0x2c, 0x98, 0x3c, 0x45, 0x0b, 0xee, 0x11, 0x15, 0x8b, 0x7d, 0x9d, 0x45, - 0x9f, 0x4b, 0x4f, 0xd4, 0x59, 0x68, 0x27, 0x92, 0xa8, 0x19, 0x1b, 0xa6, 0xe1, 0x32, 0xc7, 0x3e, - 0xdb, 0xd7, 0x8c, 0xbb, 0x2f, 0x68, 0x53, 0x33, 0x36, 0x38, 0x85, 0x9a, 0xb1, 0x2b, 0x0e, 0x2a, - 0x45, 0x77, 0x5c, 0xea, 0x6b, 0xc6, 0x51, 0x27, 0x9a, 0xd4, 0x8c, 0xdd, 0x30, 0x9d, 0x9a, 0x31, - 0x77, 0x10, 0x27, 0xe4, 0xbe, 0xd6, 0xd7, 0x8c, 0xfb, 0xc6, 0xbf, 0x53, 0x33, 0x56, 0x7b, 0x12, - 0xd1, 0x12, 0xc5, 0x7a, 0x14, 0xe3, 0x18, 0xe6, 0xae, 0x55, 0xbc, 0xdc, 0x77, 0xfe, 0x39, 0x79, - 0x54, 0x59, 0x65, 0x40, 0x4f, 0xd0, 0xa8, 0x23, 0x63, 0xa0, 0x57, 0x61, 0xb7, 0xce, 0x8a, 0x73, - 0x7d, 0x1d, 0x59, 0xcf, 0xf6, 0x31, 0x75, 0x64, 0xfb, 0x3e, 0x91, 0x4e, 0x64, 0x7c, 0x0f, 0xa7, - 0x38, 0x3f, 0x64, 0xaf, 0x20, 0x34, 0x91, 0x71, 0x0e, 0xb4, 0x08, 0xd9, 0xe7, 0x1d, 0x62, 0x1f, - 0x32, 0x37, 0x74, 0xa5, 0x2f, 0x6a, 0x3d, 0x11, 0x7b, 0x58, 0x4d, 0xe0, 0xcc, 0x73, 0x41, 0xa2, - 0xc5, 0xf3, 0x85, 0x74, 0xb1, 0xd4, 0xb7, 0xf8, 0xae, 0xad, 0x13, 0x5a, 0x3c, 0xe7, 0x40, 0x1a, - 0x9c, 0xe5, 0x7d, 0x25, 0x02, 0xe6, 0x6d, 0x11, 0x99, 0x5e, 0xbc, 0xca, 0x44, 0xf5, 0x5d, 0x96, - 0x46, 0x06, 0xef, 0x57, 0x13, 0x78, 0x46, 0xed, 0x4d, 0xa5, 0x03, 0x5e, 0x4c, 0x3d, 0x7c, 0x31, - 0x5b, 0xbc, 0xd6, 0x77, 0xc0, 0x47, 0x2c, 0xff, 0xe9, 0x80, 0x57, 0x43, 0x64, 0x3e, 0x01, 0xe9, - 0x8a, 0xe3, 0xb8, 0x74, 0xe9, 0x58, 0xbc, 0x3e, 0x60, 0x02, 0x3a, 0xb1, 0xa4, 0xe5, 0x13, 0x90, - 0x5e, 0xe3, 0x9c, 0x54, 0x90, 0xd6, 0x24, 0xaa, 0x2d, 0xdc, 0xec, 0x8d, 0xbe, 0x82, 0x7a, 0x2e, - 0x3d, 0x53, 0x41, 0x9a, 0x4f, 0xa4, 0x13, 0xb6, 0xed, 0xdd, 0xa3, 0x13, 0xf8, 0xf1, 0x66, 0xdf, - 0x09, 0x3b, 0xf2, 0xba, 0x1f, 0x9d, 0xb0, 0xed, 0xae, 0x04, 0xf4, 0x25, 0x98, 0x10, 0x97, 0x92, - 0x8a, 0xb7, 0x06, 0xa0, 0xda, 0xf0, 0x3a, 0x97, 0x8e, 0x6b, 0xc1, 0xc3, 0xbd, 0x2c, 0xbf, 0xd3, - 0xc4, 0x9b, 0x77, 0x7b, 0x80, 0x97, 0xed, 0x59, 0x62, 0x72, 0x2f, 0x1b, 0x90, 0xa9, 0x97, 0xe5, - 0x76, 0x2a, 0xe6, 0xba, 0x3b, 0x7d, 0xbd, 0x6c, 0x6f, 0xd4, 0x20, 0xf5, 0xb2, 0xcf, 0x03, 0x2a, - 0x6d, 0x99, 0xc3, 0x97, 0x56, 0xc5, 0xd7, 0xfb, 0xb6, 0xac, 0x7b, 0x89, 0x48, 0x5b, 0x26, 0x78, - 0x68, 0xb7, 0xf1, 0x08, 0x19, 0xae, 0xe9, 0xcf, 0xf4, 0xed, 0xb6, 0x9e, 0x95, 0x48, 0xd5, 0x7b, - 0xda, 0x86, 0xe3, 0xf5, 0x09, 0x11, 0x6d, 0xf4, 0x28, 0x9d, 0x99, 0x92, 0xe5, 0x47, 0xe9, 0xcc, - 0x79, 0xb9, 0xf8, 0x28, 0x9d, 0xb9, 0x20, 0xcf, 0x3e, 0x4a, 0x67, 0x2e, 0xca, 0x97, 0x4a, 0xdf, - 0x39, 0x0f, 0x79, 0x6f, 0xdd, 0xc0, 0x31, 0xfb, 0xfd, 0x30, 0x66, 0xbf, 0xdc, 0x0f, 0xb3, 0x8b, - 0x95, 0x86, 0x00, 0xed, 0xf7, 0xc3, 0xa0, 0xfd, 0x72, 0x3f, 0xd0, 0x1e, 0xf0, 0x50, 0xd4, 0x5e, - 0xef, 0x87, 0xda, 0x6f, 0x8f, 0x80, 0xda, 0x7d, 0x51, 0x27, 0x61, 0xfb, 0x52, 0x2f, 0x6c, 0xbf, - 0x36, 0x18, 0xb6, 0xfb, 0xa2, 0x42, 0xb8, 0xfd, 0xed, 0x13, 0xb8, 0xfd, 0xca, 0x00, 0xdc, 0xee, - 0xf3, 0x7b, 0xc0, 0x7d, 0x2d, 0x12, 0xb8, 0xdf, 0x18, 0x06, 0xdc, 0x7d, 0x39, 0x5d, 0xc8, 0xfd, - 0x8d, 0x2e, 0xe4, 0x3e, 0xd7, 0x17, 0xb9, 0xfb, 0xdc, 0x1c, 0xba, 0xbf, 0xdf, 0x1f, 0xba, 0xbf, - 0x3e, 0x12, 0x74, 0xf7, 0xe5, 0xf5, 0x62, 0xf7, 0x7a, 0x3f, 0xec, 0x7e, 0x7b, 0x04, 0xec, 0x1e, - 0x74, 0xdc, 0x09, 0xf0, 0x5e, 0x8d, 0x02, 0xef, 0xd7, 0x87, 0x80, 0x77, 0x5f, 0x5a, 0x18, 0xbd, - 0x57, 0xa3, 0xd0, 0xfb, 0xf5, 0x21, 0xe8, 0xfd, 0x84, 0x24, 0x0e, 0xdf, 0x37, 0xa2, 0xe1, 0xfb, - 0xcd, 0xa1, 0xf0, 0xdd, 0x97, 0xd6, 0x8d, 0xdf, 0xef, 0x86, 0xf0, 0xfb, 0x6b, 0x7d, 0xf0, 0xbb, - 0xcf, 0x4a, 0x01, 0xfc, 0x97, 0x7b, 0x00, 0x7c, 0x69, 0x10, 0x80, 0xf7, 0x79, 0x7d, 0x04, 0xff, - 0x6e, 0x1f, 0x04, 0x7f, 0x6b, 0x38, 0x82, 0xf7, 0x85, 0x9d, 0x80, 0xf0, 0xea, 0x40, 0x08, 0xff, - 0xd9, 0x11, 0x21, 0xbc, 0x2f, 0x3d, 0x0a, 0xc3, 0x7f, 0xa1, 0x1b, 0xc3, 0xcf, 0xf7, 0xc7, 0xf0, - 0xbe, 0x18, 0x01, 0xe2, 0xd7, 0x22, 0x41, 0xfc, 0x8d, 0x61, 0x20, 0x3e, 0x18, 0x7b, 0x61, 0x14, - 0xbf, 0x11, 0x8d, 0xe2, 0x6f, 0x0e, 0x45, 0xf1, 0x41, 0xf7, 0x77, 0xc1, 0xf8, 0xb5, 0x48, 0x18, - 0x7f, 0x63, 0x18, 0x8c, 0x0f, 0x2a, 0x17, 0xc6, 0xf1, 0xef, 0xf5, 0xc5, 0xf1, 0x77, 0x46, 0xc1, - 0xf1, 0xbe, 0xd0, 0x1e, 0x20, 0xff, 0x7e, 0x7f, 0x20, 0xff, 0xfa, 0x48, 0x40, 0x3e, 0x70, 0x1d, - 0x3d, 0x48, 0xfe, 0xcb, 0x3d, 0x48, 0xbe, 0x34, 0x08, 0xc9, 0x07, 0xf6, 0xec, 0x41, 0x79, 0x75, - 0x20, 0xf0, 0xfe, 0xec, 0x88, 0xc0, 0x3b, 0x30, 0xbe, 0x08, 0xe4, 0xbd, 0x1c, 0x81, 0xbc, 0xaf, - 0x0d, 0x46, 0xde, 0xc1, 0x14, 0x12, 0x40, 0xef, 0x6a, 0x14, 0xf4, 0xbe, 0x3e, 0x04, 0x7a, 0x07, - 0x5e, 0x28, 0x84, 0xbd, 0xdf, 0x3e, 0x81, 0xbd, 0xaf, 0x0c, 0x3d, 0x68, 0x0e, 0x81, 0xef, 0x72, - 0x2f, 0xf8, 0xbe, 0x3a, 0x10, 0x7c, 0xfb, 0x12, 0x02, 0xf4, 0xfd, 0xf6, 0x09, 0xf4, 0x7d, 0x65, - 0x00, 0xfa, 0x0e, 0x2a, 0x20, 0xe0, 0xb7, 0x3e, 0x18, 0x7e, 0x2f, 0x8c, 0x0a, 0xbf, 0x7d, 0xc1, - 0x91, 0xf8, 0x7b, 0x23, 0x1a, 0x7f, 0xdf, 0x1c, 0xf1, 0xcc, 0xa9, 0x07, 0x80, 0x57, 0xa3, 0x00, - 0xf8, 0xf5, 0x21, 0x00, 0x3c, 0x3c, 0x87, 0xf8, 0x08, 0xbc, 0x1a, 0x85, 0xc0, 0xaf, 0x0f, 0x41, - 0xe0, 0x81, 0xa4, 0x10, 0x04, 0xaf, 0xf7, 0x83, 0xe0, 0xb7, 0x47, 0x80, 0xe0, 0xc1, 0xbc, 0x7b, - 0x02, 0x83, 0xbf, 0x73, 0x12, 0x83, 0x97, 0x06, 0x61, 0xf0, 0x60, 0x44, 0x7a, 0x20, 0x7c, 0x23, - 0x1a, 0x84, 0xdf, 0x1c, 0x0a, 0xc2, 0xc3, 0x4e, 0x32, 0x84, 0xc2, 0xd7, 0x22, 0x51, 0xf8, 0x8d, - 0x61, 0x28, 0x3c, 0x70, 0x92, 0x61, 0x18, 0xfe, 0xce, 0x49, 0x18, 0x5e, 0x1a, 0x04, 0xc3, 0x83, - 0xc6, 0x79, 0x38, 0xbc, 0x1a, 0x85, 0xc3, 0xaf, 0x0f, 0xc1, 0xe1, 0x41, 0xe7, 0xbd, 0x02, 0x10, - 0x7f, 0x94, 0xce, 0x5c, 0x92, 0x5f, 0x2b, 0xfd, 0xf5, 0x18, 0x8c, 0x57, 0xbd, 0x68, 0x85, 0xd0, - 0x85, 0x54, 0xe9, 0x34, 0x17, 0x52, 0xd1, 0x12, 0xed, 0x5b, 0x36, 0x42, 0x04, 0x34, 0x1f, 0x70, - 0xbd, 0xbb, 0xe7, 0xbc, 0xc8, 0x63, 0x3d, 0x45, 0xb8, 0x3e, 0x7a, 0x03, 0xf2, 0x1d, 0x87, 0xd8, - 0x4a, 0xdb, 0x36, 0x2c, 0xdb, 0x70, 0x79, 0x00, 0x9b, 0x54, 0x96, 0x3f, 0x3e, 0x9a, 0x9b, 0xdc, - 0x76, 0x88, 0xbd, 0x25, 0xe8, 0x78, 0xb2, 0x13, 0xfa, 0xf2, 0x1e, 0x15, 0x1d, 0x1b, 0xfd, 0x51, - 0xd1, 0x77, 0x41, 0xb6, 0x89, 0xaa, 0x77, 0xcd, 0x55, 0xfc, 0xe2, 0x66, 0xf4, 0xb4, 0xaa, 0xea, - 0xa1, 0xe9, 0x88, 0x5d, 0xe0, 0x9c, 0xb2, 0xbb, 0x89, 0xe8, 0x1e, 0x9c, 0x6d, 0xa9, 0x07, 0xfc, - 0x6a, 0xb2, 0x37, 0xfd, 0xb3, 0xa8, 0x8d, 0x0c, 0x0b, 0x2e, 0x42, 0x2d, 0xf5, 0x80, 0xbd, 0x50, - 0xca, 0x93, 0xd8, 0x8b, 0x66, 0xd7, 0xa1, 0xa0, 0x1b, 0x8e, 0x6b, 0x98, 0x9a, 0xf7, 0xa8, 0x07, - 0x7f, 0x3d, 0x23, 0xef, 0x51, 0xf9, 0xe3, 0x1a, 0x77, 0x60, 0x5a, 0x04, 0x5c, 0x05, 0x6f, 0x96, - 0x32, 0xa0, 0x9b, 0xa1, 0xb5, 0xa0, 0x09, 0xc1, 0x13, 0xb3, 0x15, 0x98, 0x6a, 0xa8, 0x2e, 0xd9, - 0x57, 0x0f, 0x15, 0xd3, 0xd2, 0x99, 0xee, 0x73, 0xec, 0x69, 0x83, 0x8b, 0xc7, 0x47, 0x73, 0xf9, - 0x15, 0x9e, 0xb4, 0x61, 0xe9, 0xbc, 0x07, 0xc6, 0xf9, 0x3f, 0x9c, 0x6f, 0x84, 0x12, 0x74, 0xb4, - 0x08, 0x93, 0xec, 0xb9, 0x26, 0x8b, 0xbf, 0x09, 0x26, 0xe0, 0x6b, 0xbf, 0xed, 0x7f, 0xf1, 0x72, - 0x18, 0x66, 0x4f, 0x3c, 0x79, 0xcf, 0x88, 0xdd, 0x84, 0x29, 0xd5, 0x39, 0x34, 0x35, 0xa6, 0x61, - 0x62, 0x3a, 0x1d, 0x87, 0xe1, 0xd7, 0x0c, 0x2e, 0x30, 0x72, 0xc5, 0xa3, 0x8a, 0xb7, 0x41, 0x7e, - 0x57, 0x82, 0xc9, 0xae, 0x28, 0xc4, 0xb7, 0x4f, 0x9c, 0x5b, 0x5d, 0x88, 0xc6, 0xce, 0xfd, 0x02, - 0x76, 0x32, 0xa2, 0x07, 0xbc, 0x20, 0x82, 0xb9, 0xfe, 0xd8, 0x8b, 0xad, 0x5e, 0xbd, 0xb0, 0x0b, - 0x8f, 0xed, 0xad, 0xf4, 0xef, 0x7f, 0x6f, 0x2e, 0x51, 0xfa, 0x59, 0x0a, 0xf2, 0xdd, 0xd1, 0x86, - 0xab, 0x27, 0xea, 0x15, 0xe5, 0xaf, 0xba, 0x38, 0xfa, 0xd7, 0x72, 0x09, 0xb2, 0xde, 0x23, 0x4f, - 0x5e, 0x35, 0xe7, 0x07, 0x9c, 0xce, 0x85, 0xeb, 0x19, 0x30, 0xce, 0xfe, 0x34, 0xe9, 0x8f, 0xfc, - 0x05, 0x18, 0x63, 0xcf, 0x25, 0x8b, 0xaa, 0x45, 0x05, 0xf2, 0x2f, 0xd3, 0x74, 0xcc, 0xb3, 0x51, - 0x4f, 0x51, 0x3f, 0xd5, 0xd5, 0x75, 0x9f, 0x70, 0x8a, 0xe7, 0x7c, 0x4f, 0x79, 0xc5, 0x9a, 0x9d, - 0xd5, 0x35, 0x9b, 0x44, 0x73, 0xc5, 0xab, 0xcb, 0xde, 0xa3, 0xbd, 0xd7, 0x4e, 0x8a, 0x10, 0x6f, - 0x34, 0x2f, 0x60, 0xf1, 0x46, 0x73, 0x28, 0xae, 0xa3, 0xe0, 0x8b, 0x60, 0x03, 0x8b, 0x47, 0xff, - 0x88, 0xae, 0xfe, 0xa6, 0x04, 0x32, 0x1b, 0x46, 0x0f, 0x09, 0xd1, 0x63, 0xb1, 0x42, 0x2f, 0xe4, - 0x24, 0x39, 0x72, 0xc8, 0x49, 0x49, 0x85, 0x82, 0x5f, 0x07, 0xfe, 0x7a, 0xe9, 0x80, 0x97, 0x07, - 0x4e, 0x75, 0x23, 0xac, 0xf4, 0x87, 0x12, 0xcc, 0xf8, 0x65, 0x30, 0x10, 0xce, 0x83, 0xa6, 0x4f, - 0x11, 0x04, 0x88, 0xd9, 0xcb, 0xcc, 0x74, 0x11, 0xc6, 0x6e, 0x09, 0x8d, 0x64, 0x41, 0x48, 0x1c, - 0x04, 0x83, 0x58, 0xdc, 0xe9, 0xf5, 0x1a, 0x7b, 0xb3, 0x99, 0xff, 0x77, 0x4a, 0x0f, 0x43, 0x1a, - 0x60, 0xc6, 0x4a, 0x9b, 0x39, 0x92, 0x55, 0x7b, 0xcd, 0x64, 0x99, 0x4b, 0x7f, 0x27, 0x85, 0x05, - 0xbd, 0xa0, 0x93, 0xfa, 0x03, 0x48, 0xbd, 0x50, 0x9b, 0x83, 0xce, 0xc1, 0xbb, 0x54, 0x8f, 0x69, - 0x6e, 0xf4, 0x10, 0xf8, 0x4d, 0x09, 0x1e, 0x69, 0x9e, 0xec, 0xbf, 0xf2, 0xea, 0x55, 0x29, 0x0e, - 0x71, 0xa2, 0x37, 0xbd, 0x56, 0xa4, 0x86, 0x17, 0x1f, 0x1e, 0xa4, 0x6f, 0xa5, 0x3f, 0xfc, 0xde, - 0x9c, 0x74, 0xa7, 0x06, 0x33, 0x11, 0x93, 0x10, 0x2a, 0x00, 0x54, 0x36, 0x37, 0x6a, 0xab, 0xb5, - 0xfa, 0xf2, 0x46, 0xdd, 0x7b, 0x18, 0x7a, 0x71, 0x49, 0xd9, 0xde, 0xa8, 0x6c, 0xae, 0xaf, 0xaf, - 0xd6, 0xeb, 0xcb, 0x4b, 0xb2, 0x84, 0x64, 0x98, 0x5c, 0xdd, 0x08, 0xe5, 0x13, 0xef, 0x41, 0xdf, - 0xf9, 0x25, 0x80, 0xe0, 0xbd, 0x3d, 0x2a, 0x6b, 0x6d, 0xf9, 0xa9, 0xf2, 0x64, 0xf1, 0xf1, 0xf6, - 0x72, 0x4d, 0x4e, 0x20, 0x04, 0x85, 0xf2, 0x62, 0xbd, 0x52, 0x55, 0xf0, 0x72, 0x6d, 0x6b, 0x73, - 0xa3, 0xb6, 0x2c, 0x4b, 0x82, 0xef, 0x3d, 0xc8, 0x85, 0x9e, 0x32, 0xa0, 0x19, 0xb7, 0xb6, 0x6b, - 0x55, 0xa5, 0xbe, 0xba, 0xbe, 0x5c, 0xab, 0x2f, 0xae, 0x6f, 0xc9, 0x09, 0x2a, 0x8c, 0xd1, 0x16, - 0xcb, 0x9b, 0xb8, 0x2e, 0x4b, 0xfe, 0x77, 0x7d, 0x73, 0xbb, 0x52, 0x95, 0x93, 0xfe, 0xf7, 0xbb, - 0xdb, 0xcb, 0xf8, 0xa9, 0x9c, 0x12, 0x82, 0x55, 0x38, 0x1b, 0x79, 0xa7, 0x04, 0xe5, 0x60, 0x62, - 0xdb, 0x64, 0xd7, 0xe9, 0xe5, 0x04, 0xca, 0x87, 0xae, 0x95, 0xc8, 0x12, 0xca, 0xf0, 0xeb, 0x0b, - 0x72, 0x12, 0x8d, 0x43, 0xb2, 0xf6, 0x40, 0x4e, 0xa1, 0x29, 0xc8, 0x85, 0xee, 0x66, 0xc8, 0x69, - 0x94, 0x15, 0x71, 0xf7, 0xf2, 0xd8, 0x9d, 0x2b, 0x10, 0x0a, 0x6b, 0x46, 0x00, 0xe3, 0x8f, 0x55, - 0x97, 0x38, 0xae, 0x9c, 0x40, 0x13, 0x90, 0x5a, 0x6c, 0x36, 0x65, 0xe9, 0xfe, 0x8f, 0x24, 0xc8, - 0x78, 0x2f, 0x73, 0xa1, 0xc7, 0x30, 0xc6, 0x57, 0x53, 0x73, 0xfd, 0x1d, 0x3d, 0xf3, 0x15, 0xb3, - 0xf3, 0xc3, 0x66, 0x82, 0x52, 0x02, 0xbd, 0x27, 0x5e, 0x83, 0xa7, 0xbd, 0x8c, 0xae, 0x0e, 0xb2, - 0x01, 0x4f, 0xea, 0x60, 0x43, 0xa1, 0x76, 0x5d, 0x4a, 0x7c, 0x4e, 0x2a, 0x5f, 0xf9, 0xf0, 0xdf, - 0x2e, 0x27, 0x3e, 0x3c, 0xbe, 0x2c, 0x7d, 0x74, 0x7c, 0x59, 0xfa, 0xe7, 0xe3, 0xcb, 0xd2, 0xbf, - 0x1e, 0x5f, 0x96, 0x7e, 0xfb, 0xdf, 0x2f, 0x27, 0xde, 0x9f, 0x10, 0x5c, 0x3b, 0xe3, 0xec, 0x41, - 0xfa, 0x07, 0xff, 0x13, 0x00, 0x00, 0xff, 0xff, 0x4a, 0x15, 0x31, 0x30, 0x95, 0x5f, 0x00, 0x00, + // 6008 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xe4, 0x3d, 0x4b, 0x6c, 0x23, 0xc9, + 0x75, 0x6c, 0x92, 0x92, 0xc8, 0x47, 0x91, 0xa2, 0x4a, 0xf3, 0xe1, 0x68, 0x66, 0x47, 0x1a, 0xce, + 0x7f, 0xd6, 0xab, 0xf1, 0x68, 0xbc, 0x59, 0x67, 0xd7, 0x5e, 0x5b, 0xa4, 0x34, 0xa2, 0x46, 0xa3, + 0xcf, 0x16, 0xa9, 0x59, 0xcf, 0x3a, 0x9b, 0x76, 0xab, 0xbb, 0x44, 0xb5, 0x87, 0xec, 0xe6, 0x74, + 0x37, 0x47, 0xd2, 0x00, 0x41, 0x02, 0xe7, 0xe0, 0xc0, 0x30, 0x8c, 0x20, 0x09, 0x82, 0x00, 0xf9, + 0x19, 0xf0, 0x21, 0x40, 0x82, 0x18, 0x31, 0x60, 0x20, 0x48, 0x80, 0xd8, 0x97, 0x1c, 0x16, 0x41, + 0x0e, 0x4e, 0x80, 0x04, 0x41, 0x0e, 0x42, 0xa2, 0xe4, 0x60, 0xe4, 0x92, 0x43, 0xe0, 0xcb, 0x22, + 0x87, 0xa0, 0x3e, 0xfd, 0x21, 0xd9, 0xfc, 0x8c, 0xb6, 0x07, 0xd9, 0x20, 0x27, 0xb1, 0x5f, 0xd5, + 0x7b, 0x55, 0xf5, 0xde, 0xab, 0x57, 0xef, 0x55, 0xbd, 0x2a, 0xc1, 0xb4, 0x65, 0x2a, 0xea, 0x7e, + 0x6b, 0xf7, 0xae, 0xd2, 0xd2, 0x17, 0x5a, 0x96, 0xe9, 0x98, 0x68, 0x5a, 0x35, 0xd5, 0xa7, 0x0c, + 0xbc, 0x20, 0x0a, 0x67, 0x91, 0x5b, 0x4b, 0x53, 0x1c, 0x85, 0x57, 0x9b, 0x3d, 0xe3, 0xc2, 0x88, + 0x65, 0x99, 0x96, 0x2d, 0xa0, 0xe7, 0x5c, 0x68, 0x93, 0x38, 0x4a, 0xa0, 0x76, 0xd1, 0x76, 0x4c, + 0x4b, 0xa9, 0x93, 0xbb, 0xc4, 0xa8, 0xeb, 0x86, 0xfb, 0x87, 0xd6, 0x7b, 0xae, 0xaa, 0xa2, 0xce, + 0xd5, 0x41, 0x75, 0xee, 0x8b, 0x4a, 0x85, 0xb6, 0xa3, 0x37, 0xee, 0xee, 0x37, 0xd4, 0xbb, 0x8e, + 0xde, 0x24, 0xb6, 0xa3, 0x34, 0x5b, 0xa2, 0x64, 0x9e, 0x95, 0x38, 0x96, 0xa2, 0xea, 0x46, 0xfd, + 0xae, 0x45, 0x54, 0xd3, 0xd2, 0x88, 0x26, 0xdb, 0x2d, 0xc5, 0x70, 0xbb, 0x5c, 0x37, 0xeb, 0x26, + 0xfb, 0x79, 0x97, 0xfe, 0xe2, 0xd0, 0xe2, 0x2f, 0x43, 0x1a, 0x2b, 0x46, 0x9d, 0xac, 0x19, 0x7b, + 0x26, 0xfa, 0x02, 0x24, 0x35, 0x62, 0xab, 0x05, 0x69, 0x5e, 0xba, 0x95, 0x59, 0x2c, 0x2e, 0xf4, + 0xf0, 0x62, 0x81, 0xd5, 0x5d, 0x26, 0xb6, 0x6a, 0xe9, 0x2d, 0xc7, 0xb4, 0x4a, 0xc9, 0x8f, 0x8e, + 0xe7, 0x62, 0x98, 0x61, 0xa1, 0xcf, 0xc1, 0x58, 0x83, 0x28, 0x36, 0x29, 0xc4, 0x19, 0x7a, 0x21, + 0x04, 0xfd, 0x11, 0x2d, 0x17, 0x48, 0xbc, 0x72, 0xf1, 0x05, 0x64, 0x31, 0x79, 0xd6, 0x26, 0xb6, + 0x53, 0x21, 0x8a, 0x46, 0x2c, 0x74, 0x01, 0x12, 0x4f, 0xc9, 0x51, 0x21, 0x31, 0x2f, 0xdd, 0x9a, + 0x2c, 0x4d, 0x7c, 0x7c, 0x3c, 0x97, 0x58, 0x27, 0x47, 0x98, 0xc2, 0xd0, 0x3c, 0x4c, 0x10, 0x43, + 0x93, 0x69, 0x71, 0xb2, 0xb3, 0x78, 0x9c, 0x18, 0xda, 0x3a, 0x39, 0x42, 0xb3, 0x90, 0xb2, 0x29, + 0x35, 0x43, 0x25, 0x85, 0xb1, 0x79, 0xe9, 0xd6, 0x18, 0xf6, 0xbe, 0xdf, 0x4e, 0xfe, 0xf4, 0xbb, + 0x73, 0xd2, 0xc3, 0x64, 0x4a, 0xca, 0xc7, 0x1f, 0x26, 0x53, 0xf1, 0x7c, 0xa2, 0xf8, 0xed, 0x04, + 0xe4, 0x30, 0xb1, 0x5b, 0xa6, 0x61, 0x13, 0xd1, 0xfa, 0x67, 0x21, 0xe1, 0x1c, 0x1a, 0xac, 0xf5, + 0xcc, 0xe2, 0xe5, 0x90, 0x21, 0xd4, 0x2c, 0xc5, 0xb0, 0x15, 0xd5, 0xd1, 0x4d, 0x03, 0xd3, 0xaa, + 0xe8, 0xf3, 0x90, 0xb1, 0x88, 0xdd, 0x6e, 0x12, 0xc6, 0x6c, 0xd6, 0xb1, 0xcc, 0xe2, 0xf9, 0x10, + 0xcc, 0x6a, 0x4b, 0x31, 0x30, 0xf0, 0xba, 0xf4, 0x37, 0xba, 0x00, 0x29, 0xa3, 0xdd, 0xa4, 0xc3, + 0xb1, 0x59, 0x67, 0x13, 0x78, 0xc2, 0x68, 0x37, 0xd7, 0xc9, 0x91, 0x8d, 0xca, 0x90, 0xb1, 0x28, + 0xab, 0x65, 0xdd, 0xd8, 0x33, 0xed, 0xc2, 0xf8, 0x7c, 0xe2, 0x56, 0x66, 0xf1, 0x52, 0x3f, 0x81, + 0x50, 0xe1, 0x09, 0xae, 0x82, 0xe5, 0x02, 0x6c, 0x54, 0x85, 0xac, 0xe8, 0x99, 0x45, 0x14, 0xdb, + 0x34, 0x0a, 0x13, 0xf3, 0xd2, 0xad, 0xdc, 0xe2, 0x42, 0x18, 0x99, 0x0e, 0x2e, 0xd0, 0xcf, 0x76, + 0x93, 0x60, 0x86, 0x85, 0x27, 0xad, 0xc0, 0x57, 0xf1, 0x09, 0x4c, 0x06, 0x4b, 0x11, 0x82, 0x1c, + 0x5e, 0xa9, 0xee, 0x6c, 0xac, 0xc8, 0x3b, 0x9b, 0xeb, 0x9b, 0x5b, 0xef, 0x6f, 0xe6, 0x63, 0xe8, + 0x0c, 0xe4, 0x05, 0x6c, 0x7d, 0xe5, 0x89, 0xfc, 0x68, 0x6d, 0x63, 0xad, 0x96, 0x97, 0xd0, 0x05, + 0x38, 0x2b, 0xa0, 0x78, 0x69, 0x73, 0x75, 0x45, 0x2e, 0x6d, 0xed, 0x6c, 0x2e, 0x2f, 0xe1, 0x27, + 0xf9, 0xf8, 0x6c, 0xf2, 0xd7, 0xbe, 0x77, 0x39, 0x56, 0x7c, 0x0c, 0xb0, 0x4a, 0x1c, 0xa1, 0x0d, + 0xa8, 0x04, 0xe3, 0xfb, 0xac, 0x37, 0x42, 0x1d, 0xe7, 0x43, 0xbb, 0x1d, 0xd0, 0x9c, 0x52, 0x8a, + 0x72, 0xe0, 0x27, 0xc7, 0x73, 0x12, 0x16, 0x98, 0x5c, 0xe4, 0xc5, 0x1f, 0x4b, 0x90, 0x61, 0x84, + 0xf9, 0x18, 0x51, 0xb9, 0x8b, 0xf2, 0x95, 0xa1, 0x0c, 0xe9, 0x25, 0x8d, 0x16, 0x60, 0xec, 0xb9, + 0xd2, 0x68, 0x0f, 0xd2, 0xf6, 0xc7, 0xb4, 0x1c, 0xf3, 0x6a, 0xe8, 0x1d, 0x98, 0xd4, 0x0d, 0x87, + 0x18, 0x8e, 0xcc, 0xd1, 0x12, 0x43, 0xd0, 0x32, 0xbc, 0x36, 0xfb, 0x28, 0xfe, 0xa5, 0x04, 0xb0, + 0xdd, 0x8e, 0x92, 0x35, 0x74, 0xb6, 0x8e, 0xd4, 0x7f, 0x77, 0xb6, 0xf2, 0x51, 0x9c, 0x83, 0x71, + 0xdd, 0x68, 0xe8, 0x06, 0xef, 0x7f, 0x0a, 0x8b, 0x2f, 0x74, 0x06, 0xc6, 0x76, 0x1b, 0xba, 0xa1, + 0x31, 0xf5, 0x4f, 0x61, 0xfe, 0x21, 0xd8, 0x8f, 0x21, 0xc3, 0xfa, 0x1e, 0x21, 0xf7, 0x8b, 0xff, + 0x2e, 0xc1, 0xd9, 0xb2, 0x69, 0x68, 0x3a, 0x9d, 0x87, 0x4a, 0xe3, 0x53, 0xc1, 0x9b, 0x37, 0x21, + 0x4d, 0x0e, 0x5b, 0x23, 0x8a, 0x37, 0x45, 0x0e, 0x5b, 0xec, 0xd7, 0x40, 0xd6, 0x7d, 0x08, 0xe7, + 0xba, 0x47, 0x19, 0x25, 0x17, 0xff, 0x5e, 0x82, 0xdc, 0x9a, 0xa1, 0x3b, 0x9f, 0x0a, 0xf6, 0x79, + 0x7c, 0x48, 0x04, 0xf8, 0x80, 0xee, 0x40, 0x7e, 0x4f, 0xd1, 0x1b, 0x5b, 0x46, 0xcd, 0x6c, 0xee, + 0xda, 0x8e, 0x69, 0x10, 0x5b, 0x30, 0xaa, 0x07, 0x2e, 0x78, 0xf6, 0x18, 0xa6, 0xbc, 0x31, 0x45, + 0xc9, 0xac, 0x17, 0x90, 0x5f, 0x33, 0x54, 0x8b, 0x34, 0x89, 0x11, 0x29, 0xb7, 0x2e, 0x41, 0x5a, + 0x77, 0xe9, 0x32, 0x8e, 0x25, 0xb0, 0x0f, 0x10, 0x63, 0x6a, 0xc3, 0x74, 0xa0, 0xed, 0x28, 0xcd, + 0xd8, 0x45, 0x48, 0x1b, 0xe4, 0x40, 0xf6, 0xe5, 0x95, 0xc0, 0x29, 0x83, 0x1c, 0x70, 0xb3, 0xf3, + 0x04, 0xb2, 0xcb, 0xa4, 0x41, 0x1c, 0x12, 0xbd, 0x4d, 0xde, 0x81, 0x9c, 0x4b, 0x3a, 0x4a, 0x21, + 0xfd, 0xbe, 0x04, 0x48, 0xd0, 0xa5, 0xeb, 0x60, 0x94, 0x72, 0x9a, 0xa3, 0xeb, 0xbc, 0xd3, 0xb6, + 0x0c, 0xbe, 0x60, 0x73, 0x2d, 0x05, 0x0e, 0x62, 0x6b, 0xb6, 0x6f, 0x1b, 0x93, 0x41, 0xdb, 0xe8, + 0xf9, 0x1d, 0xd4, 0xe3, 0x38, 0x80, 0x99, 0x8e, 0xee, 0x45, 0x2b, 0xca, 0x24, 0xeb, 0x59, 0x7c, + 0x3e, 0x11, 0x74, 0x8d, 0x18, 0xb0, 0xf8, 0x21, 0x4c, 0x97, 0x1b, 0x44, 0xb1, 0xa2, 0x66, 0x8b, + 0x10, 0xe7, 0x13, 0x40, 0x41, 0xf2, 0x51, 0x8a, 0x54, 0x87, 0x4c, 0x55, 0x55, 0x8c, 0xad, 0x16, + 0x35, 0x82, 0x36, 0xba, 0x0f, 0xe7, 0x6c, 0xc7, 0x6c, 0xc9, 0x8a, 0x23, 0x73, 0x0f, 0x69, 0xd7, + 0x6c, 0x1b, 0x9a, 0x62, 0x1d, 0xb1, 0x36, 0x52, 0x78, 0x86, 0x96, 0x2e, 0x39, 0xac, 0x23, 0x25, + 0x51, 0x44, 0x65, 0xd7, 0xd4, 0x0d, 0x99, 0x3a, 0x32, 0x0d, 0xc7, 0x16, 0x7a, 0x0e, 0x4d, 0xdd, + 0xc0, 0x1c, 0x22, 0x46, 0xf1, 0x3d, 0x89, 0xb7, 0x15, 0xa5, 0xda, 0xbc, 0x0b, 0x19, 0x5b, 0x55, + 0x0c, 0x79, 0xcf, 0xb4, 0x9a, 0x8a, 0xc3, 0x54, 0x23, 0xb7, 0xf8, 0x5a, 0x98, 0x7b, 0xa8, 0x2a, + 0xc6, 0x03, 0x56, 0x09, 0x83, 0xed, 0xfd, 0x0e, 0x6a, 0xcf, 0xc3, 0x64, 0x2a, 0x91, 0x4f, 0x16, + 0x7f, 0x26, 0xc1, 0x24, 0xef, 0x65, 0x94, 0xda, 0xf3, 0x26, 0x24, 0x2d, 0xf3, 0x80, 0x6b, 0x4f, + 0x66, 0xf1, 0x62, 0x08, 0x89, 0x75, 0x72, 0x14, 0x34, 0xdb, 0xac, 0x3a, 0x2a, 0x81, 0x70, 0x54, + 0x64, 0x86, 0x9d, 0x18, 0x15, 0x1b, 0x38, 0x16, 0xa6, 0x34, 0xae, 0x43, 0x6e, 0x57, 0x71, 0xd4, + 0x7d, 0x2a, 0x1f, 0xd6, 0x49, 0xee, 0xdd, 0xe3, 0x2c, 0x83, 0xba, 0x3d, 0x2f, 0xfe, 0xb1, 0x04, + 0x08, 0x93, 0xe7, 0xc4, 0xb2, 0xc9, 0xa7, 0x5f, 0x48, 0xff, 0x2d, 0xc1, 0x4c, 0x47, 0x67, 0xff, + 0x7f, 0xc9, 0xea, 0x57, 0x24, 0x38, 0x5f, 0xde, 0x27, 0xea, 0xd3, 0xb2, 0x69, 0xd8, 0xba, 0xed, + 0x10, 0x43, 0x3d, 0x8a, 0x52, 0x60, 0x17, 0x21, 0x7d, 0xa0, 0x3b, 0xfb, 0xb2, 0xa6, 0xef, 0xed, + 0xb1, 0xe9, 0x9c, 0xc2, 0x29, 0x0a, 0x58, 0xd6, 0xf7, 0xf6, 0xc4, 0x64, 0x96, 0xa1, 0xd0, 0xdb, + 0x83, 0x68, 0x1d, 0x82, 0xb3, 0x98, 0xa8, 0x66, 0xb3, 0xd5, 0x76, 0x48, 0xd5, 0x51, 0x1c, 0x3b, + 0xca, 0x01, 0x9e, 0x87, 0x09, 0xcd, 0x3a, 0x92, 0xad, 0xb6, 0x21, 0x86, 0x37, 0xae, 0x59, 0x47, + 0xb8, 0x6d, 0x88, 0xc1, 0xfd, 0xb9, 0x04, 0xe7, 0xba, 0x1b, 0x8f, 0x52, 0xc3, 0xbe, 0x02, 0x19, + 0x45, 0xd3, 0x88, 0x26, 0x6b, 0xa4, 0xe1, 0x28, 0xc2, 0x91, 0xbb, 0x17, 0xa0, 0x24, 0x76, 0x2b, + 0x16, 0xf8, 0x36, 0xc5, 0x82, 0xbb, 0x5b, 0xb1, 0xb0, 0xf1, 0xb8, 0x5c, 0x66, 0xfd, 0x59, 0xa6, + 0x88, 0xae, 0x02, 0x31, 0x5a, 0x0c, 0x52, 0x54, 0xe1, 0x7c, 0x89, 0xd4, 0x75, 0x23, 0x18, 0x47, + 0x47, 0xbe, 0x1c, 0xc9, 0x50, 0xe8, 0x6d, 0x24, 0x4a, 0xd9, 0xff, 0x5d, 0x02, 0xce, 0xae, 0x18, + 0xda, 0xab, 0x19, 0x04, 0xf5, 0x24, 0x54, 0xb3, 0xd9, 0xd4, 0x1d, 0x57, 0xf6, 0xfc, 0x0b, 0xfd, + 0x3c, 0xa4, 0x34, 0xa2, 0x68, 0x5e, 0xfc, 0x95, 0xe9, 0xb0, 0x51, 0x6d, 0x47, 0x6f, 0x2c, 0xec, + 0x37, 0xd4, 0x85, 0x9a, 0xbb, 0x37, 0x84, 0xbd, 0xea, 0xe8, 0x6b, 0x70, 0x9e, 0xce, 0x62, 0xcb, + 0x50, 0x1a, 0x32, 0xa7, 0x26, 0x3b, 0x96, 0x5e, 0xaf, 0x13, 0x4b, 0xec, 0x58, 0xdc, 0x0a, 0xe9, + 0xe7, 0x9a, 0xc0, 0x28, 0x33, 0x84, 0x1a, 0xaf, 0x8f, 0xcf, 0xea, 0x61, 0x60, 0xf4, 0x65, 0x2f, + 0xc0, 0xb5, 0x5b, 0x8a, 0x61, 0x17, 0xc6, 0x98, 0x79, 0xe9, 0xb7, 0x11, 0x22, 0x34, 0x43, 0x18, + 0x24, 0x0a, 0xb1, 0xd1, 0x5d, 0xea, 0x61, 0x3d, 0x6b, 0xeb, 0x16, 0x91, 0xef, 0xb5, 0xd4, 0xc2, + 0x38, 0x1d, 0x7b, 0x29, 0x77, 0x72, 0x3c, 0x07, 0x98, 0x83, 0xef, 0x6d, 0x97, 0xa9, 0xc7, 0xc5, + 0x7f, 0xb7, 0x54, 0x74, 0x0b, 0xf2, 0x86, 0x29, 0x5b, 0x64, 0xcf, 0x22, 0xf6, 0xbe, 0x68, 0x36, + 0xc5, 0x38, 0x96, 0x33, 0x4c, 0xcc, 0xc1, 0x9c, 0xf4, 0x39, 0x18, 0x6f, 0x99, 0xba, 0x6d, 0x1a, + 0x85, 0x34, 0xe7, 0x28, 0xff, 0xf2, 0x0c, 0xf7, 0x44, 0x3e, 0x55, 0xfc, 0x4d, 0x09, 0xce, 0x75, + 0xcb, 0x34, 0xca, 0x39, 0x75, 0x0b, 0xf2, 0xa6, 0x41, 0xe4, 0xd6, 0xbe, 0x62, 0x13, 0x21, 0x03, + 0xe1, 0x29, 0xe6, 0x4c, 0x83, 0x6c, 0x53, 0x30, 0xe7, 0x68, 0xc7, 0x42, 0xf2, 0xab, 0x12, 0x4c, + 0x2f, 0x69, 0x4d, 0xdd, 0xa8, 0xb6, 0x1a, 0x7a, 0xa4, 0x81, 0xc7, 0x35, 0x48, 0xdb, 0x94, 0x26, + 0xdb, 0x4f, 0x8b, 0x77, 0xee, 0xa7, 0xa5, 0x58, 0xc9, 0x3a, 0x39, 0xf2, 0xfd, 0xbb, 0x60, 0x27, + 0xa2, 0x9c, 0x4a, 0x1f, 0x8a, 0xf1, 0x6d, 0x10, 0xeb, 0x15, 0x79, 0xa6, 0x41, 0xf2, 0x51, 0xf6, + 0xfc, 0x5b, 0x12, 0x5c, 0x60, 0xb4, 0x99, 0xca, 0xec, 0x11, 0x8b, 0x6d, 0x6f, 0x46, 0x29, 0xa2, + 0xab, 0x30, 0xee, 0x28, 0x56, 0x9d, 0x70, 0x43, 0x30, 0x56, 0xca, 0x7c, 0x7c, 0x3c, 0x37, 0x51, + 0x75, 0x4c, 0x8b, 0xac, 0x2d, 0x63, 0x51, 0x24, 0xc6, 0xa9, 0xc0, 0x6c, 0x58, 0x5f, 0xa2, 0x1c, + 0xef, 0x7f, 0x49, 0xa2, 0x8d, 0xf2, 0x3e, 0x77, 0xf3, 0x5b, 0x0d, 0x5d, 0x55, 0x22, 0x5d, 0xf6, + 0x56, 0x20, 0xa3, 0x32, 0xe2, 0xb2, 0x73, 0xd4, 0xe2, 0x01, 0x69, 0x6e, 0xf1, 0x5a, 0x28, 0x21, + 0xd6, 0x38, 0xef, 0x49, 0xed, 0xa8, 0x45, 0x30, 0xa8, 0xde, 0x6f, 0xb4, 0x0c, 0x13, 0x9c, 0x39, + 0xae, 0x97, 0x33, 0x80, 0x04, 0x9d, 0xe8, 0x35, 0x56, 0x59, 0xd8, 0x24, 0x17, 0x55, 0x30, 0x76, + 0x17, 0x2e, 0x86, 0x0e, 0x3a, 0x4a, 0xce, 0xfe, 0x89, 0xab, 0x49, 0x98, 0x34, 0x4c, 0x55, 0x79, + 0x05, 0xd1, 0x6b, 0x80, 0x23, 0xf1, 0x4f, 0xca, 0x11, 0x57, 0xd5, 0xba, 0x3a, 0x1b, 0x25, 0x43, + 0x7e, 0x5b, 0x82, 0x99, 0x0a, 0x51, 0x2c, 0x67, 0x97, 0x28, 0x4e, 0xed, 0x30, 0xd2, 0xd5, 0xf5, + 0x4d, 0x48, 0x18, 0xe6, 0x81, 0xf0, 0x69, 0x06, 0x2f, 0xa0, 0x62, 0xfc, 0xb4, 0xbe, 0x18, 0xfb, + 0x57, 0xe1, 0x4c, 0x67, 0xbf, 0xa2, 0x1c, 0xf5, 0x0f, 0x12, 0x90, 0x5e, 0x2d, 0x47, 0x39, 0xd6, + 0x2f, 0x88, 0x3d, 0x01, 0x3e, 0x0b, 0xc2, 0x4e, 0x74, 0xbc, 0xf6, 0x16, 0x56, 0xcb, 0xeb, 0xe4, + 0xc8, 0x0d, 0x18, 0x28, 0x16, 0x5a, 0x82, 0xb4, 0xb3, 0x4f, 0x17, 0x51, 0xb3, 0xa1, 0x09, 0x37, + 0x61, 0x24, 0x7e, 0xf9, 0x58, 0xa8, 0x01, 0x67, 0x9d, 0x43, 0x83, 0xad, 0xcd, 0x72, 0x5d, 0x95, + 0x7d, 0x72, 0x63, 0xa3, 0x90, 0x9b, 0xa5, 0xe4, 0x4e, 0x8e, 0xe7, 0x50, 0xed, 0xd0, 0xa0, 0x0b, + 0xf9, 0x6a, 0xb9, 0xe6, 0x12, 0xc0, 0xc8, 0x11, 0x30, 0xd5, 0x83, 0xcd, 0x3e, 0x85, 0x31, 0x36, + 0x0a, 0xf7, 0x10, 0x49, 0x0a, 0x39, 0x44, 0xa2, 0x83, 0x72, 0x1b, 0x78, 0x19, 0x25, 0xf0, 0xb1, + 0xb8, 0x2a, 0x08, 0x85, 0x78, 0x0f, 0x80, 0xb2, 0x30, 0x4a, 0x35, 0xf8, 0x51, 0x02, 0x72, 0xdb, + 0x6d, 0x7b, 0x3f, 0x62, 0xbd, 0x2f, 0x03, 0xb4, 0xda, 0xf6, 0x3e, 0xb1, 0x64, 0xe7, 0xd0, 0x10, + 0x23, 0x1f, 0x72, 0xc2, 0xe5, 0x0e, 0x9d, 0xe3, 0xd5, 0x0e, 0x0d, 0xb4, 0x25, 0x88, 0x10, 0xd9, + 0x3f, 0x26, 0xbb, 0x33, 0x42, 0x5c, 0x50, 0x3b, 0x34, 0x36, 0x88, 0x17, 0x10, 0x70, 0x82, 0x84, + 0x12, 0xfc, 0x02, 0x4c, 0xd0, 0x0f, 0xd9, 0x31, 0x5f, 0x46, 0xc3, 0xc6, 0x29, 0x4e, 0xcd, 0x74, + 0xe7, 0xf2, 0xd8, 0xcb, 0xcd, 0x65, 0xf4, 0x0e, 0xa4, 0x79, 0xa3, 0x74, 0x91, 0x19, 0x67, 0x8b, + 0x4c, 0x18, 0x27, 0x84, 0x10, 0xd8, 0xf2, 0x92, 0x62, 0x2d, 0xd2, 0xc5, 0xe5, 0x0c, 0x8c, 0xed, + 0x99, 0x96, 0x4a, 0xd8, 0x71, 0x5a, 0x0a, 0xf3, 0x0f, 0xcf, 0x93, 0x4c, 0xe5, 0xd3, 0xc5, 0xdf, + 0x95, 0x60, 0xca, 0x13, 0x60, 0x94, 0x2e, 0x64, 0xb9, 0x83, 0xfb, 0x2f, 0x2f, 0x42, 0xca, 0xf1, + 0xe2, 0xf7, 0xe3, 0x30, 0xf5, 0x5e, 0x9b, 0x58, 0x47, 0x11, 0xeb, 0x57, 0x89, 0x1f, 0x9d, 0xc6, + 0x4f, 0xa9, 0x13, 0xec, 0x30, 0xf5, 0x06, 0x4c, 0x1d, 0x28, 0xba, 0x23, 0xef, 0x99, 0x96, 0xdc, + 0x6e, 0x69, 0x8a, 0xe3, 0x1e, 0x34, 0x65, 0x29, 0xf8, 0x81, 0x69, 0xed, 0x30, 0x20, 0x22, 0x80, + 0x9e, 0x1a, 0xe6, 0x81, 0x21, 0x53, 0xb0, 0x6e, 0xd4, 0x29, 0x3f, 0xec, 0x42, 0x92, 0xed, 0x7c, + 0xbe, 0xf5, 0xcf, 0xc7, 0x73, 0xf7, 0xeb, 0xba, 0xb3, 0xdf, 0xde, 0x5d, 0x50, 0xcd, 0xe6, 0x5d, + 0xaf, 0x23, 0xda, 0xae, 0xff, 0xfb, 0x6e, 0xeb, 0x69, 0xfd, 0x2e, 0x3b, 0x33, 0x6f, 0xb7, 0x75, + 0x6d, 0x61, 0x67, 0x67, 0x6d, 0x19, 0xe7, 0x19, 0xc9, 0xf7, 0x39, 0xc5, 0xda, 0xa1, 0xe1, 0xae, + 0x77, 0x1f, 0x4b, 0x90, 0xf7, 0x19, 0x16, 0xa5, 0x3c, 0x57, 0x20, 0xf3, 0xac, 0x4d, 0x2c, 0x9d, + 0x68, 0x2f, 0x2d, 0x50, 0x10, 0x88, 0x74, 0x0e, 0x7d, 0x00, 0x93, 0x1d, 0x7c, 0x48, 0x7c, 0x32, + 0x3e, 0x64, 0x0e, 0x7c, 0x16, 0x14, 0x7f, 0x14, 0x07, 0xc4, 0x06, 0xbf, 0xc6, 0x37, 0x81, 0x3e, + 0x65, 0x0a, 0xf3, 0x18, 0x40, 0xdf, 0x93, 0x9b, 0xba, 0x6d, 0xeb, 0x46, 0x9d, 0xe9, 0x4a, 0x6e, + 0xf1, 0xad, 0x90, 0xbe, 0xf4, 0x0e, 0x61, 0x61, 0x6d, 0x6f, 0x83, 0xa3, 0x95, 0xc8, 0xbe, 0xf2, + 0x5c, 0x37, 0x2d, 0x9c, 0xd6, 0x5d, 0x50, 0xb1, 0x04, 0xd3, 0x3d, 0xe5, 0x28, 0x07, 0xb0, 0xbc, + 0x25, 0x6f, 0x6e, 0xd5, 0x2a, 0x6b, 0x9b, 0xab, 0xf9, 0x18, 0xca, 0xc3, 0x24, 0x5e, 0xa9, 0xed, + 0xe0, 0x4d, 0x79, 0x05, 0xe3, 0x2d, 0x9c, 0x97, 0x50, 0x06, 0x26, 0xb6, 0xf1, 0xca, 0xe3, 0x95, + 0xcd, 0x5a, 0x3e, 0x2e, 0xb4, 0xe7, 0x97, 0x60, 0xa6, 0xa3, 0xf1, 0x28, 0xf5, 0xe7, 0x0a, 0x4c, + 0xee, 0x99, 0x6d, 0x43, 0x93, 0x79, 0x18, 0x2d, 0xb6, 0x0b, 0x32, 0x0c, 0xc6, 0xdb, 0x2b, 0x7e, + 0x33, 0x0e, 0x67, 0x30, 0xb1, 0xcd, 0xc6, 0x73, 0x12, 0xbd, 0x04, 0xb7, 0x40, 0xec, 0x0d, 0xca, + 0x9f, 0x44, 0x90, 0x69, 0x4e, 0x83, 0xaf, 0x06, 0xe3, 0xb6, 0xa3, 0x38, 0x6d, 0x5b, 0x88, 0xf2, + 0xda, 0xe0, 0xb9, 0x50, 0x65, 0x75, 0xb1, 0xc0, 0x09, 0x44, 0xf9, 0xc9, 0xde, 0x28, 0xbf, 0xf8, + 0x0b, 0x70, 0xb6, 0x8b, 0x11, 0x51, 0x2e, 0xda, 0xff, 0x18, 0x87, 0x0b, 0x9d, 0xe4, 0xa3, 0x76, + 0xe1, 0xff, 0x6f, 0x30, 0x1b, 0x55, 0x20, 0xdb, 0xd4, 0x0d, 0xd9, 0xf7, 0xb1, 0x5e, 0x62, 0x71, + 0x9e, 0xa4, 0x01, 0x6c, 0xa7, 0x9b, 0x45, 0xa3, 0x8d, 0x30, 0xbe, 0x46, 0x29, 0xbb, 0xef, 0x48, + 0x30, 0x19, 0xf5, 0xf6, 0xc3, 0xe9, 0x4e, 0xc1, 0xc5, 0x98, 0x6b, 0x90, 0x7d, 0x05, 0xfb, 0x15, + 0x7f, 0x24, 0x01, 0xaa, 0x59, 0x6d, 0x83, 0xc6, 0x6c, 0x8f, 0xcc, 0x7a, 0x94, 0x83, 0x3d, 0x03, + 0x63, 0xba, 0xa1, 0x91, 0x43, 0x36, 0xd8, 0x24, 0xe6, 0x1f, 0xe8, 0x1e, 0xa4, 0x44, 0x16, 0x13, + 0x3f, 0xd5, 0x4f, 0x94, 0xce, 0x9d, 0x1c, 0xcf, 0x4d, 0xf0, 0x9c, 0xa5, 0xe5, 0x8f, 0xfd, 0x9f, + 0x78, 0x82, 0xa7, 0x2d, 0xb9, 0x79, 0x0f, 0x1f, 0xc0, 0x4c, 0x47, 0x47, 0xa3, 0xe4, 0xc2, 0xf7, + 0xe3, 0x30, 0x23, 0x86, 0x13, 0xf9, 0x7e, 0xcd, 0xa9, 0x52, 0xe0, 0xd0, 0x17, 0x01, 0x5a, 0x16, + 0x79, 0x2e, 0x73, 0xd4, 0xc4, 0x48, 0xa8, 0x69, 0x8a, 0xc1, 0x00, 0xe8, 0x2b, 0x30, 0x45, 0x27, + 0x5c, 0xcb, 0x32, 0x5b, 0xa6, 0x4d, 0x3d, 0x09, 0x7b, 0x34, 0x4f, 0x7a, 0xfa, 0xe4, 0x78, 0x2e, + 0xbb, 0xa1, 0x1b, 0xdb, 0x02, 0xb1, 0x56, 0xc5, 0x74, 0xe6, 0x7a, 0x9f, 0xae, 0xfb, 0xf3, 0x0f, + 0x12, 0x9c, 0x79, 0x65, 0x3b, 0x5c, 0xff, 0x1b, 0x1c, 0xf3, 0xd6, 0x83, 0x3c, 0xfb, 0x5c, 0x33, + 0xf6, 0xcc, 0xe8, 0xf7, 0x1d, 0xbf, 0x23, 0xc1, 0x74, 0x80, 0x7c, 0x94, 0xab, 0xfe, 0xe9, 0x12, + 0x2d, 0xbf, 0x4a, 0xfd, 0x80, 0xa0, 0xda, 0x47, 0x39, 0xa9, 0xfe, 0x40, 0x82, 0x73, 0x65, 0x7e, + 0x1a, 0xc5, 0x0e, 0xdd, 0xec, 0x76, 0x33, 0x4a, 0x2d, 0x29, 0xc0, 0xc4, 0x73, 0x62, 0xd9, 0xba, + 0xc9, 0xd7, 0xbd, 0x2c, 0x76, 0x3f, 0x59, 0xc2, 0xa7, 0xa1, 0xb4, 0xec, 0x7d, 0xd3, 0xdd, 0x4c, + 0xf7, 0xbe, 0xbd, 0x90, 0x2c, 0x91, 0x4f, 0x16, 0xff, 0x46, 0x82, 0xf3, 0x3d, 0x1d, 0x8c, 0x52, + 0x28, 0x5f, 0x87, 0x8c, 0x2a, 0x08, 0x53, 0x73, 0xc7, 0xf7, 0xd3, 0xd7, 0x68, 0xb5, 0x53, 0xba, + 0xe1, 0x27, 0xc7, 0x73, 0xe0, 0x76, 0x75, 0x6d, 0x19, 0x83, 0x4b, 0x7d, 0x4d, 0x2b, 0xfe, 0x6c, + 0x02, 0xb2, 0x2b, 0x87, 0x2d, 0xd3, 0x72, 0xaa, 0x7c, 0xcd, 0x47, 0xcb, 0x90, 0x6a, 0x59, 0xe6, + 0x73, 0xdd, 0x1d, 0x44, 0x2e, 0xf4, 0x3c, 0xa7, 0x03, 0x67, 0x5b, 0xd4, 0xc7, 0x1e, 0x26, 0xc2, + 0x90, 0x7e, 0x64, 0xaa, 0x4a, 0xe3, 0x81, 0xde, 0x70, 0x95, 0x6b, 0x61, 0x18, 0x99, 0x05, 0x0f, + 0x63, 0x5b, 0x71, 0xf6, 0xdd, 0xb9, 0xe6, 0x01, 0xd1, 0x2a, 0xa4, 0x2a, 0x8e, 0xd3, 0xa2, 0x85, + 0x62, 0xa2, 0x5e, 0x1f, 0x4a, 0x92, 0x22, 0x08, 0x4a, 0x1e, 0x32, 0xc2, 0x30, 0xbd, 0x6a, 0x9a, + 0xf5, 0x06, 0x29, 0x37, 0xcc, 0xb6, 0x56, 0x36, 0x8d, 0x3d, 0xbd, 0x2e, 0x0c, 0xdd, 0xb5, 0xa1, + 0x14, 0x57, 0xcb, 0x55, 0xdc, 0x8b, 0x8e, 0xbe, 0x04, 0xa9, 0xea, 0x7d, 0x41, 0x8a, 0xbb, 0x29, + 0x57, 0x87, 0x92, 0xaa, 0xde, 0xc7, 0x1e, 0x12, 0xaa, 0x40, 0x66, 0xe9, 0x45, 0xdb, 0x22, 0x82, + 0xc6, 0x38, 0xa3, 0x71, 0x63, 0x28, 0x0d, 0x86, 0x83, 0x83, 0xa8, 0xb3, 0xb7, 0x21, 0xdb, 0xc1, + 0x49, 0x84, 0x20, 0xd9, 0xa2, 0x4c, 0xa3, 0xe2, 0x4c, 0x63, 0xf6, 0x9b, 0x6b, 0xf4, 0xec, 0x0d, + 0x48, 0x52, 0xae, 0xd0, 0x59, 0xb1, 0xab, 0xd8, 0x64, 0xc7, 0xd2, 0x45, 0x25, 0xf7, 0x53, 0xd4, + 0xfb, 0x6b, 0x09, 0xe2, 0xd5, 0xfb, 0xd4, 0x51, 0xdb, 0x6d, 0xab, 0x4f, 0x89, 0x23, 0x6a, 0x89, + 0x2f, 0xe6, 0xc0, 0x59, 0x64, 0x4f, 0xe7, 0x8b, 0x76, 0x1a, 0x8b, 0x2f, 0xf4, 0x1a, 0x80, 0xa2, + 0xaa, 0xc4, 0xb6, 0x65, 0x37, 0x0f, 0x3b, 0x8d, 0xd3, 0x1c, 0xb2, 0x4e, 0x8e, 0x28, 0x9a, 0x4d, + 0x54, 0x8b, 0xf0, 0xf9, 0x96, 0xc6, 0xe2, 0x8b, 0xa2, 0x39, 0xa4, 0xd9, 0x92, 0x1d, 0xf3, 0x29, + 0x31, 0x18, 0x37, 0xd3, 0x38, 0x4d, 0x21, 0x35, 0x0a, 0xa0, 0x13, 0x95, 0x18, 0x5a, 0xcb, 0xd4, + 0x0d, 0x87, 0xb1, 0x29, 0x8d, 0xbd, 0x6f, 0x4a, 0xd2, 0x22, 0x75, 0x5d, 0x64, 0x28, 0xa7, 0xb1, + 0xf8, 0x12, 0xc3, 0x38, 0x84, 0xc4, 0x6a, 0xb9, 0xfa, 0xd2, 0xc3, 0x40, 0x90, 0x54, 0xda, 0x42, + 0xe9, 0xd2, 0x98, 0xfd, 0x46, 0x37, 0x61, 0x6a, 0x57, 0x6f, 0x34, 0x68, 0xa0, 0xdc, 0xb2, 0xcc, + 0xaf, 0x13, 0xd5, 0x1d, 0x44, 0x4e, 0x80, 0xb7, 0x39, 0x54, 0xb4, 0xfc, 0x4d, 0x09, 0xc6, 0x98, + 0x8c, 0xd0, 0x25, 0x48, 0xab, 0xa6, 0xe1, 0x28, 0xba, 0x21, 0x26, 0x58, 0x1a, 0xfb, 0x80, 0xbe, + 0x5d, 0xb8, 0x02, 0x93, 0x8a, 0xaa, 0x9a, 0x6d, 0xc3, 0x91, 0x0d, 0xa5, 0x49, 0x44, 0x57, 0x32, + 0x02, 0xb6, 0xa9, 0x34, 0x09, 0x9a, 0x03, 0xf7, 0xd3, 0x4b, 0x6b, 0x4f, 0x63, 0x10, 0x20, 0xef, + 0xfc, 0x4d, 0xac, 0x29, 0x7f, 0x2a, 0xc1, 0xf4, 0xfb, 0x96, 0xee, 0x90, 0x12, 0x4f, 0xb6, 0x88, + 0xce, 0xc0, 0xbe, 0x0d, 0x69, 0x4d, 0x71, 0x14, 0x9e, 0xc2, 0x1e, 0x1f, 0x98, 0xc2, 0xee, 0x4e, + 0x4c, 0x5a, 0x9f, 0xa5, 0xb1, 0x23, 0x48, 0xd2, 0xdf, 0x3c, 0x63, 0x1f, 0xb3, 0xdf, 0xfe, 0xd9, + 0x5b, 0xb0, 0xbb, 0x51, 0x2e, 0x38, 0xdf, 0x48, 0xb8, 0x26, 0x30, 0x4a, 0x36, 0x7c, 0x19, 0x26, + 0x44, 0x14, 0x25, 0x98, 0x30, 0x3f, 0x6c, 0x2a, 0xbb, 0x27, 0x24, 0x02, 0x0d, 0x95, 0x00, 0x6c, + 0x47, 0xb1, 0x1c, 0x16, 0xff, 0x8c, 0x74, 0x48, 0xef, 0x9a, 0x4c, 0x86, 0x46, 0xa1, 0x68, 0x13, + 0x32, 0xcd, 0xe7, 0xaa, 0x2a, 0xef, 0xe9, 0x0d, 0x47, 0x9c, 0xcf, 0x87, 0x67, 0x23, 0x6d, 0x3c, + 0x2e, 0x97, 0x1f, 0xb0, 0x4a, 0xfc, 0x98, 0xdc, 0xff, 0xc6, 0x40, 0x29, 0xf0, 0xdf, 0xe8, 0x33, + 0x20, 0xd2, 0x14, 0x65, 0xdb, 0x76, 0xd8, 0xcc, 0x4c, 0x95, 0xb2, 0x27, 0xc7, 0x73, 0x69, 0xcc, + 0xa0, 0xd5, 0x6a, 0x0d, 0xa7, 0x79, 0x85, 0xaa, 0xed, 0xa0, 0xab, 0x90, 0x35, 0x9b, 0xba, 0x23, + 0xbb, 0xeb, 0x0d, 0x3f, 0x87, 0xc7, 0x93, 0x14, 0xe8, 0xae, 0x47, 0x42, 0xbe, 0xdf, 0x96, 0x20, + 0x5b, 0x6a, 0x37, 0x9e, 0x6e, 0xb5, 0xaa, 0xed, 0x66, 0x53, 0xb1, 0x8e, 0xd0, 0x45, 0x57, 0x8f, + 0xf4, 0x17, 0x84, 0xc9, 0x21, 0x21, 0x14, 0x45, 0x7f, 0x41, 0xa8, 0xa2, 0x88, 0xb4, 0x25, 0x0a, + 0xe7, 0x39, 0x49, 0x57, 0x21, 0xcb, 0x62, 0x05, 0x99, 0x18, 0x8e, 0xa5, 0x13, 0x1e, 0x8a, 0x26, + 0xf0, 0x24, 0x03, 0xae, 0x70, 0x18, 0xba, 0x0e, 0x39, 0xfb, 0xc8, 0x76, 0x48, 0x53, 0xe6, 0x17, + 0x5b, 0xb8, 0x83, 0x9b, 0xc0, 0x59, 0x0e, 0xc5, 0x1c, 0x58, 0xfc, 0xb3, 0x04, 0xe4, 0x5c, 0x9d, + 0x88, 0x72, 0x69, 0x2f, 0xc1, 0xd8, 0x9e, 0xde, 0x20, 0xee, 0xc9, 0x59, 0x7f, 0xf3, 0xee, 0x52, + 0x5a, 0xa0, 0x46, 0xdc, 0xf5, 0xbe, 0x18, 0x6a, 0x14, 0x7a, 0x31, 0xfb, 0x13, 0x09, 0x92, 0x6c, + 0x4d, 0xbd, 0x07, 0x49, 0x36, 0x51, 0xa5, 0x51, 0x26, 0x2a, 0xab, 0xea, 0xad, 0x26, 0x71, 0x7f, + 0x35, 0x61, 0x96, 0x7c, 0x5f, 0x79, 0xf3, 0xde, 0x22, 0xd3, 0x89, 0x49, 0x2c, 0xbe, 0x50, 0x09, + 0x52, 0x84, 0x8d, 0x87, 0x68, 0x62, 0x45, 0x0b, 0x9b, 0x06, 0x1d, 0x82, 0x77, 0x8d, 0x82, 0x8b, + 0x87, 0x2e, 0x40, 0x82, 0x2a, 0xdb, 0x04, 0x3f, 0x80, 0x39, 0x39, 0x9e, 0x4b, 0x50, 0x35, 0xa3, + 0x30, 0xee, 0x90, 0x3d, 0x4c, 0xa6, 0x92, 0xf9, 0xb1, 0xe2, 0x0f, 0x93, 0x90, 0x5d, 0x6b, 0x46, + 0x3d, 0x8d, 0x97, 0x3a, 0x05, 0x16, 0xe6, 0x70, 0x74, 0x34, 0x1a, 0x22, 0xaf, 0x0e, 0x83, 0x98, + 0x78, 0x39, 0x83, 0xb8, 0x46, 0x97, 0x33, 0x71, 0xab, 0x87, 0xb6, 0xff, 0xfa, 0xd0, 0xf6, 0x6b, + 0xca, 0x6e, 0x83, 0x60, 0x8a, 0xe3, 0x9e, 0x6f, 0x70, 0x02, 0xe8, 0x5d, 0xb6, 0x6a, 0x72, 0xa5, + 0x19, 0x1f, 0x5d, 0x69, 0x26, 0x88, 0xa1, 0x31, 0x95, 0x39, 0x14, 0x1a, 0xf3, 0x79, 0x48, 0x68, + 0xfa, 0x20, 0x96, 0x86, 0x19, 0x35, 0x8a, 0x32, 0x44, 0x71, 0x92, 0x41, 0xc5, 0x09, 0x3a, 0xdc, + 0xb3, 0x5b, 0x00, 0xfe, 0xa8, 0xd0, 0x3c, 0x8c, 0x9b, 0x0d, 0x8d, 0x3a, 0xc6, 0xb4, 0x0b, 0xd9, + 0x52, 0xfa, 0xe4, 0x78, 0x6e, 0x6c, 0xab, 0xa1, 0xad, 0x2d, 0xe3, 0x31, 0xb3, 0xa1, 0xad, 0x69, + 0xec, 0x32, 0x14, 0x39, 0x90, 0xd9, 0xfd, 0x33, 0xe6, 0x3c, 0xe3, 0x09, 0x83, 0x1c, 0x2c, 0x13, + 0x5b, 0x0d, 0x2e, 0x81, 0x42, 0x6d, 0xfe, 0x50, 0x82, 0x9c, 0xcb, 0xc1, 0x68, 0x67, 0x7a, 0x4a, + 0x6f, 0x0a, 0xcd, 0x4f, 0xbc, 0x9c, 0xe6, 0xbb, 0x78, 0x22, 0xd1, 0xfb, 0x5b, 0x12, 0xcc, 0xf0, + 0x8c, 0x19, 0x55, 0x71, 0xa8, 0x41, 0x8e, 0x50, 0xbd, 0x6f, 0x43, 0xde, 0x52, 0x0c, 0xcd, 0x6c, + 0xea, 0x2f, 0x08, 0x0f, 0x81, 0x6d, 0xb1, 0xf3, 0x3b, 0xe5, 0xc1, 0x59, 0x8c, 0xe7, 0x46, 0xf0, + 0xff, 0x21, 0xc1, 0x99, 0xce, 0xce, 0x44, 0xc9, 0xb4, 0x75, 0x18, 0x67, 0xbb, 0x37, 0xee, 0x74, + 0x7b, 0x23, 0x84, 0x48, 0x58, 0xeb, 0xfc, 0xee, 0x9a, 0xa7, 0xf0, 0x8c, 0xc4, 0xec, 0x97, 0x61, + 0x8c, 0x81, 0x4f, 0x61, 0xe3, 0x04, 0xe7, 0x9f, 0xc1, 0xf4, 0x92, 0xa6, 0x55, 0xab, 0x42, 0xfb, + 0xa2, 0x63, 0xbb, 0xeb, 0xe7, 0xc4, 0xc3, 0xfc, 0x9c, 0x60, 0x93, 0x51, 0xfa, 0x39, 0x2d, 0xc8, + 0x89, 0x44, 0xb6, 0x88, 0xb7, 0xeb, 0x0e, 0xa8, 0x63, 0x26, 0xd4, 0x86, 0x7f, 0xf8, 0xf7, 0x67, + 0xbc, 0x16, 0xa3, 0x1c, 0x49, 0x1b, 0x66, 0x5c, 0xba, 0x51, 0xef, 0x8c, 0x0f, 0x1a, 0x0e, 0xdb, + 0xf6, 0x08, 0x36, 0x1b, 0xe5, 0x98, 0xbe, 0x27, 0x41, 0xae, 0xda, 0xde, 0xe5, 0xd7, 0x21, 0xa3, + 0x1b, 0xcf, 0x2a, 0x40, 0x83, 0xec, 0x89, 0x0b, 0x0e, 0xc2, 0x13, 0x1d, 0xfd, 0x36, 0x6e, 0x9a, + 0xe2, 0xb2, 0x22, 0xc1, 0x82, 0x1f, 0xc6, 0x61, 0xca, 0xeb, 0x65, 0x94, 0x33, 0xff, 0x17, 0x81, + 0xb9, 0x99, 0xb2, 0xed, 0x28, 0x8e, 0x2d, 0x0c, 0xe6, 0x67, 0x5e, 0x26, 0x49, 0xb8, 0x34, 0x2d, + 0x12, 0x3c, 0xd2, 0x1e, 0x08, 0xa7, 0x29, 0x49, 0xf6, 0x13, 0x2d, 0xc0, 0x0c, 0x33, 0x6f, 0xb2, + 0xd2, 0x6a, 0x35, 0x74, 0xa2, 0xc9, 0x7c, 0x8f, 0x39, 0xc9, 0xf6, 0x98, 0xa7, 0x59, 0xd1, 0x12, + 0x2f, 0x59, 0x63, 0xfb, 0xcd, 0x0f, 0x60, 0x72, 0xcf, 0x22, 0xe4, 0x05, 0x91, 0x99, 0xd3, 0xf4, + 0x32, 0x27, 0x0f, 0x19, 0x8e, 0x58, 0xa5, 0x78, 0xc2, 0x90, 0x7c, 0x08, 0xd3, 0x8c, 0x8b, 0x51, + 0xe7, 0x76, 0x0b, 0xa9, 0xfc, 0x85, 0x04, 0x28, 0x48, 0xff, 0xd5, 0x09, 0x26, 0x1e, 0xb5, 0x60, + 0x8a, 0xbf, 0x51, 0x80, 0x49, 0x31, 0xca, 0x1d, 0x43, 0x37, 0x0d, 0x74, 0x0f, 0x12, 0x75, 0x11, + 0x99, 0x67, 0x42, 0x43, 0x15, 0xff, 0x62, 0x6f, 0x25, 0x86, 0x69, 0x5d, 0x8a, 0xd2, 0x6a, 0x3b, + 0x21, 0x19, 0x38, 0x7e, 0xf6, 0x45, 0x10, 0xa5, 0xd5, 0x76, 0x50, 0x15, 0xa6, 0x54, 0xff, 0x3a, + 0xa4, 0x4c, 0xd1, 0x13, 0x7d, 0x93, 0x97, 0x43, 0xaf, 0x87, 0x56, 0x62, 0x38, 0xa7, 0x76, 0x14, + 0xa0, 0x72, 0xf0, 0xfe, 0x5d, 0xb2, 0xef, 0x26, 0x50, 0xf7, 0xdd, 0xbf, 0x4a, 0x2c, 0x70, 0x4d, + 0x0f, 0xbd, 0x0d, 0xe3, 0x1a, 0xbb, 0xd7, 0x25, 0x74, 0x2e, 0x4c, 0x2d, 0x3a, 0xae, 0xd2, 0x55, + 0x62, 0x58, 0x60, 0xa0, 0x87, 0x30, 0xc9, 0x7f, 0x89, 0xf9, 0x3e, 0xde, 0x77, 0x97, 0xac, 0xf7, + 0x66, 0x5b, 0x25, 0x86, 0x33, 0x9a, 0x0f, 0x45, 0x9f, 0x83, 0xa4, 0xad, 0x2a, 0x7c, 0x1f, 0x25, + 0x3c, 0x93, 0x20, 0x70, 0x75, 0xa6, 0x42, 0x17, 0x4e, 0x55, 0x31, 0xd0, 0x13, 0x98, 0xde, 0x25, + 0x75, 0xdd, 0x90, 0x1d, 0xff, 0xb4, 0x8f, 0x25, 0x52, 0x77, 0x1e, 0x30, 0x7a, 0xfe, 0x4f, 0x78, + 0xfe, 0x7e, 0x25, 0x86, 0xf3, 0xbb, 0x5d, 0x45, 0x54, 0x64, 0xcc, 0x81, 0x0d, 0x10, 0x4e, 0xf7, + 0x15, 0x59, 0x68, 0x46, 0x3d, 0x15, 0x19, 0xe9, 0x28, 0x40, 0xab, 0x90, 0x51, 0xa8, 0x43, 0x21, + 0xb3, 0x2c, 0xe5, 0x02, 0xf4, 0xdd, 0x04, 0xec, 0x49, 0x9c, 0xae, 0xb0, 0xcb, 0x08, 0x2e, 0xd0, + 0x27, 0xd4, 0x24, 0x56, 0x9d, 0x14, 0x32, 0x83, 0x09, 0x05, 0x8f, 0x08, 0x3d, 0x42, 0x0c, 0x88, + 0x36, 0x20, 0xbb, 0xef, 0xa6, 0x05, 0xb2, 0xe3, 0xd9, 0xc9, 0xbe, 0x3b, 0x81, 0x21, 0x69, 0x8d, + 0x95, 0x18, 0x9e, 0xdc, 0x0f, 0x80, 0xd1, 0x02, 0xc4, 0xeb, 0x6a, 0x21, 0xcb, 0x68, 0x5c, 0x1a, + 0x94, 0xb4, 0x57, 0x89, 0xe1, 0x78, 0x5d, 0xa5, 0x61, 0x02, 0xcf, 0x67, 0x3a, 0x34, 0x0a, 0xb9, + 0xbe, 0x66, 0xa3, 0x33, 0xa7, 0xac, 0x12, 0xc3, 0x2c, 0xf3, 0x8a, 0xb6, 0xb7, 0x0d, 0x39, 0x8b, + 0x9f, 0xb1, 0xba, 0x99, 0x04, 0x79, 0x46, 0xe5, 0x66, 0xb8, 0xf1, 0xe9, 0x49, 0x26, 0xa8, 0xc4, + 0x70, 0xd6, 0x0a, 0xc2, 0xd1, 0xd7, 0xe0, 0x4c, 0x27, 0x45, 0xa1, 0xdc, 0xd3, 0x3d, 0xb6, 0x28, + 0x9c, 0x6e, 0xa7, 0x8e, 0x23, 0xab, 0xa7, 0x10, 0xbd, 0x05, 0x63, 0x5c, 0x6a, 0x88, 0x91, 0x9c, + 0x0b, 0xdb, 0x1f, 0xe9, 0x14, 0x18, 0xaf, 0x4f, 0xe7, 0x9b, 0x23, 0x0e, 0x17, 0xe5, 0x86, 0x59, + 0x2f, 0xcc, 0xf4, 0x9d, 0x6f, 0xbd, 0x87, 0xa5, 0x74, 0xbe, 0x39, 0x3e, 0x94, 0xca, 0xdd, 0xe2, + 0x25, 0xe2, 0x2c, 0xea, 0x4c, 0x5f, 0xb9, 0x87, 0x9c, 0x39, 0x52, 0xb9, 0x5b, 0x01, 0x30, 0xed, + 0x9a, 0xc5, 0x2f, 0x8d, 0xc9, 0x6c, 0x1a, 0x9f, 0xed, 0xdb, 0xb5, 0xde, 0x8b, 0x70, 0xb4, 0x6b, + 0x96, 0x0f, 0x45, 0x8f, 0x21, 0x2f, 0xee, 0x07, 0xf9, 0x5b, 0x39, 0xe7, 0x18, 0xbd, 0xdb, 0xa1, + 0xd6, 0x32, 0xec, 0xf0, 0xa6, 0x12, 0xc3, 0x53, 0x6a, 0x67, 0x09, 0x35, 0x16, 0x8c, 0x9e, 0xac, + 0xfa, 0x17, 0xab, 0x0a, 0x85, 0xbe, 0xc6, 0xa2, 0xcf, 0x2d, 0x30, 0x6a, 0x2c, 0xd4, 0xae, 0x22, + 0xaa, 0xc6, 0xba, 0xa1, 0x3b, 0xcc, 0xb0, 0xcf, 0xf6, 0x55, 0xe3, 0xce, 0x1b, 0xeb, 0x54, 0x8d, + 0x75, 0x0e, 0xa1, 0x6a, 0xec, 0x88, 0x83, 0x4a, 0x21, 0x8e, 0x4b, 0x7d, 0xd5, 0x38, 0xec, 0x44, + 0x93, 0xaa, 0xb1, 0x13, 0x84, 0x53, 0x35, 0xe6, 0x06, 0xa2, 0x8b, 0xee, 0x6b, 0x7d, 0xd5, 0xb8, + 0xef, 0x85, 0x00, 0xaa, 0xc6, 0x4a, 0x4f, 0x21, 0x5a, 0xa6, 0xbe, 0x1e, 0xf5, 0x71, 0x74, 0x63, + 0xcf, 0x2c, 0x5c, 0xee, 0xbb, 0xfe, 0x74, 0x1f, 0x55, 0x56, 0x98, 0xa3, 0x27, 0x60, 0xd4, 0x90, + 0x31, 0xa7, 0x57, 0x66, 0xd7, 0xf0, 0x0a, 0x73, 0x7d, 0x0d, 0x59, 0xcf, 0xf6, 0x31, 0x35, 0x64, + 0x07, 0x1e, 0x90, 0x2e, 0x64, 0x7c, 0x0f, 0xa7, 0x30, 0x3f, 0x64, 0xaf, 0x20, 0xb0, 0x90, 0x71, + 0x0c, 0xb4, 0x04, 0xe9, 0x67, 0x6d, 0x62, 0x1d, 0x31, 0x33, 0x74, 0xa5, 0xaf, 0xd7, 0xda, 0x95, + 0x7b, 0x58, 0x89, 0xe1, 0xd4, 0x33, 0x01, 0xa2, 0xcd, 0xf3, 0x40, 0xba, 0x50, 0xec, 0xdb, 0x7c, + 0xc7, 0xd6, 0x09, 0x6d, 0x9e, 0x63, 0x20, 0x15, 0xce, 0x72, 0x59, 0x89, 0x1b, 0x04, 0x96, 0x48, + 0xd5, 0x2f, 0x5c, 0x65, 0xa4, 0xfa, 0x86, 0xa5, 0xa1, 0xb7, 0x19, 0x2a, 0x31, 0x3c, 0xa3, 0xf4, + 0x96, 0xd2, 0x09, 0x2f, 0x96, 0x1e, 0x1e, 0xcc, 0x16, 0xae, 0xf5, 0x9d, 0xf0, 0x21, 0xe1, 0x3f, + 0x9d, 0xf0, 0x4a, 0x00, 0xcc, 0x17, 0x20, 0x4d, 0xb6, 0x6d, 0x87, 0x86, 0x8e, 0x85, 0xeb, 0x03, + 0x16, 0xa0, 0xae, 0x90, 0x96, 0x2f, 0x40, 0x5a, 0x95, 0x63, 0x52, 0x42, 0x6a, 0x83, 0x28, 0x96, + 0x30, 0xb3, 0x37, 0xfa, 0x12, 0xea, 0xb9, 0x05, 0x4e, 0x09, 0xa9, 0x1e, 0x90, 0x2e, 0xd8, 0x96, + 0x7b, 0xb1, 0x50, 0xf8, 0x8f, 0x37, 0xfb, 0x2e, 0xd8, 0xa1, 0xf7, 0x1f, 0xe9, 0x82, 0x6d, 0x75, + 0x14, 0xa0, 0x2f, 0xc2, 0x84, 0xb8, 0xa5, 0x55, 0xb8, 0x35, 0xc0, 0xab, 0x0d, 0xc6, 0xb9, 0x74, + 0x5e, 0x0b, 0x1c, 0x6e, 0x65, 0xf9, 0x25, 0x2f, 0x3e, 0xbc, 0xdb, 0x03, 0xac, 0x6c, 0x4f, 0x88, + 0xc9, 0xad, 0xac, 0x0f, 0xa6, 0x56, 0x96, 0xeb, 0xa9, 0x58, 0xeb, 0xee, 0xf4, 0xb5, 0xb2, 0xbd, + 0x59, 0x83, 0xd4, 0xca, 0x3e, 0xf3, 0xa1, 0x74, 0x64, 0x36, 0x0f, 0xad, 0x0a, 0xaf, 0xf7, 0x1d, + 0x59, 0x67, 0x88, 0x48, 0x47, 0x26, 0x70, 0xa8, 0xd8, 0x78, 0x86, 0x0c, 0xe7, 0xf4, 0x67, 0xfa, + 0x8a, 0xad, 0x27, 0x12, 0xa9, 0xb8, 0x6f, 0xfd, 0x70, 0x0e, 0x7b, 0x86, 0xca, 0x12, 0x97, 0x32, + 0x04, 0xa7, 0xde, 0x18, 0x6c, 0xa8, 0xc2, 0xee, 0x9b, 0x78, 0x86, 0xaa, 0xa3, 0xb0, 0x34, 0x21, + 0xf2, 0x99, 0x1e, 0x26, 0x53, 0x53, 0xf9, 0xfc, 0xc3, 0x64, 0xea, 0x7c, 0xbe, 0xf0, 0x30, 0x99, + 0xba, 0x90, 0x9f, 0x7d, 0x98, 0x4c, 0x5d, 0xcc, 0x5f, 0x2a, 0xfe, 0xe7, 0x79, 0xc8, 0xba, 0x91, + 0x09, 0x8f, 0x0a, 0x16, 0x83, 0x51, 0xc1, 0xe5, 0x7e, 0x51, 0x81, 0x88, 0x65, 0x44, 0x58, 0xb0, + 0x18, 0x0c, 0x0b, 0x2e, 0xf7, 0x0b, 0x0b, 0x7c, 0x1c, 0x1a, 0x17, 0xd4, 0xfa, 0xc5, 0x05, 0xb7, + 0x47, 0x88, 0x0b, 0x3c, 0x52, 0xdd, 0x81, 0xc1, 0x72, 0x6f, 0x60, 0x70, 0x6d, 0x70, 0x60, 0xe0, + 0x91, 0x0a, 0x44, 0x06, 0xef, 0x74, 0x45, 0x06, 0x57, 0x06, 0x44, 0x06, 0x1e, 0xbe, 0x1b, 0x1a, + 0xac, 0x87, 0x86, 0x06, 0x37, 0x86, 0x85, 0x06, 0x1e, 0x9d, 0x8e, 0xd8, 0xe0, 0xcd, 0x8e, 0xd8, + 0x60, 0xae, 0x6f, 0x6c, 0xe0, 0x61, 0xf3, 0xe0, 0xe0, 0x83, 0xfe, 0xc1, 0xc1, 0xeb, 0x23, 0x05, + 0x07, 0x1e, 0xbd, 0xde, 0xe8, 0xa0, 0xd6, 0x2f, 0x3a, 0xb8, 0x3d, 0x42, 0x74, 0xe0, 0x0b, 0xae, + 0x2b, 0x3c, 0xa8, 0x84, 0x85, 0x07, 0xd7, 0x87, 0x84, 0x07, 0x1e, 0xb5, 0x60, 0x7c, 0x50, 0x09, + 0x8b, 0x0f, 0xae, 0x0f, 0x89, 0x0f, 0xba, 0x28, 0xf1, 0x00, 0x61, 0x33, 0x3c, 0x40, 0xb8, 0x39, + 0x34, 0x40, 0xf0, 0xa8, 0x75, 0x46, 0x08, 0x77, 0x03, 0x11, 0xc2, 0x6b, 0x7d, 0x22, 0x04, 0x0f, + 0x95, 0x86, 0x08, 0x5f, 0xea, 0x09, 0x11, 0x8a, 0x83, 0x42, 0x04, 0x0f, 0xd7, 0x8b, 0x11, 0xde, + 0xeb, 0x13, 0x23, 0xdc, 0x1a, 0x1e, 0x23, 0x78, 0xc4, 0xba, 0x82, 0x04, 0x65, 0x60, 0x90, 0xf0, + 0xc6, 0x88, 0x41, 0x82, 0x47, 0x3d, 0x2c, 0x4a, 0xf8, 0x7c, 0x67, 0x94, 0x30, 0xdf, 0x3f, 0x4a, + 0xf0, 0xc8, 0x88, 0x30, 0x61, 0x3d, 0x34, 0x4c, 0xb8, 0x31, 0x2c, 0x4c, 0xf0, 0xe7, 0x5e, 0x30, + 0x4e, 0xd8, 0x0c, 0x8f, 0x13, 0x6e, 0x0e, 0x8d, 0x13, 0x7c, 0xf1, 0x77, 0x04, 0x0a, 0xeb, 0xa1, + 0x81, 0xc2, 0x8d, 0x61, 0x81, 0x82, 0xdf, 0xb9, 0x60, 0xa4, 0xf0, 0x7e, 0xdf, 0x48, 0xe1, 0xce, + 0x28, 0x91, 0x82, 0x47, 0xb4, 0x27, 0x54, 0xf8, 0xa0, 0x7f, 0xa8, 0xf0, 0xfa, 0x48, 0xa1, 0x82, + 0x6f, 0x3a, 0x7a, 0x62, 0x85, 0x2f, 0xf5, 0xc4, 0x0a, 0xc5, 0x41, 0xb1, 0x82, 0xaf, 0xcf, 0x6e, + 0xb0, 0xa0, 0x0c, 0x74, 0xed, 0xdf, 0x18, 0xd1, 0xb5, 0xf7, 0x95, 0x2f, 0xc4, 0xb7, 0x5f, 0x09, + 0xf1, 0xed, 0xaf, 0x0d, 0xf6, 0xed, 0xfd, 0x25, 0xc4, 0x77, 0xee, 0x2b, 0x61, 0xce, 0xfd, 0xf5, + 0x21, 0xce, 0xbd, 0x6f, 0x85, 0x02, 0xde, 0xfd, 0x3b, 0x5d, 0xde, 0xfd, 0x95, 0xa1, 0x47, 0xd9, + 0x01, 0xf7, 0xbe, 0xd4, 0xeb, 0xde, 0x5f, 0x1d, 0xe8, 0xde, 0x7b, 0x14, 0x7c, 0xff, 0xfe, 0x9d, + 0x2e, 0xff, 0xfe, 0xca, 0x00, 0xff, 0xde, 0xef, 0x80, 0x70, 0xf0, 0xb5, 0xc1, 0x0e, 0xfe, 0xc2, + 0xa8, 0x0e, 0xbe, 0x47, 0x38, 0xd4, 0xc3, 0xdf, 0x0c, 0xf7, 0xf0, 0x6f, 0x8e, 0x78, 0xaa, 0xd5, + 0xe3, 0xe2, 0x57, 0xc2, 0x5c, 0xfc, 0xeb, 0x43, 0x5c, 0xfc, 0xe0, 0x1a, 0xe2, 0xf9, 0xf8, 0x95, + 0x30, 0x1f, 0xff, 0xfa, 0x10, 0x1f, 0xdf, 0xa7, 0x14, 0x70, 0xf2, 0x6b, 0xfd, 0x9c, 0xfc, 0xdb, + 0x23, 0x38, 0xf9, 0xfe, 0xba, 0xdb, 0xe5, 0xe5, 0xbf, 0xdb, 0xed, 0xe5, 0x17, 0x07, 0x79, 0xf9, + 0xfe, 0x8c, 0x74, 0xdd, 0xfc, 0xcd, 0x70, 0x37, 0xff, 0xe6, 0x50, 0x37, 0x3f, 0x68, 0x24, 0x03, + 0x7e, 0xfe, 0x7a, 0xa8, 0x9f, 0x7f, 0x63, 0x98, 0x9f, 0xef, 0x1b, 0xc9, 0xa0, 0xa3, 0xff, 0x6e, + 0xb7, 0xa3, 0x5f, 0x1c, 0xe4, 0xe8, 0xfb, 0x83, 0x73, 0x3d, 0xfd, 0x4a, 0x98, 0xa7, 0x7f, 0x7d, + 0x88, 0xa7, 0xef, 0x0b, 0x2f, 0xe0, 0xea, 0x2b, 0x03, 0x5d, 0xfd, 0x37, 0x46, 0x74, 0xf5, 0xbb, + 0x0c, 0xd7, 0xcb, 0xf9, 0xfa, 0x0f, 0x93, 0xa9, 0x4b, 0xf9, 0xd7, 0x8a, 0x7f, 0x35, 0x06, 0xe3, + 0x15, 0x37, 0xe5, 0x22, 0x70, 0xab, 0x56, 0x3a, 0xcd, 0xad, 0x5a, 0xb4, 0x4c, 0xd5, 0x87, 0x4d, + 0x42, 0xe1, 0xfd, 0x0f, 0xb8, 0xa2, 0xde, 0x73, 0xe8, 0xe5, 0xa2, 0x9e, 0xe2, 0xce, 0x01, 0x7a, + 0x13, 0xb2, 0x6d, 0x9b, 0x58, 0x72, 0xcb, 0xd2, 0x4d, 0x4b, 0x77, 0x78, 0x16, 0x9e, 0x54, 0xca, + 0x7f, 0x7c, 0x3c, 0x37, 0xb9, 0x63, 0x13, 0x6b, 0x5b, 0xc0, 0xf1, 0x64, 0x3b, 0xf0, 0xe5, 0x3e, + 0x15, 0x3b, 0x36, 0xfa, 0x53, 0xb1, 0xef, 0x41, 0xde, 0x22, 0x8a, 0xd6, 0xb1, 0x1c, 0xf2, 0xdb, + 0xa7, 0xe1, 0x2b, 0xb7, 0xa2, 0x05, 0x56, 0x3c, 0x76, 0x0b, 0x75, 0xca, 0xea, 0x04, 0xa2, 0x7b, + 0x70, 0xb6, 0xa9, 0x1c, 0xf2, 0xfb, 0xd5, 0xae, 0x87, 0xc1, 0x52, 0x4f, 0x52, 0x2c, 0x43, 0x0a, + 0x35, 0x95, 0x43, 0xf6, 0xee, 0x2c, 0x2f, 0x62, 0xef, 0xd4, 0x5d, 0x87, 0x9c, 0xa6, 0xdb, 0x8e, + 0x6e, 0xa8, 0xee, 0x53, 0x2d, 0xfc, 0x4d, 0x94, 0xac, 0x0b, 0xe5, 0x4f, 0xa6, 0xdc, 0x81, 0x69, + 0x91, 0x35, 0xe6, 0xbf, 0x44, 0xcb, 0x7c, 0xe9, 0x14, 0xed, 0x05, 0x2d, 0xf0, 0x1f, 0x0e, 0x2e, + 0xc3, 0x54, 0x5d, 0x71, 0xc8, 0x81, 0x72, 0x24, 0x1b, 0xa6, 0xc6, 0x78, 0x9f, 0x61, 0x0f, 0x56, + 0x5c, 0x3c, 0x39, 0x9e, 0xcb, 0xae, 0xf2, 0xa2, 0x4d, 0x53, 0xe3, 0x12, 0x18, 0xe7, 0xbf, 0x70, + 0xb6, 0x1e, 0x28, 0xd0, 0xd0, 0x12, 0x4c, 0xb2, 0x47, 0xb8, 0x4c, 0xfe, 0xd2, 0x9b, 0xf0, 0x90, + 0xfb, 0x9d, 0x61, 0x88, 0xf7, 0xe0, 0x30, 0x7b, 0xb8, 0xcb, 0x7d, 0x1c, 0xee, 0x26, 0x4c, 0x29, + 0xf6, 0x91, 0xa1, 0x32, 0x0e, 0x13, 0xc3, 0x6e, 0xdb, 0xcc, 0x45, 0x4e, 0xe1, 0x1c, 0x03, 0x97, + 0x5d, 0xa8, 0x78, 0xf1, 0xe5, 0xb7, 0x24, 0x98, 0xec, 0x48, 0xa5, 0x7c, 0xa7, 0xeb, 0xf0, 0xed, + 0x42, 0xb8, 0x7b, 0xde, 0x2f, 0xeb, 0x28, 0x25, 0x24, 0xe0, 0x66, 0x42, 0xcc, 0xf5, 0x77, 0xef, + 0x58, 0x80, 0xec, 0xe6, 0x8e, 0xb8, 0x68, 0x6f, 0x27, 0x7f, 0xe7, 0xbb, 0x73, 0xb1, 0xe2, 0x4f, + 0x13, 0x90, 0xed, 0x4c, 0x99, 0x5c, 0xeb, 0xea, 0x57, 0x98, 0x49, 0xec, 0xc0, 0x58, 0x18, 0xf0, + 0x10, 0x44, 0xda, 0x7d, 0xba, 0xcb, 0xed, 0xe6, 0xfc, 0x80, 0x23, 0xc6, 0x60, 0x3f, 0x7d, 0xc4, + 0xd9, 0x1f, 0xc7, 0xbd, 0x99, 0xbf, 0x00, 0x63, 0xec, 0x11, 0x6c, 0xd1, 0xb5, 0xb0, 0xdb, 0x08, + 0x2b, 0xb4, 0x1c, 0xf3, 0x6a, 0xd4, 0x52, 0xd4, 0x4e, 0x75, 0xff, 0xde, 0x03, 0x9c, 0xe2, 0x91, + 0xe6, 0x53, 0xde, 0x13, 0x67, 0x07, 0x8e, 0x8d, 0x06, 0x51, 0x1d, 0xf1, 0x96, 0xb6, 0xfb, 0x14, + 0xf3, 0xb5, 0x6e, 0x12, 0xe2, 0xe5, 0xed, 0x05, 0x2c, 0x5e, 0xde, 0x0e, 0x24, 0xa7, 0xe4, 0x3c, + 0x12, 0x6c, 0x62, 0xf1, 0x14, 0x26, 0x21, 0xea, 0x6f, 0x48, 0x90, 0x67, 0xd3, 0xe8, 0x01, 0x21, + 0x5a, 0x24, 0x5a, 0xe8, 0xe6, 0xcd, 0xc4, 0x47, 0xce, 0x9b, 0x29, 0x2a, 0x90, 0xf3, 0xfa, 0xc0, + 0xdf, 0xa4, 0x1d, 0xf0, 0x7c, 0xc2, 0xa9, 0xae, 0xb5, 0x15, 0x7f, 0x4f, 0x82, 0x19, 0xaf, 0x0d, + 0xe6, 0xe7, 0xf3, 0xcc, 0xef, 0x53, 0x64, 0x32, 0x62, 0xf6, 0xde, 0x36, 0x8d, 0xf3, 0xd8, 0x55, + 0xa7, 0x91, 0x34, 0x08, 0x89, 0xd3, 0x6c, 0x10, 0xf1, 0xa3, 0x56, 0xab, 0xb2, 0x97, 0xb8, 0xf9, + 0x6f, 0xbb, 0xf8, 0x20, 0xc0, 0x01, 0xa6, 0xac, 0x74, 0x98, 0x23, 0x69, 0xb5, 0x3b, 0x4c, 0x56, + 0xb9, 0xf8, 0xb7, 0x52, 0x90, 0xd0, 0x73, 0xea, 0x37, 0xdc, 0x87, 0xc4, 0x73, 0xa5, 0x31, 0xe8, + 0x30, 0xbf, 0x83, 0xf5, 0x98, 0xd6, 0x46, 0x0f, 0x80, 0x5f, 0xf7, 0xe0, 0xe9, 0xf2, 0xf1, 0xfe, + 0xc1, 0x5d, 0x2f, 0x4b, 0x71, 0x00, 0x13, 0xbd, 0xe5, 0x8e, 0x22, 0x31, 0xbc, 0xf9, 0xe0, 0x24, + 0x7d, 0x3b, 0xf9, 0xd1, 0x77, 0xe7, 0xa4, 0x3b, 0x55, 0x98, 0x09, 0x59, 0x84, 0x50, 0x0e, 0xa0, + 0xbc, 0xb5, 0x59, 0x5d, 0xab, 0xd6, 0x56, 0x36, 0x6b, 0xee, 0x73, 0xdf, 0x4b, 0xcb, 0xf2, 0xce, + 0x66, 0x79, 0x6b, 0x63, 0x63, 0xad, 0x56, 0x5b, 0x59, 0xce, 0x4b, 0x28, 0x0f, 0x93, 0x6b, 0x9b, + 0x81, 0x7a, 0xe2, 0x95, 0xef, 0x3b, 0x3f, 0x07, 0xe0, 0xbf, 0xa2, 0x48, 0x69, 0xad, 0xaf, 0x3c, + 0x91, 0x1f, 0x2f, 0x3d, 0xda, 0x59, 0xa9, 0xe6, 0x63, 0x08, 0x41, 0xae, 0xb4, 0x54, 0x2b, 0x57, + 0x64, 0xbc, 0x52, 0xdd, 0xde, 0xda, 0xac, 0xae, 0xe4, 0x25, 0x81, 0xf7, 0x3e, 0x64, 0x02, 0xef, + 0x31, 0xd0, 0x8a, 0xdb, 0x3b, 0xd5, 0x8a, 0x5c, 0x5b, 0xdb, 0x58, 0xa9, 0xd6, 0x96, 0x36, 0xb6, + 0xf3, 0x31, 0x4a, 0x8c, 0xc1, 0x96, 0x4a, 0x5b, 0xb8, 0x96, 0x97, 0xbc, 0xef, 0xda, 0xd6, 0x4e, + 0xb9, 0x92, 0x8f, 0x7b, 0xdf, 0xef, 0xed, 0xac, 0xe0, 0x27, 0xf9, 0x84, 0x20, 0xac, 0xc0, 0xd9, + 0xd0, 0x8b, 0x31, 0x28, 0x03, 0x13, 0x3b, 0x06, 0x7b, 0x13, 0x20, 0x1f, 0x43, 0xd9, 0xc0, 0xdd, + 0x98, 0xbc, 0x84, 0x52, 0xfc, 0x0e, 0x46, 0x3e, 0x8e, 0xc6, 0x21, 0x5e, 0xbd, 0x9f, 0x4f, 0xa0, + 0x29, 0xc8, 0x04, 0x2e, 0x98, 0xe4, 0x93, 0x28, 0x2d, 0x2e, 0x0f, 0xe4, 0xc7, 0xee, 0x5c, 0x81, + 0x40, 0x6e, 0x36, 0x02, 0x18, 0x7f, 0xa4, 0x38, 0xc4, 0x76, 0xf2, 0x31, 0x34, 0x01, 0x89, 0xa5, + 0x46, 0x23, 0x2f, 0x2d, 0xfe, 0x40, 0x82, 0x94, 0xfb, 0xde, 0x1a, 0x7a, 0x04, 0x63, 0x3c, 0x60, + 0x9b, 0xeb, 0x6f, 0xe8, 0x99, 0xad, 0x98, 0x9d, 0x1f, 0xb6, 0x12, 0x14, 0x63, 0xe8, 0x7d, 0xf1, + 0xc6, 0x3f, 0x95, 0x32, 0xba, 0x3a, 0x48, 0x07, 0x5c, 0xaa, 0x83, 0x15, 0x85, 0xea, 0x75, 0x31, + 0xf6, 0x59, 0xa9, 0x74, 0xe5, 0xa3, 0x7f, 0xbd, 0x1c, 0xfb, 0xe8, 0xe4, 0xb2, 0xf4, 0x93, 0x93, + 0xcb, 0xd2, 0x3f, 0x9d, 0x5c, 0x96, 0xfe, 0xe5, 0xe4, 0xb2, 0xf4, 0xeb, 0xff, 0x76, 0x39, 0xf6, + 0xc1, 0x84, 0xc0, 0xda, 0x1d, 0x67, 0xff, 0x66, 0xe0, 0xfe, 0xff, 0x04, 0x00, 0x00, 0xff, 0xff, + 0x92, 0x79, 0x1f, 0xd9, 0x6b, 0x61, 0x00, 0x00, } diff --git a/pkg/roachpb/api.proto b/pkg/roachpb/api.proto index 49afd7ef67d3..33c41b0c37cf 100644 --- a/pkg/roachpb/api.proto +++ b/pkg/roachpb/api.proto @@ -619,6 +619,21 @@ message AdminChangeReplicasResponse { ResponseHeader header = 1 [(gogoproto.nullable) = false, (gogoproto.embed) = true]; } +// An AdminRelocateRangeRequest is the argument to the AdminRelocateRange() +// method. Relocates the replicas for a range to the specified target stores. +// The first store in the list of targets becomes the new leaseholder. +message AdminRelocateRangeRequest { + option (gogoproto.equal) = true; + + RequestHeader header = 1 [(gogoproto.nullable) = false, (gogoproto.embed) = true]; + repeated ReplicationTarget targets = 2 [(gogoproto.nullable) = false]; + // TODO(a-robinson): Add "reason"/"details" string fields? +} + +message AdminRelocateRangeResponse { + ResponseHeader header = 1 [(gogoproto.nullable) = false, (gogoproto.embed) = true]; +} + // A HeartbeatTxnRequest is arguments to the HeartbeatTxn() // method. It's sent by transaction coordinators to let the system // know that the transaction is still ongoing. Note that this @@ -1349,6 +1364,7 @@ message RequestUnion { AdminMergeRequest admin_merge = 11; AdminTransferLeaseRequest admin_transfer_lease = 29; AdminChangeReplicasRequest admin_change_replicas = 35; + AdminRelocateRangeRequest admin_relocate_range = 45; HeartbeatTxnRequest heartbeat_txn = 12; GCRequest gc = 13; PushTxnRequest push_txn = 14; @@ -1397,6 +1413,7 @@ message ResponseUnion { AdminMergeResponse admin_merge = 11; AdminTransferLeaseResponse admin_transfer_lease = 29; AdminChangeReplicasResponse admin_change_replicas = 35; + AdminRelocateRangeResponse admin_relocate_range = 45; HeartbeatTxnResponse heartbeat_txn = 12; GCResponse gc = 13; PushTxnResponse push_txn = 14; diff --git a/pkg/roachpb/batch_generated.go b/pkg/roachpb/batch_generated.go index c3051dc1c2ba..f6c396c3553d 100644 --- a/pkg/roachpb/batch_generated.go +++ b/pkg/roachpb/batch_generated.go @@ -106,6 +106,8 @@ func (ru RequestUnion) GetInner() Request { return t.AdminTransferLease case *RequestUnion_AdminChangeReplicas: return t.AdminChangeReplicas + case *RequestUnion_AdminRelocateRange: + return t.AdminRelocateRange case *RequestUnion_HeartbeatTxn: return t.HeartbeatTxn case *RequestUnion_Gc: @@ -194,6 +196,8 @@ func (ru ResponseUnion) GetInner() Response { return t.AdminTransferLease case *ResponseUnion_AdminChangeReplicas: return t.AdminChangeReplicas + case *ResponseUnion_AdminRelocateRange: + return t.AdminRelocateRange case *ResponseUnion_HeartbeatTxn: return t.HeartbeatTxn case *ResponseUnion_Gc: @@ -350,6 +354,8 @@ func (ru *RequestUnion) SetInner(r Request) bool { union = &RequestUnion_AdminTransferLease{t} case *AdminChangeReplicasRequest: union = &RequestUnion_AdminChangeReplicas{t} + case *AdminRelocateRangeRequest: + union = &RequestUnion_AdminRelocateRange{t} case *HeartbeatTxnRequest: union = &RequestUnion_HeartbeatTxn{t} case *GCRequest: @@ -441,6 +447,8 @@ func (ru *ResponseUnion) SetInner(r Response) bool { union = &ResponseUnion_AdminTransferLease{t} case *AdminChangeReplicasResponse: union = &ResponseUnion_AdminChangeReplicas{t} + case *AdminRelocateRangeResponse: + union = &ResponseUnion_AdminRelocateRange{t} case *HeartbeatTxnResponse: union = &ResponseUnion_HeartbeatTxn{t} case *GCResponse: @@ -498,7 +506,7 @@ func (ru *ResponseUnion) SetInner(r Response) bool { return true } -type reqCounts [40]int32 +type reqCounts [41]int32 // getReqCounts returns the number of times each // request type appears in the batch. @@ -534,58 +542,60 @@ func (ba *BatchRequest) getReqCounts() reqCounts { counts[12]++ case *RequestUnion_AdminChangeReplicas: counts[13]++ - case *RequestUnion_HeartbeatTxn: + case *RequestUnion_AdminRelocateRange: counts[14]++ - case *RequestUnion_Gc: + case *RequestUnion_HeartbeatTxn: counts[15]++ - case *RequestUnion_PushTxn: + case *RequestUnion_Gc: counts[16]++ - case *RequestUnion_ResolveIntent: + case *RequestUnion_PushTxn: counts[17]++ - case *RequestUnion_ResolveIntentRange: + case *RequestUnion_ResolveIntent: counts[18]++ - case *RequestUnion_Merge: + case *RequestUnion_ResolveIntentRange: counts[19]++ - case *RequestUnion_TruncateLog: + case *RequestUnion_Merge: counts[20]++ - case *RequestUnion_RequestLease: + case *RequestUnion_TruncateLog: counts[21]++ - case *RequestUnion_ReverseScan: + case *RequestUnion_RequestLease: counts[22]++ - case *RequestUnion_ComputeChecksum: + case *RequestUnion_ReverseScan: counts[23]++ - case *RequestUnion_CheckConsistency: + case *RequestUnion_ComputeChecksum: counts[24]++ - case *RequestUnion_InitPut: + case *RequestUnion_CheckConsistency: counts[25]++ - case *RequestUnion_TransferLease: + case *RequestUnion_InitPut: counts[26]++ - case *RequestUnion_LeaseInfo: + case *RequestUnion_TransferLease: counts[27]++ - case *RequestUnion_WriteBatch: + case *RequestUnion_LeaseInfo: counts[28]++ - case *RequestUnion_Export: + case *RequestUnion_WriteBatch: counts[29]++ - case *RequestUnion_Import: + case *RequestUnion_Export: counts[30]++ - case *RequestUnion_QueryTxn: + case *RequestUnion_Import: counts[31]++ - case *RequestUnion_QueryIntent: + case *RequestUnion_QueryTxn: counts[32]++ - case *RequestUnion_AdminScatter: + case *RequestUnion_QueryIntent: counts[33]++ - case *RequestUnion_AddSstable: + case *RequestUnion_AdminScatter: counts[34]++ - case *RequestUnion_RecomputeStats: + case *RequestUnion_AddSstable: counts[35]++ - case *RequestUnion_Refresh: + case *RequestUnion_RecomputeStats: counts[36]++ - case *RequestUnion_RefreshRange: + case *RequestUnion_Refresh: counts[37]++ - case *RequestUnion_Subsume: + case *RequestUnion_RefreshRange: counts[38]++ - case *RequestUnion_RangeStats: + case *RequestUnion_Subsume: counts[39]++ + case *RequestUnion_RangeStats: + counts[40]++ default: panic(fmt.Sprintf("unsupported request: %+v", ru)) } @@ -608,6 +618,7 @@ var requestNames = []string{ "AdmMerge", "AdmTransferLease", "AdmChangeReplicas", + "AdmRelocateRng", "HeartbeatTxn", "Gc", "PushTxn", @@ -717,6 +728,10 @@ type adminChangeReplicasResponseAlloc struct { union ResponseUnion_AdminChangeReplicas resp AdminChangeReplicasResponse } +type adminRelocateRangeResponseAlloc struct { + union ResponseUnion_AdminRelocateRange + resp AdminRelocateRangeResponse +} type heartbeatTxnResponseAlloc struct { union ResponseUnion_HeartbeatTxn resp HeartbeatTxnResponse @@ -841,32 +856,33 @@ func (ba *BatchRequest) CreateReply() *BatchResponse { var buf11 []adminMergeResponseAlloc var buf12 []adminTransferLeaseResponseAlloc var buf13 []adminChangeReplicasResponseAlloc - var buf14 []heartbeatTxnResponseAlloc - var buf15 []gCResponseAlloc - var buf16 []pushTxnResponseAlloc - var buf17 []resolveIntentResponseAlloc - var buf18 []resolveIntentRangeResponseAlloc - var buf19 []mergeResponseAlloc - var buf20 []truncateLogResponseAlloc - var buf21 []requestLeaseResponseAlloc - var buf22 []reverseScanResponseAlloc - var buf23 []computeChecksumResponseAlloc - var buf24 []checkConsistencyResponseAlloc - var buf25 []initPutResponseAlloc - var buf26 []requestLeaseResponseAlloc - var buf27 []leaseInfoResponseAlloc - var buf28 []writeBatchResponseAlloc - var buf29 []exportResponseAlloc - var buf30 []importResponseAlloc - var buf31 []queryTxnResponseAlloc - var buf32 []queryIntentResponseAlloc - var buf33 []adminScatterResponseAlloc - var buf34 []addSSTableResponseAlloc - var buf35 []recomputeStatsResponseAlloc - var buf36 []refreshResponseAlloc - var buf37 []refreshRangeResponseAlloc - var buf38 []subsumeResponseAlloc - var buf39 []rangeStatsResponseAlloc + var buf14 []adminRelocateRangeResponseAlloc + var buf15 []heartbeatTxnResponseAlloc + var buf16 []gCResponseAlloc + var buf17 []pushTxnResponseAlloc + var buf18 []resolveIntentResponseAlloc + var buf19 []resolveIntentRangeResponseAlloc + var buf20 []mergeResponseAlloc + var buf21 []truncateLogResponseAlloc + var buf22 []requestLeaseResponseAlloc + var buf23 []reverseScanResponseAlloc + var buf24 []computeChecksumResponseAlloc + var buf25 []checkConsistencyResponseAlloc + var buf26 []initPutResponseAlloc + var buf27 []requestLeaseResponseAlloc + var buf28 []leaseInfoResponseAlloc + var buf29 []writeBatchResponseAlloc + var buf30 []exportResponseAlloc + var buf31 []importResponseAlloc + var buf32 []queryTxnResponseAlloc + var buf33 []queryIntentResponseAlloc + var buf34 []adminScatterResponseAlloc + var buf35 []addSSTableResponseAlloc + var buf36 []recomputeStatsResponseAlloc + var buf37 []refreshResponseAlloc + var buf38 []refreshRangeResponseAlloc + var buf39 []subsumeResponseAlloc + var buf40 []rangeStatsResponseAlloc for i, r := range ba.Requests { switch r.GetValue().(type) { @@ -968,188 +984,195 @@ func (ba *BatchRequest) CreateReply() *BatchResponse { buf13[0].union.AdminChangeReplicas = &buf13[0].resp br.Responses[i].Value = &buf13[0].union buf13 = buf13[1:] - case *RequestUnion_HeartbeatTxn: + case *RequestUnion_AdminRelocateRange: if buf14 == nil { - buf14 = make([]heartbeatTxnResponseAlloc, counts[14]) + buf14 = make([]adminRelocateRangeResponseAlloc, counts[14]) } - buf14[0].union.HeartbeatTxn = &buf14[0].resp + buf14[0].union.AdminRelocateRange = &buf14[0].resp br.Responses[i].Value = &buf14[0].union buf14 = buf14[1:] - case *RequestUnion_Gc: + case *RequestUnion_HeartbeatTxn: if buf15 == nil { - buf15 = make([]gCResponseAlloc, counts[15]) + buf15 = make([]heartbeatTxnResponseAlloc, counts[15]) } - buf15[0].union.Gc = &buf15[0].resp + buf15[0].union.HeartbeatTxn = &buf15[0].resp br.Responses[i].Value = &buf15[0].union buf15 = buf15[1:] - case *RequestUnion_PushTxn: + case *RequestUnion_Gc: if buf16 == nil { - buf16 = make([]pushTxnResponseAlloc, counts[16]) + buf16 = make([]gCResponseAlloc, counts[16]) } - buf16[0].union.PushTxn = &buf16[0].resp + buf16[0].union.Gc = &buf16[0].resp br.Responses[i].Value = &buf16[0].union buf16 = buf16[1:] - case *RequestUnion_ResolveIntent: + case *RequestUnion_PushTxn: if buf17 == nil { - buf17 = make([]resolveIntentResponseAlloc, counts[17]) + buf17 = make([]pushTxnResponseAlloc, counts[17]) } - buf17[0].union.ResolveIntent = &buf17[0].resp + buf17[0].union.PushTxn = &buf17[0].resp br.Responses[i].Value = &buf17[0].union buf17 = buf17[1:] - case *RequestUnion_ResolveIntentRange: + case *RequestUnion_ResolveIntent: if buf18 == nil { - buf18 = make([]resolveIntentRangeResponseAlloc, counts[18]) + buf18 = make([]resolveIntentResponseAlloc, counts[18]) } - buf18[0].union.ResolveIntentRange = &buf18[0].resp + buf18[0].union.ResolveIntent = &buf18[0].resp br.Responses[i].Value = &buf18[0].union buf18 = buf18[1:] - case *RequestUnion_Merge: + case *RequestUnion_ResolveIntentRange: if buf19 == nil { - buf19 = make([]mergeResponseAlloc, counts[19]) + buf19 = make([]resolveIntentRangeResponseAlloc, counts[19]) } - buf19[0].union.Merge = &buf19[0].resp + buf19[0].union.ResolveIntentRange = &buf19[0].resp br.Responses[i].Value = &buf19[0].union buf19 = buf19[1:] - case *RequestUnion_TruncateLog: + case *RequestUnion_Merge: if buf20 == nil { - buf20 = make([]truncateLogResponseAlloc, counts[20]) + buf20 = make([]mergeResponseAlloc, counts[20]) } - buf20[0].union.TruncateLog = &buf20[0].resp + buf20[0].union.Merge = &buf20[0].resp br.Responses[i].Value = &buf20[0].union buf20 = buf20[1:] - case *RequestUnion_RequestLease: + case *RequestUnion_TruncateLog: if buf21 == nil { - buf21 = make([]requestLeaseResponseAlloc, counts[21]) + buf21 = make([]truncateLogResponseAlloc, counts[21]) } - buf21[0].union.RequestLease = &buf21[0].resp + buf21[0].union.TruncateLog = &buf21[0].resp br.Responses[i].Value = &buf21[0].union buf21 = buf21[1:] - case *RequestUnion_ReverseScan: + case *RequestUnion_RequestLease: if buf22 == nil { - buf22 = make([]reverseScanResponseAlloc, counts[22]) + buf22 = make([]requestLeaseResponseAlloc, counts[22]) } - buf22[0].union.ReverseScan = &buf22[0].resp + buf22[0].union.RequestLease = &buf22[0].resp br.Responses[i].Value = &buf22[0].union buf22 = buf22[1:] - case *RequestUnion_ComputeChecksum: + case *RequestUnion_ReverseScan: if buf23 == nil { - buf23 = make([]computeChecksumResponseAlloc, counts[23]) + buf23 = make([]reverseScanResponseAlloc, counts[23]) } - buf23[0].union.ComputeChecksum = &buf23[0].resp + buf23[0].union.ReverseScan = &buf23[0].resp br.Responses[i].Value = &buf23[0].union buf23 = buf23[1:] - case *RequestUnion_CheckConsistency: + case *RequestUnion_ComputeChecksum: if buf24 == nil { - buf24 = make([]checkConsistencyResponseAlloc, counts[24]) + buf24 = make([]computeChecksumResponseAlloc, counts[24]) } - buf24[0].union.CheckConsistency = &buf24[0].resp + buf24[0].union.ComputeChecksum = &buf24[0].resp br.Responses[i].Value = &buf24[0].union buf24 = buf24[1:] - case *RequestUnion_InitPut: + case *RequestUnion_CheckConsistency: if buf25 == nil { - buf25 = make([]initPutResponseAlloc, counts[25]) + buf25 = make([]checkConsistencyResponseAlloc, counts[25]) } - buf25[0].union.InitPut = &buf25[0].resp + buf25[0].union.CheckConsistency = &buf25[0].resp br.Responses[i].Value = &buf25[0].union buf25 = buf25[1:] - case *RequestUnion_TransferLease: + case *RequestUnion_InitPut: if buf26 == nil { - buf26 = make([]requestLeaseResponseAlloc, counts[26]) + buf26 = make([]initPutResponseAlloc, counts[26]) } - buf26[0].union.RequestLease = &buf26[0].resp + buf26[0].union.InitPut = &buf26[0].resp br.Responses[i].Value = &buf26[0].union buf26 = buf26[1:] - case *RequestUnion_LeaseInfo: + case *RequestUnion_TransferLease: if buf27 == nil { - buf27 = make([]leaseInfoResponseAlloc, counts[27]) + buf27 = make([]requestLeaseResponseAlloc, counts[27]) } - buf27[0].union.LeaseInfo = &buf27[0].resp + buf27[0].union.RequestLease = &buf27[0].resp br.Responses[i].Value = &buf27[0].union buf27 = buf27[1:] - case *RequestUnion_WriteBatch: + case *RequestUnion_LeaseInfo: if buf28 == nil { - buf28 = make([]writeBatchResponseAlloc, counts[28]) + buf28 = make([]leaseInfoResponseAlloc, counts[28]) } - buf28[0].union.WriteBatch = &buf28[0].resp + buf28[0].union.LeaseInfo = &buf28[0].resp br.Responses[i].Value = &buf28[0].union buf28 = buf28[1:] - case *RequestUnion_Export: + case *RequestUnion_WriteBatch: if buf29 == nil { - buf29 = make([]exportResponseAlloc, counts[29]) + buf29 = make([]writeBatchResponseAlloc, counts[29]) } - buf29[0].union.Export = &buf29[0].resp + buf29[0].union.WriteBatch = &buf29[0].resp br.Responses[i].Value = &buf29[0].union buf29 = buf29[1:] - case *RequestUnion_Import: + case *RequestUnion_Export: if buf30 == nil { - buf30 = make([]importResponseAlloc, counts[30]) + buf30 = make([]exportResponseAlloc, counts[30]) } - buf30[0].union.Import = &buf30[0].resp + buf30[0].union.Export = &buf30[0].resp br.Responses[i].Value = &buf30[0].union buf30 = buf30[1:] - case *RequestUnion_QueryTxn: + case *RequestUnion_Import: if buf31 == nil { - buf31 = make([]queryTxnResponseAlloc, counts[31]) + buf31 = make([]importResponseAlloc, counts[31]) } - buf31[0].union.QueryTxn = &buf31[0].resp + buf31[0].union.Import = &buf31[0].resp br.Responses[i].Value = &buf31[0].union buf31 = buf31[1:] - case *RequestUnion_QueryIntent: + case *RequestUnion_QueryTxn: if buf32 == nil { - buf32 = make([]queryIntentResponseAlloc, counts[32]) + buf32 = make([]queryTxnResponseAlloc, counts[32]) } - buf32[0].union.QueryIntent = &buf32[0].resp + buf32[0].union.QueryTxn = &buf32[0].resp br.Responses[i].Value = &buf32[0].union buf32 = buf32[1:] - case *RequestUnion_AdminScatter: + case *RequestUnion_QueryIntent: if buf33 == nil { - buf33 = make([]adminScatterResponseAlloc, counts[33]) + buf33 = make([]queryIntentResponseAlloc, counts[33]) } - buf33[0].union.AdminScatter = &buf33[0].resp + buf33[0].union.QueryIntent = &buf33[0].resp br.Responses[i].Value = &buf33[0].union buf33 = buf33[1:] - case *RequestUnion_AddSstable: + case *RequestUnion_AdminScatter: if buf34 == nil { - buf34 = make([]addSSTableResponseAlloc, counts[34]) + buf34 = make([]adminScatterResponseAlloc, counts[34]) } - buf34[0].union.AddSstable = &buf34[0].resp + buf34[0].union.AdminScatter = &buf34[0].resp br.Responses[i].Value = &buf34[0].union buf34 = buf34[1:] - case *RequestUnion_RecomputeStats: + case *RequestUnion_AddSstable: if buf35 == nil { - buf35 = make([]recomputeStatsResponseAlloc, counts[35]) + buf35 = make([]addSSTableResponseAlloc, counts[35]) } - buf35[0].union.RecomputeStats = &buf35[0].resp + buf35[0].union.AddSstable = &buf35[0].resp br.Responses[i].Value = &buf35[0].union buf35 = buf35[1:] - case *RequestUnion_Refresh: + case *RequestUnion_RecomputeStats: if buf36 == nil { - buf36 = make([]refreshResponseAlloc, counts[36]) + buf36 = make([]recomputeStatsResponseAlloc, counts[36]) } - buf36[0].union.Refresh = &buf36[0].resp + buf36[0].union.RecomputeStats = &buf36[0].resp br.Responses[i].Value = &buf36[0].union buf36 = buf36[1:] - case *RequestUnion_RefreshRange: + case *RequestUnion_Refresh: if buf37 == nil { - buf37 = make([]refreshRangeResponseAlloc, counts[37]) + buf37 = make([]refreshResponseAlloc, counts[37]) } - buf37[0].union.RefreshRange = &buf37[0].resp + buf37[0].union.Refresh = &buf37[0].resp br.Responses[i].Value = &buf37[0].union buf37 = buf37[1:] - case *RequestUnion_Subsume: + case *RequestUnion_RefreshRange: if buf38 == nil { - buf38 = make([]subsumeResponseAlloc, counts[38]) + buf38 = make([]refreshRangeResponseAlloc, counts[38]) } - buf38[0].union.Subsume = &buf38[0].resp + buf38[0].union.RefreshRange = &buf38[0].resp br.Responses[i].Value = &buf38[0].union buf38 = buf38[1:] - case *RequestUnion_RangeStats: + case *RequestUnion_Subsume: if buf39 == nil { - buf39 = make([]rangeStatsResponseAlloc, counts[39]) + buf39 = make([]subsumeResponseAlloc, counts[39]) } - buf39[0].union.RangeStats = &buf39[0].resp + buf39[0].union.Subsume = &buf39[0].resp br.Responses[i].Value = &buf39[0].union buf39 = buf39[1:] + case *RequestUnion_RangeStats: + if buf40 == nil { + buf40 = make([]rangeStatsResponseAlloc, counts[40]) + } + buf40[0].union.RangeStats = &buf40[0].resp + br.Responses[i].Value = &buf40[0].union + buf40 = buf40[1:] default: panic(fmt.Sprintf("unsupported request: %+v", r)) } diff --git a/pkg/roachpb/method.go b/pkg/roachpb/method.go index 27f6c3754c9c..7e42b1694f19 100644 --- a/pkg/roachpb/method.go +++ b/pkg/roachpb/method.go @@ -70,6 +70,9 @@ const ( AdminTransferLease // AdminChangeReplicas is called to add or remove replicas for a range. AdminChangeReplicas + // AdminRelocateRange is called to relocate the replicas for a range onto a + // specified list of stores. + AdminRelocateRange // HeartbeatTxn sends a periodic heartbeat to extant // transaction rows to indicate the client is still alive and // the transaction should not be considered abandoned. diff --git a/pkg/roachpb/method_string.go b/pkg/roachpb/method_string.go index 5f6e3d6f4a6c..d778e0db385c 100644 --- a/pkg/roachpb/method_string.go +++ b/pkg/roachpb/method_string.go @@ -4,9 +4,9 @@ package roachpb import "strconv" -const _Method_name = "GetPutConditionalPutIncrementDeleteDeleteRangeClearRangeScanReverseScanBeginTransactionEndTransactionAdminSplitAdminMergeAdminTransferLeaseAdminChangeReplicasHeartbeatTxnGCPushTxnQueryTxnQueryIntentResolveIntentResolveIntentRangeMergeTruncateLogRequestLeaseTransferLeaseLeaseInfoComputeChecksumCheckConsistencyInitPutWriteBatchExportImportAdminScatterAddSSTableRecomputeStatsRefreshRefreshRangeSubsumeRangeStats" +const _Method_name = "GetPutConditionalPutIncrementDeleteDeleteRangeClearRangeScanReverseScanBeginTransactionEndTransactionAdminSplitAdminMergeAdminTransferLeaseAdminChangeReplicasAdminRelocateRangeHeartbeatTxnGCPushTxnQueryTxnQueryIntentResolveIntentResolveIntentRangeMergeTruncateLogRequestLeaseTransferLeaseLeaseInfoComputeChecksumCheckConsistencyInitPutWriteBatchExportImportAdminScatterAddSSTableRecomputeStatsRefreshRefreshRangeSubsumeRangeStats" -var _Method_index = [...]uint16{0, 3, 6, 20, 29, 35, 46, 56, 60, 71, 87, 101, 111, 121, 139, 158, 170, 172, 179, 187, 198, 211, 229, 234, 245, 257, 270, 279, 294, 310, 317, 327, 333, 339, 351, 361, 375, 382, 394, 401, 411} +var _Method_index = [...]uint16{0, 3, 6, 20, 29, 35, 46, 56, 60, 71, 87, 101, 111, 121, 139, 158, 176, 188, 190, 197, 205, 216, 229, 247, 252, 263, 275, 288, 297, 312, 328, 335, 345, 351, 357, 369, 379, 393, 400, 412, 419, 429} func (i Method) String() string { if i < 0 || i >= Method(len(_Method_index)-1) { diff --git a/pkg/sql/crdb_internal.go b/pkg/sql/crdb_internal.go index 3c9d2697a130..3a606445e8fb 100644 --- a/pkg/sql/crdb_internal.go +++ b/pkg/sql/crdb_internal.go @@ -1603,12 +1603,19 @@ CREATE TABLE crdb_internal.ranges ( if err := r.ValueProto(&desc); err != nil { return nil, err } + + var replicas []int + for _, rd := range desc.Replicas { + replicas = append(replicas, int(rd.StoreID)) + } + sort.Ints(replicas) arr := tree.NewDArray(types.Int) - for _, replica := range desc.Replicas { - if err := arr.Append(tree.NewDInt(tree.DInt(replica.StoreID))); err != nil { + for _, replica := range replicas { + if err := arr.Append(tree.NewDInt(tree.DInt(replica))); err != nil { return nil, err } } + var dbName, tableName, indexName string if _, id, err := keys.DecodeTablePrefix(desc.StartKey.AsRawKey()); err == nil { parent := parents[id] diff --git a/pkg/sql/logictest/testdata/logic_test/ranges b/pkg/sql/logictest/testdata/logic_test/ranges index 35da2e1a3dc5..ac1ba33c7356 100644 --- a/pkg/sql/logictest/testdata/logic_test/ranges +++ b/pkg/sql/logictest/testdata/logic_test/ranges @@ -53,7 +53,7 @@ NULL /1 {1} 1 /10 NULL {1} 1 statement ok -ALTER TABLE t EXPERIMENTAL_RELOCATE VALUES (ARRAY[1,2,3], 5, 1), (ARRAY[5,2,3], 5, 2), (ARRAY[4,1,2], 5, 3) +ALTER TABLE t EXPERIMENTAL_RELOCATE VALUES (ARRAY[1,2,3], 5, 1), (ARRAY[5,2,3], 5, 2), (ARRAY[4,2,1], 5, 3) statement ok ALTER TABLE t EXPERIMENTAL_RELOCATE VALUES (ARRAY[3,4], 4) @@ -280,11 +280,11 @@ SELECT * FROM crdb_internal.ranges ---- range_id start_key start_pretty end_key end_pretty database table index replicas lease_holder 1 · /Min [189 137 137] /Table/53/1/1 · · · {1} 1 -2 [189 137 137] /Table/53/1/1 [189 137 141 137] /Table/53/1/5/1 test t · {4,3} 3 -11 [189 137 141 137] /Table/53/1/5/1 [189 137 141 138] /Table/53/1/5/2 test t · {3,1,2} 1 -12 [189 137 141 138] /Table/53/1/5/2 [189 137 141 139] /Table/53/1/5/3 test t · {3,5,2} 5 -13 [189 137 141 139] /Table/53/1/5/3 [189 137 143 144 254 190 137 145] /Table/53/1/7/8/#/54/1/9 test t · {4,1,2} 4 -14 [189 137 143 144 254 190 137 145] /Table/53/1/7/8/#/54/1/9 [189 137 146] /Table/53/1/10 test t · {4,1,2} 4 +2 [189 137 137] /Table/53/1/1 [189 137 141 137] /Table/53/1/5/1 test t · {3,4} 3 +11 [189 137 141 137] /Table/53/1/5/1 [189 137 141 138] /Table/53/1/5/2 test t · {1,2,3} 1 +12 [189 137 141 138] /Table/53/1/5/2 [189 137 141 139] /Table/53/1/5/3 test t · {2,3,5} 5 +13 [189 137 141 139] /Table/53/1/5/3 [189 137 143 144 254 190 137 145] /Table/53/1/7/8/#/54/1/9 test t · {1,2,4} 4 +14 [189 137 143 144 254 190 137 145] /Table/53/1/7/8/#/54/1/9 [189 137 146] /Table/53/1/10 test t · {1,2,4} 4 3 [189 137 146] /Table/53/1/10 [189 137 147] /Table/53/1/11 test t · {1} 1 8 [189 137 147] /Table/53/1/11 [189 137 151 152 254 191 138] /Table/53/1/15/16/#/55/2 test t · {1} 1 9 [189 137 151 152 254 191 138] /Table/53/1/15/16/#/55/2 [189 138 144] /Table/53/2/8 test t · {1} 1 diff --git a/pkg/sql/logictest/testdata/planner_test/distsql_agg b/pkg/sql/logictest/testdata/planner_test/distsql_agg index e191824d1645..cc391b0124d0 100644 --- a/pkg/sql/logictest/testdata/planner_test/distsql_agg +++ b/pkg/sql/logictest/testdata/planner_test/distsql_agg @@ -244,6 +244,22 @@ statement ok ALTER TABLE sorted_data EXPERIMENTAL_RELOCATE SELECT ARRAY[i%5+1], i FROM generate_series(0, 9) AS g(i) +# Verify data placement. +query TTTI colnames +SELECT start_key, end_key, replicas, lease_holder FROM [SHOW EXPERIMENTAL_RANGES FROM TABLE sorted_data] +---- +start_key end_key replicas lease_holder +NULL /1 {1} 1 +/1 /2 {2} 2 +/2 /3 {3} 3 +/3 /4 {4} 4 +/4 /5 {5} 5 +/5 /6 {1} 1 +/6 /7 {2} 2 +/7 /8 {3} 3 +/8 /9 {4} 4 +/9 NULL {5} 5 + query T SELECT url FROM [EXPLAIN (DISTSQL) SELECT a, max(b) FROM sorted_data GROUP BY a] ---- diff --git a/pkg/sql/logictest/testdata/planner_test/distsql_lookup_join b/pkg/sql/logictest/testdata/planner_test/distsql_lookup_join index 11426c07159d..80d03e19fca3 100644 --- a/pkg/sql/logictest/testdata/planner_test/distsql_lookup_join +++ b/pkg/sql/logictest/testdata/planner_test/distsql_lookup_join @@ -22,6 +22,22 @@ statement ok ALTER TABLE data EXPERIMENTAL_RELOCATE SELECT ARRAY[i%5+1], i FROM generate_series(0, 9) AS g(i) +# Verify data placement. +query TTITI colnames +SHOW EXPERIMENTAL_RANGES FROM TABLE data +---- +start_key end_key range_id replicas lease_holder +NULL /1 1 {1} 1 +/1 /2 2 {2} 2 +/2 /3 3 {3} 3 +/3 /4 4 {4} 4 +/4 /5 5 {5} 5 +/5 /6 6 {1} 1 +/6 /7 7 {2} 2 +/7 /8 8 {3} 3 +/8 /9 9 {4} 4 +/9 NULL 10 {5} 5 + statement ok CREATE TABLE distsql_lookup_test_1 (a INT, b INT, c INT, PRIMARY KEY (a, c)) @@ -162,6 +178,22 @@ statement ok ALTER TABLE multiples EXPERIMENTAL_RELOCATE SELECT ARRAY[i%5+1], i FROM generate_series(0, 9) AS g(i) +# Verify data placement. +query TTITI colnames +SHOW EXPERIMENTAL_RANGES FROM TABLE multiples +---- +start_key end_key range_id replicas lease_holder +NULL /1 10 {1} 1 +/1 /2 21 {2} 2 +/2 /3 22 {3} 3 +/3 /4 23 {4} 4 +/4 /5 24 {5} 5 +/5 /6 25 {1} 1 +/6 /7 26 {2} 2 +/7 /8 27 {3} 3 +/8 /9 28 {4} 4 +/9 NULL 29 {5} 5 + # Lookup join on covering secondary index query TTT colnames EXPLAIN SELECT t1.a, t2.c FROM multiples t1 JOIN multiples@bc t2 ON t1.a = t2.b diff --git a/pkg/sql/opt/exec/execbuilder/testdata/distsql_agg b/pkg/sql/opt/exec/execbuilder/testdata/distsql_agg index 574789394c06..1ffcd229544d 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/distsql_agg +++ b/pkg/sql/opt/exec/execbuilder/testdata/distsql_agg @@ -244,6 +244,22 @@ statement ok ALTER TABLE sorted_data EXPERIMENTAL_RELOCATE SELECT ARRAY[i%5+1], i FROM generate_series(0, 9) AS g(i) +# Verify data placement. +query TTTI colnames +SELECT start_key, end_key, replicas, lease_holder FROM [SHOW EXPERIMENTAL_RANGES FROM TABLE sorted_data] +---- +start_key end_key replicas lease_holder +NULL /1 {1} 1 +/1 /2 {2} 2 +/2 /3 {3} 3 +/3 /4 {4} 4 +/4 /5 {5} 5 +/5 /6 {1} 1 +/6 /7 {2} 2 +/7 /8 {3} 3 +/8 /9 {4} 4 +/9 NULL {5} 5 + query T SELECT url FROM [EXPLAIN (DISTSQL) SELECT a, max(b) FROM sorted_data GROUP BY a] ---- diff --git a/pkg/sql/relocate.go b/pkg/sql/relocate.go index 2ca41919d02d..ed5adb3741dd 100644 --- a/pkg/sql/relocate.go +++ b/pkg/sql/relocate.go @@ -27,7 +27,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sem/types" "github.com/cockroachdb/cockroach/pkg/sql/sqlbase" - "github.com/cockroachdb/cockroach/pkg/storage" "github.com/cockroachdb/cockroach/pkg/util/log" ) @@ -210,7 +209,7 @@ func (n *relocateNode) Next(params runParams) (bool, error) { return false, err } } else { - if err := storage.RelocateRange(params.ctx, params.p.ExecCfg().DB, rangeDesc, relocationTargets); err != nil { + if err := params.p.ExecCfg().DB.AdminRelocateRange(params.ctx, rowKey, relocationTargets); err != nil { return false, err } } diff --git a/pkg/storage/client_merge_test.go b/pkg/storage/client_merge_test.go index 62f4f6a8d1f8..4cf6b018c52e 100644 --- a/pkg/storage/client_merge_test.go +++ b/pkg/storage/client_merge_test.go @@ -2199,6 +2199,7 @@ func TestMergeQueue(t *testing.T) { mtc.storeConfig = &storeCfg mtc.Start(t, 2) defer mtc.Stop() + mtc.initGossipNetwork() // needed for the non-collocated case's rebalancing to work store := mtc.Store(0) store.SetMergeQueueActive(true) diff --git a/pkg/storage/merge_queue.go b/pkg/storage/merge_queue.go index 7181680009a3..1073d69c01d0 100644 --- a/pkg/storage/merge_queue.go +++ b/pkg/storage/merge_queue.go @@ -247,13 +247,9 @@ func (mq *mergeQueue) process( break } } - // TODO(benesch): RelocateRange needs to be made more robust. It cannot - // currently handle certain edge cases, like multiple stores on one node. It - // also adds all new replicas before removing any old replicas, rather than - // performing interleaved adds/removes, resulting in a moment where the - // number of replicas is potentially double the configured replication - // factor. - if err := RelocateRange(ctx, mq.db, rhsDesc, targets); err != nil { + // TODO(benesch): RelocateRange can sometimes fail if it needs to move a replica + // from one store to another store on the same node. + if err := mq.store.DB().AdminRelocateRange(ctx, rhsDesc.StartKey, targets); err != nil { return err } } diff --git a/pkg/storage/replica.go b/pkg/storage/replica.go index 2b97bffceae8..7ab0805e292b 100644 --- a/pkg/storage/replica.go +++ b/pkg/storage/replica.go @@ -2690,6 +2690,11 @@ func (r *Replica) executeAdminBatch( pErr = roachpb.NewError(err) resp = &roachpb.AdminChangeReplicasResponse{} + case *roachpb.AdminRelocateRangeRequest: + err := r.store.AdminRelocateRange(ctx, *r.Desc(), tArgs.Targets) + pErr = roachpb.NewError(err) + resp = &roachpb.AdminRelocateRangeResponse{} + case *roachpb.CheckConsistencyRequest: var reply roachpb.CheckConsistencyResponse reply, pErr = r.CheckConsistency(ctx, *tArgs) diff --git a/pkg/storage/replica_command.go b/pkg/storage/replica_command.go index 923fd1e5cd9d..29271d6e109f 100644 --- a/pkg/storage/replica_command.go +++ b/pkg/storage/replica_command.go @@ -1012,24 +1012,24 @@ func updateRangeDescriptor( return nil } -// RelocateRange relocates a given range to a given set of stores. The first -// store in the slice becomes the new leaseholder. +// AdminRelocateRange relocates a given range to a given set of stores. The +// first store in the slice becomes the new leaseholder. // -// This is best-effort; if replication queues are enabled and a change in -// membership happens at the same time, there will be errors. -func RelocateRange( - ctx context.Context, - db *client.DB, - rangeDesc roachpb.RangeDescriptor, - targets []roachpb.ReplicationTarget, +// This is best-effort; it's possible that the replicate queue on the +// leaseholder could take action at the same time, causing errors. +// +// TODO: How does this being a method on the leaseholder replica affect things? +func (s *Store) AdminRelocateRange( + ctx context.Context, rangeDesc roachpb.RangeDescriptor, targets []roachpb.ReplicationTarget, ) error { - // Step 1: Add any stores that don't already have a replica of the range. + // Step 1: Compute which replicas are to be added and which are to be removed. // - // TODO(radu): we can't have multiple replicas on different stores on the same - // node, which can lead to some odd corner cases where we would have to first - // remove some replicas (currently these cases fail). + // TODO(radu): we can't have multiple replicas on different stores on the + // same node, and this code doesn't do anything to specifically avoid that + // case (although the allocator will avoid even trying to send snapshots to + // such stores), so it could cause some failures. - var addTargets []roachpb.ReplicationTarget + var addTargets []roachpb.ReplicaDescriptor for _, t := range targets { found := false for _, replicaDesc := range rangeDesc.Replicas { @@ -1039,7 +1039,27 @@ func RelocateRange( } } if !found { - addTargets = append(addTargets, t) + addTargets = append(addTargets, roachpb.ReplicaDescriptor{ + NodeID: t.NodeID, + StoreID: t.StoreID, + }) + } + } + + var removeTargets []roachpb.ReplicaDescriptor + for _, replicaDesc := range rangeDesc.Replicas { + found := false + for _, t := range targets { + if replicaDesc.StoreID == t.StoreID && replicaDesc.NodeID == t.NodeID { + found = true + break + } + } + if !found { + removeTargets = append(removeTargets, roachpb.ReplicaDescriptor{ + NodeID: replicaDesc.NodeID, + StoreID: replicaDesc.StoreID, + }) } } @@ -1056,83 +1076,160 @@ func RelocateRange( return false } + transferLease := func() { + if err := s.DB().AdminTransferLease( + ctx, rangeDesc.StartKey.AsRawKey(), targets[0].StoreID, + ); err != nil { + log.Warningf(ctx, "while transferring lease: %s", err) + } + } + + sysCfg := s.cfg.Gossip.GetSystemConfig() + if sysCfg == nil { + return fmt.Errorf("no system config available, unable to perform RelocateRange") + } + zone, err := sysCfg.GetZoneConfigForKey(rangeDesc.StartKey) + if err != nil { + return err + } + + storeList, _, _ := s.allocator.storePool.getStoreList(rangeDesc.RangeID, storeFilterNone) + storeMap := storeListToMap(storeList) + + // Deep-copy the Replicas slice (in our shallow copy of the RangeDescriptor) + // since we'll mutate it in the loop below. + desc := rangeDesc + desc.Replicas = append([]roachpb.ReplicaDescriptor(nil), desc.Replicas...) + rangeInfo := RangeInfo{Desc: &rangeDesc} + + // Step 2: Repeatedly add a replica then remove a replica until we reach the + // desired state. every := log.Every(time.Minute) re := retry.StartWithCtx(ctx, retry.Options{MaxBackoff: 5 * time.Second}) - for len(addTargets) > 0 { + for len(addTargets) > 0 || len(removeTargets) > 0 { if err := ctx.Err(); err != nil { return err } - target := addTargets[0] - if err := db.AdminChangeReplicas( - ctx, rangeDesc.StartKey.AsRawKey(), roachpb.ADD_REPLICA, []roachpb.ReplicationTarget{target}, - ); err != nil { - returnErr := errors.Wrapf(err, "while adding target %v", target) - if !canRetry(err) { - return returnErr + if len(addTargets) > 0 && len(addTargets) >= len(removeTargets) { + // Each iteration, pick the most desirable replica to add. However, + // prefer the first target if it doesn't yet have a replica so that we + // can always transfer the lease to it before removing a replica below. + // This makes it easier to avoid removing a replica that's still + // leaseholder without needing to bounce the lease around a bunch. + candidates := addTargets + if storeHasReplica(targets[0].StoreID, candidates) { + candidates = []roachpb.ReplicaDescriptor{ + {NodeID: targets[0].NodeID, StoreID: targets[0].StoreID}, + } } - if every.ShouldLog() { - log.Warning(ctx, returnErr) + + // The storeList's list of stores is used to constrain which stores the + // allocator considers putting a new replica on. We want it to only + // consider the stores in candidates. + candidateDescs := make([]roachpb.StoreDescriptor, 0, len(candidates)) + for _, candidate := range candidates { + store, ok := storeMap[candidate.StoreID] + if !ok { + return fmt.Errorf("cannot up-repliate to s%d; missing gossiped StoreDescriptor", candidate.StoreID) + } + candidateDescs = append(candidateDescs, *store) + } + storeList = makeStoreList(candidateDescs) + + targetStore, _ := s.allocator.allocateTargetFromList( + ctx, + storeList, + zone, + rangeInfo.Desc.Replicas, + rangeInfo, + s.allocator.scorerOptions()) + if targetStore == nil { + return fmt.Errorf("none of the remaining targets %v are legal additions to %v", + addTargets, rangeInfo.Desc.Replicas) } - re.Next() - continue - } - addTargets = addTargets[1:] - } - // Step 2: Transfer the lease to the first target. This needs to happen before - // we remove replicas or we may try to remove the lease holder. + target := roachpb.ReplicationTarget{ + NodeID: targetStore.Node.NodeID, + StoreID: targetStore.StoreID, + } + if err := s.DB().AdminChangeReplicas( + ctx, rangeDesc.StartKey.AsRawKey(), roachpb.ADD_REPLICA, []roachpb.ReplicationTarget{target}, + ); err != nil { + returnErr := errors.Wrapf(err, "while adding target %v", target) + if !canRetry(err) { + return returnErr + } + if every.ShouldLog() { + log.Warning(ctx, returnErr) + } + re.Next() + continue + } - transferLease := func() { - if err := db.AdminTransferLease( - ctx, rangeDesc.StartKey.AsRawKey(), targets[0].StoreID, - ); err != nil { - log.Warningf(ctx, "while transferring lease: %s", err) + // Upon success, remove the target from our to-do list and add it to our + // local copy of the range descriptor such that future allocator + // decisions take it into account. + addTargets = removeTargetFromSlice(addTargets, target) + rangeInfo.Desc.Replicas = append(rangeInfo.Desc.Replicas, roachpb.ReplicaDescriptor{ + NodeID: target.NodeID, + StoreID: target.StoreID, + }) } - } - - transferLease() - // Step 3: Remove any replicas that are not targets. - - var removeTargets []roachpb.ReplicationTarget - for _, replicaDesc := range rangeDesc.Replicas { - found := false - for _, t := range targets { - if replicaDesc.StoreID == t.StoreID && replicaDesc.NodeID == t.NodeID { - found = true - break + if len(removeTargets) > 0 && len(removeTargets) > len(addTargets) { + targetStore, _, err := s.allocator.RemoveTarget(ctx, zone, removeTargets, rangeInfo) + if err != nil { + return errors.Wrapf(err, "unable to select removal target from %v; current replicas %v", + removeTargets, rangeInfo.Desc.Replicas) } - } - if !found { - removeTargets = append(removeTargets, roachpb.ReplicationTarget{ - StoreID: replicaDesc.StoreID, - NodeID: replicaDesc.NodeID, - }) + target := roachpb.ReplicationTarget{ + NodeID: targetStore.NodeID, + StoreID: targetStore.StoreID, + } + // Note that attempting to remove the leaseholder won't work, so transfer + // the lease first in such scenarios. The first specified target should be + // the leaseholder now, so we can always transfer the lease there. + transferLease() + if err := s.DB().AdminChangeReplicas( + ctx, rangeDesc.StartKey.AsRawKey(), roachpb.REMOVE_REPLICA, []roachpb.ReplicationTarget{target}, + ); err != nil { + log.Warningf(ctx, "while removing target %v: %s", target, err) + if !canRetry(err) { + return err + } + re.Next() + continue + } + + // Upon success, remove the target from our to-do list and from our local + // copy of the range descriptor such that future allocator decisions take + // its absence into account. + removeTargets = removeTargetFromSlice(removeTargets, target) + rangeInfo.Desc.Replicas = removeTargetFromSlice(rangeInfo.Desc.Replicas, target) } } - re.Reset() - for len(removeTargets) > 0 { - if err := ctx.Err(); err != nil { - return err - } + // Step 3: Transfer the lease to the first listed target, as the API specifies. + transferLease() - target := removeTargets[0] - transferLease() - if err := db.AdminChangeReplicas( - ctx, rangeDesc.StartKey.AsRawKey(), roachpb.REMOVE_REPLICA, []roachpb.ReplicationTarget{target}, - ); err != nil { - log.Warningf(ctx, "while removing target %v: %s", target, err) - if !canRetry(err) { - return err - } - re.Next() - continue + return ctx.Err() +} + +// Modifies the underlying storage of the slice rather than copying. +// Don't use on a shared slice where the order matters. +func removeTargetFromSlice( + targets []roachpb.ReplicaDescriptor, target roachpb.ReplicationTarget, +) []roachpb.ReplicaDescriptor { + for i, t := range targets { + if t.NodeID == target.NodeID && t.StoreID == target.StoreID { + // Swap the removed target with the last element in the slice and return + // a slice that's 1 element shorter than before. + targets[i], targets[len(targets)-1] = targets[len(targets)-1], targets[i] + return targets[:len(targets)-1] } - removeTargets = removeTargets[1:] } - return ctx.Err() + return targets } // adminScatter moves replicas and leaseholders for a selection of ranges. diff --git a/pkg/storage/store_rebalancer.go b/pkg/storage/store_rebalancer.go index b4043f07868c..a0377973be00 100644 --- a/pkg/storage/store_rebalancer.go +++ b/pkg/storage/store_rebalancer.go @@ -305,9 +305,7 @@ func (sr *StoreRebalancer) rebalanceStore( log.VEventf(ctx, 1, "rebalancing r%d (%.2f qps) from %v to %v to better balance load", replWithStats.repl.RangeID, replWithStats.qps, descBeforeRebalance.Replicas, targets) replCtx, cancel := context.WithTimeout(replWithStats.repl.AnnotateCtx(ctx), sr.rq.processTimeout) - // TODO(a-robinson): Either make RelocateRange production-ready or do the - // rebalancing another way. - if err := RelocateRange(replCtx, sr.rq.store.DB(), *descBeforeRebalance, targets); err != nil { + if err := sr.rq.store.AdminRelocateRange(replCtx, *descBeforeRebalance, targets); err != nil { cancel() log.Errorf(replCtx, "unable to relocate range to %v: %v", targets, err) continue @@ -594,8 +592,7 @@ func (sr *StoreRebalancer) chooseReplicaToRebalance( } // Pick the replica with the least QPS to be leaseholder; - // TestingRelocateRange transfers the lease to the first provided - // target. + // RelocateRange transfers the lease to the first provided target. newLeaseIdx := 0 newLeaseQPS := math.MaxFloat64 for i := 0; i < len(targets); i++ {