diff --git a/pkg/kv/dist_sender_server_test.go b/pkg/kv/dist_sender_server_test.go index 5cf50214557c..3b6918f3105b 100644 --- a/pkg/kv/dist_sender_server_test.go +++ b/pkg/kv/dist_sender_server_test.go @@ -1731,7 +1731,7 @@ func TestTxnCoordSenderRetries(t *testing.T) { // No retries, 1pc commit. }, { - name: "require1PC commit with injected unknown serializable error", + name: "require1PC commit with injected possible replay error", retryable: func(ctx context.Context, txn *client.Txn) error { b := txn.NewBatch() b.Put("a", "put") @@ -2086,9 +2086,6 @@ func TestTxnCoordSenderRetries(t *testing.T) { expFailure: "unexpected value", // condition failed error when failing on tombstones }, { - // This test sends a 1PC batch with Put+EndTxn. - // The Put gets a write too old error but, since there's no refresh spans, - // the commit succeeds. name: "write too old with put in batch commit", afterTxnStart: func(ctx context.Context, db *client.DB) error { return db.Put(ctx, "a", "put") @@ -2100,28 +2097,6 @@ func TestTxnCoordSenderRetries(t *testing.T) { }, // No retries, 1pc commit. }, - { - // This test is like the previous one in that the commit batch succeeds at - // an updated timestamp, but this time the EndTxn puts the - // transaction in the STAGING state instead of COMMITTED because there had - // been previous write in a different batch. Like above, the commit is - // successful since there are no refresh spans. - name: "write too old in staging commit", - beforeTxnStart: func(ctx context.Context, db *client.DB) error { - return db.Put(ctx, "a", "orig") - }, - afterTxnStart: func(ctx context.Context, db *client.DB) error { - return db.Put(ctx, "a", "put") - }, - retryable: func(ctx context.Context, txn *client.Txn) error { - if err := txn.Put(ctx, "aother", "another put"); err != nil { - return err - } - b := txn.NewBatch() - b.Put("a", "final value") - return txn.CommitInBatch(ctx, b) - }, - }, { name: "write too old with cput in batch commit", beforeTxnStart: func(ctx context.Context, db *client.DB) error { @@ -2135,28 +2110,7 @@ func TestTxnCoordSenderRetries(t *testing.T) { b.CPut("a", "cput", strToValue("put")) return txn.CommitInBatch(ctx, b) // will be a 1PC, won't get auto retry }, - // No client-side retries, 1PC commit. On the server-side, the batch is - // evaluated twice: once at the original timestamp, where it gets a - // WriteTooOldError, and then once at the pushed timestamp. The - // server-side retry is enabled by the fact that there have not been any - // previous reads and so the transaction can commit at a pushed timestamp. - }, - { - // This test is like the previous one, except the 1PC batch cannot commit - // at the updated timestamp. - name: "write too old with failed cput in batch commit", - beforeTxnStart: func(ctx context.Context, db *client.DB) error { - return db.Put(ctx, "a", "orig") - }, - afterTxnStart: func(ctx context.Context, db *client.DB) error { - return db.Put(ctx, "a", "put") - }, - retryable: func(ctx context.Context, txn *client.Txn) error { - b := txn.NewBatch() - b.CPut("a", "cput", strToValue("orig")) - return txn.CommitInBatch(ctx, b) // will be a 1PC, won't get auto retry - }, - expFailure: "unexpected value", // The CPut cannot succeed. + // No retries, 1pc commit. }, { name: "multi-range batch with forwarded timestamp", @@ -2286,8 +2240,7 @@ func TestTxnCoordSenderRetries(t *testing.T) { b.Put("c", "put") return txn.CommitInBatch(ctx, b) }, - // We expect the request to succeed after a server-side retry. - txnCoordRetry: false, + txnCoordRetry: true, }, { name: "multi-range batch with deferred write too old and failed cput", @@ -2322,8 +2275,8 @@ func TestTxnCoordSenderRetries(t *testing.T) { b.Put("c", "put") return txn.CommitInBatch(ctx, b) }, - // We expect the request to succeed after a server-side retry. - txnCoordRetry: false, + // Expect a transaction coord retry, which should succeed. + txnCoordRetry: true, }, { name: "cput within uncertainty interval", diff --git a/pkg/kv/txn_coord_sender_test.go b/pkg/kv/txn_coord_sender_test.go index aed971f27c60..3e8b235e980f 100644 --- a/pkg/kv/txn_coord_sender_test.go +++ b/pkg/kv/txn_coord_sender_test.go @@ -553,7 +553,6 @@ func TestTxnCoordSenderAddIntentOnError(t *testing.T) { } func assertTransactionRetryError(t *testing.T, e error) { - t.Helper() if retErr, ok := e.(*roachpb.TransactionRetryWithProtoRefreshError); ok { if !testutils.IsError(retErr, "TransactionRetryError") { t.Fatalf("expected the cause to be TransactionRetryError, but got %s", diff --git a/pkg/kv/txn_interceptor_span_refresher.go b/pkg/kv/txn_interceptor_span_refresher.go index 73774ddd5d54..459810b5decb 100644 --- a/pkg/kv/txn_interceptor_span_refresher.go +++ b/pkg/kv/txn_interceptor_span_refresher.go @@ -170,19 +170,6 @@ func (sr *txnSpanRefresher) SendLocked( // Send through wrapped lockedSender. Unlocks while sending then re-locks. br, pErr, largestRefreshTS := sr.sendLockedWithRefreshAttempts(ctx, ba, maxAttempts) if pErr != nil { - // The server sometimes "performs refreshes" - it updates the transaction's - // ReadTimestamp when the client said that there's nothing to refresh. In - // these cases, we need to update our refreshedTimestamp too. This is pretty - // inconsequential: the server only does this on an EndTxn. If the - // respective batch succeeds, then there won't be any more requests and so - // sr.refreshedTimestamp doesn't matter (that's why we don't handle the - // success case). However, the server can "refresh" and then return an - // error. In this case, the client will rollback and, if we don't update - // sr.refreshedTimestamp, then an assertion will fire about the rollback's - // timestamp being inconsistent. - if pErr.GetTxn() != nil { - sr.refreshedTimestamp.Forward(pErr.GetTxn().ReadTimestamp) - } return nil, pErr } diff --git a/pkg/roachpb/api.go b/pkg/roachpb/api.go index 17d6530cc86c..1d36bec36dca 100644 --- a/pkg/roachpb/api.go +++ b/pkg/roachpb/api.go @@ -107,6 +107,13 @@ func IsReadOnly(args Request) bool { return (flags&isRead) != 0 && (flags&isWrite) == 0 } +// IsReadAndWrite returns true if the request both reads and writes +// (such as conditional puts). +func IsReadAndWrite(args Request) bool { + flags := args.flags() + return (flags&isRead) != 0 && (flags&isWrite) != 0 +} + // IsTransactional returns true if the request may be part of a // transaction. func IsTransactional(args Request) bool { diff --git a/pkg/roachpb/api.pb.go b/pkg/roachpb/api.pb.go index e44faea1c133..39b54d1fb202 100644 --- a/pkg/roachpb/api.pb.go +++ b/pkg/roachpb/api.pb.go @@ -71,7 +71,7 @@ func (x ReadConsistencyType) String() string { return proto.EnumName(ReadConsistencyType_name, int32(x)) } func (ReadConsistencyType) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_e73e8f3e4a270190, []int{0} + return fileDescriptor_api_c364818c36b06139, []int{0} } // ScanFormat is an enumeration of the available response formats for MVCCScan @@ -99,7 +99,7 @@ func (x ScanFormat) String() string { return proto.EnumName(ScanFormat_name, int32(x)) } func (ScanFormat) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_e73e8f3e4a270190, []int{1} + return fileDescriptor_api_c364818c36b06139, []int{1} } type ChecksumMode int32 @@ -146,7 +146,7 @@ func (x ChecksumMode) String() string { return proto.EnumName(ChecksumMode_name, int32(x)) } func (ChecksumMode) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_e73e8f3e4a270190, []int{2} + return fileDescriptor_api_c364818c36b06139, []int{2} } // PushTxnType determines what action to take when pushing a transaction. @@ -177,7 +177,7 @@ func (x PushTxnType) String() string { return proto.EnumName(PushTxnType_name, int32(x)) } func (PushTxnType) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_e73e8f3e4a270190, []int{3} + return fileDescriptor_api_c364818c36b06139, []int{3} } type ExternalStorageProvider int32 @@ -215,7 +215,7 @@ func (x ExternalStorageProvider) String() string { return proto.EnumName(ExternalStorageProvider_name, int32(x)) } func (ExternalStorageProvider) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_e73e8f3e4a270190, []int{4} + return fileDescriptor_api_c364818c36b06139, []int{4} } type MVCCFilter int32 @@ -238,7 +238,7 @@ func (x MVCCFilter) String() string { return proto.EnumName(MVCCFilter_name, int32(x)) } func (MVCCFilter) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_e73e8f3e4a270190, []int{5} + return fileDescriptor_api_c364818c36b06139, []int{5} } type ResponseHeader_ResumeReason int32 @@ -270,7 +270,7 @@ func (x ResponseHeader_ResumeReason) String() string { return proto.EnumName(ResponseHeader_ResumeReason_name, int32(x)) } func (ResponseHeader_ResumeReason) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_e73e8f3e4a270190, []int{2, 0} + return fileDescriptor_api_c364818c36b06139, []int{2, 0} } type CheckConsistencyResponse_Status int32 @@ -312,7 +312,7 @@ func (x CheckConsistencyResponse_Status) String() string { return proto.EnumName(CheckConsistencyResponse_Status_name, int32(x)) } func (CheckConsistencyResponse_Status) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_e73e8f3e4a270190, []int{26, 0} + return fileDescriptor_api_c364818c36b06139, []int{26, 0} } // RangeInfo describes a range which executed a request. It contains @@ -326,7 +326,7 @@ func (m *RangeInfo) Reset() { *m = RangeInfo{} } func (m *RangeInfo) String() string { return proto.CompactTextString(m) } func (*RangeInfo) ProtoMessage() {} func (*RangeInfo) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e73e8f3e4a270190, []int{0} + return fileDescriptor_api_c364818c36b06139, []int{0} } func (m *RangeInfo) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -369,7 +369,7 @@ func (m *RequestHeader) Reset() { *m = RequestHeader{} } func (m *RequestHeader) String() string { return proto.CompactTextString(m) } func (*RequestHeader) ProtoMessage() {} func (*RequestHeader) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e73e8f3e4a270190, []int{1} + return fileDescriptor_api_c364818c36b06139, []int{1} } func (m *RequestHeader) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -431,7 +431,7 @@ func (m *ResponseHeader) Reset() { *m = ResponseHeader{} } func (m *ResponseHeader) String() string { return proto.CompactTextString(m) } func (*ResponseHeader) ProtoMessage() {} func (*ResponseHeader) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e73e8f3e4a270190, []int{2} + return fileDescriptor_api_c364818c36b06139, []int{2} } func (m *ResponseHeader) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -465,7 +465,7 @@ func (m *GetRequest) Reset() { *m = GetRequest{} } func (m *GetRequest) String() string { return proto.CompactTextString(m) } func (*GetRequest) ProtoMessage() {} func (*GetRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e73e8f3e4a270190, []int{3} + return fileDescriptor_api_c364818c36b06139, []int{3} } func (m *GetRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -503,7 +503,7 @@ func (m *GetResponse) Reset() { *m = GetResponse{} } func (m *GetResponse) String() string { return proto.CompactTextString(m) } func (*GetResponse) ProtoMessage() {} func (*GetResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e73e8f3e4a270190, []int{4} + return fileDescriptor_api_c364818c36b06139, []int{4} } func (m *GetResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -546,7 +546,7 @@ func (m *PutRequest) Reset() { *m = PutRequest{} } func (m *PutRequest) String() string { return proto.CompactTextString(m) } func (*PutRequest) ProtoMessage() {} func (*PutRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e73e8f3e4a270190, []int{5} + return fileDescriptor_api_c364818c36b06139, []int{5} } func (m *PutRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -580,7 +580,7 @@ func (m *PutResponse) Reset() { *m = PutResponse{} } func (m *PutResponse) String() string { return proto.CompactTextString(m) } func (*PutResponse) ProtoMessage() {} func (*PutResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e73e8f3e4a270190, []int{6} + return fileDescriptor_api_c364818c36b06139, []int{6} } func (m *PutResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -634,7 +634,7 @@ func (m *ConditionalPutRequest) Reset() { *m = ConditionalPutRequest{} } func (m *ConditionalPutRequest) String() string { return proto.CompactTextString(m) } func (*ConditionalPutRequest) ProtoMessage() {} func (*ConditionalPutRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e73e8f3e4a270190, []int{7} + return fileDescriptor_api_c364818c36b06139, []int{7} } func (m *ConditionalPutRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -669,7 +669,7 @@ func (m *ConditionalPutResponse) Reset() { *m = ConditionalPutResponse{} func (m *ConditionalPutResponse) String() string { return proto.CompactTextString(m) } func (*ConditionalPutResponse) ProtoMessage() {} func (*ConditionalPutResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e73e8f3e4a270190, []int{8} + return fileDescriptor_api_c364818c36b06139, []int{8} } func (m *ConditionalPutResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -715,7 +715,7 @@ func (m *InitPutRequest) Reset() { *m = InitPutRequest{} } func (m *InitPutRequest) String() string { return proto.CompactTextString(m) } func (*InitPutRequest) ProtoMessage() {} func (*InitPutRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e73e8f3e4a270190, []int{9} + return fileDescriptor_api_c364818c36b06139, []int{9} } func (m *InitPutRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -749,7 +749,7 @@ func (m *InitPutResponse) Reset() { *m = InitPutResponse{} } func (m *InitPutResponse) String() string { return proto.CompactTextString(m) } func (*InitPutResponse) ProtoMessage() {} func (*InitPutResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e73e8f3e4a270190, []int{10} + return fileDescriptor_api_c364818c36b06139, []int{10} } func (m *InitPutResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -789,7 +789,7 @@ func (m *IncrementRequest) Reset() { *m = IncrementRequest{} } func (m *IncrementRequest) String() string { return proto.CompactTextString(m) } func (*IncrementRequest) ProtoMessage() {} func (*IncrementRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e73e8f3e4a270190, []int{11} + return fileDescriptor_api_c364818c36b06139, []int{11} } func (m *IncrementRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -826,7 +826,7 @@ func (m *IncrementResponse) Reset() { *m = IncrementResponse{} } func (m *IncrementResponse) String() string { return proto.CompactTextString(m) } func (*IncrementResponse) ProtoMessage() {} func (*IncrementResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e73e8f3e4a270190, []int{12} + return fileDescriptor_api_c364818c36b06139, []int{12} } func (m *IncrementResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -860,7 +860,7 @@ func (m *DeleteRequest) Reset() { *m = DeleteRequest{} } func (m *DeleteRequest) String() string { return proto.CompactTextString(m) } func (*DeleteRequest) ProtoMessage() {} func (*DeleteRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e73e8f3e4a270190, []int{13} + return fileDescriptor_api_c364818c36b06139, []int{13} } func (m *DeleteRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -894,7 +894,7 @@ func (m *DeleteResponse) Reset() { *m = DeleteResponse{} } func (m *DeleteResponse) String() string { return proto.CompactTextString(m) } func (*DeleteResponse) ProtoMessage() {} func (*DeleteResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e73e8f3e4a270190, []int{14} + return fileDescriptor_api_c364818c36b06139, []int{14} } func (m *DeleteResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -946,7 +946,7 @@ func (m *DeleteRangeRequest) Reset() { *m = DeleteRangeRequest{} } func (m *DeleteRangeRequest) String() string { return proto.CompactTextString(m) } func (*DeleteRangeRequest) ProtoMessage() {} func (*DeleteRangeRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e73e8f3e4a270190, []int{15} + return fileDescriptor_api_c364818c36b06139, []int{15} } func (m *DeleteRangeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -983,7 +983,7 @@ func (m *DeleteRangeResponse) Reset() { *m = DeleteRangeResponse{} } func (m *DeleteRangeResponse) String() string { return proto.CompactTextString(m) } func (*DeleteRangeResponse) ProtoMessage() {} func (*DeleteRangeResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e73e8f3e4a270190, []int{16} + return fileDescriptor_api_c364818c36b06139, []int{16} } func (m *DeleteRangeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1030,7 +1030,7 @@ func (m *ClearRangeRequest) Reset() { *m = ClearRangeRequest{} } func (m *ClearRangeRequest) String() string { return proto.CompactTextString(m) } func (*ClearRangeRequest) ProtoMessage() {} func (*ClearRangeRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e73e8f3e4a270190, []int{17} + return fileDescriptor_api_c364818c36b06139, []int{17} } func (m *ClearRangeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1064,7 +1064,7 @@ func (m *ClearRangeResponse) Reset() { *m = ClearRangeResponse{} } func (m *ClearRangeResponse) String() string { return proto.CompactTextString(m) } func (*ClearRangeResponse) ProtoMessage() {} func (*ClearRangeResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e73e8f3e4a270190, []int{18} + return fileDescriptor_api_c364818c36b06139, []int{18} } func (m *ClearRangeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1105,7 +1105,7 @@ func (m *RevertRangeRequest) Reset() { *m = RevertRangeRequest{} } func (m *RevertRangeRequest) String() string { return proto.CompactTextString(m) } func (*RevertRangeRequest) ProtoMessage() {} func (*RevertRangeRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e73e8f3e4a270190, []int{19} + return fileDescriptor_api_c364818c36b06139, []int{19} } func (m *RevertRangeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1139,7 +1139,7 @@ func (m *RevertRangeResponse) Reset() { *m = RevertRangeResponse{} } func (m *RevertRangeResponse) String() string { return proto.CompactTextString(m) } func (*RevertRangeResponse) ProtoMessage() {} func (*RevertRangeResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e73e8f3e4a270190, []int{20} + return fileDescriptor_api_c364818c36b06139, []int{20} } func (m *RevertRangeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1179,7 +1179,7 @@ func (m *ScanRequest) Reset() { *m = ScanRequest{} } func (m *ScanRequest) String() string { return proto.CompactTextString(m) } func (*ScanRequest) ProtoMessage() {} func (*ScanRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e73e8f3e4a270190, []int{21} + return fileDescriptor_api_c364818c36b06139, []int{21} } func (m *ScanRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1226,7 +1226,7 @@ func (m *ScanResponse) Reset() { *m = ScanResponse{} } func (m *ScanResponse) String() string { return proto.CompactTextString(m) } func (*ScanResponse) ProtoMessage() {} func (*ScanResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e73e8f3e4a270190, []int{22} + return fileDescriptor_api_c364818c36b06139, []int{22} } func (m *ScanResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1266,7 +1266,7 @@ func (m *ReverseScanRequest) Reset() { *m = ReverseScanRequest{} } func (m *ReverseScanRequest) String() string { return proto.CompactTextString(m) } func (*ReverseScanRequest) ProtoMessage() {} func (*ReverseScanRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e73e8f3e4a270190, []int{23} + return fileDescriptor_api_c364818c36b06139, []int{23} } func (m *ReverseScanRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1313,7 +1313,7 @@ func (m *ReverseScanResponse) Reset() { *m = ReverseScanResponse{} } func (m *ReverseScanResponse) String() string { return proto.CompactTextString(m) } func (*ReverseScanResponse) ProtoMessage() {} func (*ReverseScanResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e73e8f3e4a270190, []int{24} + return fileDescriptor_api_c364818c36b06139, []int{24} } func (m *ReverseScanResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1366,7 +1366,7 @@ func (m *CheckConsistencyRequest) Reset() { *m = CheckConsistencyRequest func (m *CheckConsistencyRequest) String() string { return proto.CompactTextString(m) } func (*CheckConsistencyRequest) ProtoMessage() {} func (*CheckConsistencyRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e73e8f3e4a270190, []int{25} + return fileDescriptor_api_c364818c36b06139, []int{25} } func (m *CheckConsistencyRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1403,7 +1403,7 @@ func (m *CheckConsistencyResponse) Reset() { *m = CheckConsistencyRespon func (m *CheckConsistencyResponse) String() string { return proto.CompactTextString(m) } func (*CheckConsistencyResponse) ProtoMessage() {} func (*CheckConsistencyResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e73e8f3e4a270190, []int{26} + return fileDescriptor_api_c364818c36b06139, []int{26} } func (m *CheckConsistencyResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1447,7 +1447,7 @@ func (m *CheckConsistencyResponse_Result) Reset() { *m = CheckConsistenc func (m *CheckConsistencyResponse_Result) String() string { return proto.CompactTextString(m) } func (*CheckConsistencyResponse_Result) ProtoMessage() {} func (*CheckConsistencyResponse_Result) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e73e8f3e4a270190, []int{26, 0} + return fileDescriptor_api_c364818c36b06139, []int{26, 0} } func (m *CheckConsistencyResponse_Result) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1495,7 +1495,7 @@ func (m *RecomputeStatsRequest) Reset() { *m = RecomputeStatsRequest{} } func (m *RecomputeStatsRequest) String() string { return proto.CompactTextString(m) } func (*RecomputeStatsRequest) ProtoMessage() {} func (*RecomputeStatsRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e73e8f3e4a270190, []int{27} + return fileDescriptor_api_c364818c36b06139, []int{27} } func (m *RecomputeStatsRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1531,7 +1531,7 @@ func (m *RecomputeStatsResponse) Reset() { *m = RecomputeStatsResponse{} func (m *RecomputeStatsResponse) String() string { return proto.CompactTextString(m) } func (*RecomputeStatsResponse) ProtoMessage() {} func (*RecomputeStatsResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e73e8f3e4a270190, []int{28} + return fileDescriptor_api_c364818c36b06139, []int{28} } func (m *RecomputeStatsResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1622,7 +1622,7 @@ func (m *EndTxnRequest) Reset() { *m = EndTxnRequest{} } func (m *EndTxnRequest) String() string { return proto.CompactTextString(m) } func (*EndTxnRequest) ProtoMessage() {} func (*EndTxnRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e73e8f3e4a270190, []int{29} + return fileDescriptor_api_c364818c36b06139, []int{29} } func (m *EndTxnRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1668,7 +1668,7 @@ func (m *EndTxnResponse) Reset() { *m = EndTxnResponse{} } func (m *EndTxnResponse) String() string { return proto.CompactTextString(m) } func (*EndTxnResponse) ProtoMessage() {} func (*EndTxnResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e73e8f3e4a270190, []int{30} + return fileDescriptor_api_c364818c36b06139, []int{30} } func (m *EndTxnResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1729,7 +1729,7 @@ func (m *AdminSplitRequest) Reset() { *m = AdminSplitRequest{} } func (m *AdminSplitRequest) String() string { return proto.CompactTextString(m) } func (*AdminSplitRequest) ProtoMessage() {} func (*AdminSplitRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e73e8f3e4a270190, []int{31} + return fileDescriptor_api_c364818c36b06139, []int{31} } func (m *AdminSplitRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1764,7 +1764,7 @@ func (m *AdminSplitResponse) Reset() { *m = AdminSplitResponse{} } func (m *AdminSplitResponse) String() string { return proto.CompactTextString(m) } func (*AdminSplitResponse) ProtoMessage() {} func (*AdminSplitResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e73e8f3e4a270190, []int{32} + return fileDescriptor_api_c364818c36b06139, []int{32} } func (m *AdminSplitResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1803,7 +1803,7 @@ func (m *AdminUnsplitRequest) Reset() { *m = AdminUnsplitRequest{} } func (m *AdminUnsplitRequest) String() string { return proto.CompactTextString(m) } func (*AdminUnsplitRequest) ProtoMessage() {} func (*AdminUnsplitRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e73e8f3e4a270190, []int{33} + return fileDescriptor_api_c364818c36b06139, []int{33} } func (m *AdminUnsplitRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1838,7 +1838,7 @@ func (m *AdminUnsplitResponse) Reset() { *m = AdminUnsplitResponse{} } func (m *AdminUnsplitResponse) String() string { return proto.CompactTextString(m) } func (*AdminUnsplitResponse) ProtoMessage() {} func (*AdminUnsplitResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e73e8f3e4a270190, []int{34} + return fileDescriptor_api_c364818c36b06139, []int{34} } func (m *AdminUnsplitResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1881,7 +1881,7 @@ func (m *AdminMergeRequest) Reset() { *m = AdminMergeRequest{} } func (m *AdminMergeRequest) String() string { return proto.CompactTextString(m) } func (*AdminMergeRequest) ProtoMessage() {} func (*AdminMergeRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e73e8f3e4a270190, []int{35} + return fileDescriptor_api_c364818c36b06139, []int{35} } func (m *AdminMergeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1916,7 +1916,7 @@ func (m *AdminMergeResponse) Reset() { *m = AdminMergeResponse{} } func (m *AdminMergeResponse) String() string { return proto.CompactTextString(m) } func (*AdminMergeResponse) ProtoMessage() {} func (*AdminMergeResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e73e8f3e4a270190, []int{36} + return fileDescriptor_api_c364818c36b06139, []int{36} } func (m *AdminMergeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1954,7 +1954,7 @@ func (m *AdminTransferLeaseRequest) Reset() { *m = AdminTransferLeaseReq func (m *AdminTransferLeaseRequest) String() string { return proto.CompactTextString(m) } func (*AdminTransferLeaseRequest) ProtoMessage() {} func (*AdminTransferLeaseRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e73e8f3e4a270190, []int{37} + return fileDescriptor_api_c364818c36b06139, []int{37} } func (m *AdminTransferLeaseRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1987,7 +1987,7 @@ func (m *AdminTransferLeaseResponse) Reset() { *m = AdminTransferLeaseRe func (m *AdminTransferLeaseResponse) String() string { return proto.CompactTextString(m) } func (*AdminTransferLeaseResponse) ProtoMessage() {} func (*AdminTransferLeaseResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e73e8f3e4a270190, []int{38} + return fileDescriptor_api_c364818c36b06139, []int{38} } func (m *AdminTransferLeaseResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2022,7 +2022,7 @@ func (m *ReplicationChange) Reset() { *m = ReplicationChange{} } func (m *ReplicationChange) String() string { return proto.CompactTextString(m) } func (*ReplicationChange) ProtoMessage() {} func (*ReplicationChange) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e73e8f3e4a270190, []int{39} + return fileDescriptor_api_c364818c36b06139, []int{39} } func (m *ReplicationChange) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2080,7 +2080,7 @@ func (m *AdminChangeReplicasRequest) Reset() { *m = AdminChangeReplicasR func (m *AdminChangeReplicasRequest) String() string { return proto.CompactTextString(m) } func (*AdminChangeReplicasRequest) ProtoMessage() {} func (*AdminChangeReplicasRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e73e8f3e4a270190, []int{40} + return fileDescriptor_api_c364818c36b06139, []int{40} } func (m *AdminChangeReplicasRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2115,7 +2115,7 @@ func (m *AdminChangeReplicasResponse) Reset() { *m = AdminChangeReplicas func (m *AdminChangeReplicasResponse) String() string { return proto.CompactTextString(m) } func (*AdminChangeReplicasResponse) ProtoMessage() {} func (*AdminChangeReplicasResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e73e8f3e4a270190, []int{41} + return fileDescriptor_api_c364818c36b06139, []int{41} } func (m *AdminChangeReplicasResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2152,7 +2152,7 @@ func (m *AdminRelocateRangeRequest) Reset() { *m = AdminRelocateRangeReq func (m *AdminRelocateRangeRequest) String() string { return proto.CompactTextString(m) } func (*AdminRelocateRangeRequest) ProtoMessage() {} func (*AdminRelocateRangeRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e73e8f3e4a270190, []int{42} + return fileDescriptor_api_c364818c36b06139, []int{42} } func (m *AdminRelocateRangeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2185,7 +2185,7 @@ func (m *AdminRelocateRangeResponse) Reset() { *m = AdminRelocateRangeRe func (m *AdminRelocateRangeResponse) String() string { return proto.CompactTextString(m) } func (*AdminRelocateRangeResponse) ProtoMessage() {} func (*AdminRelocateRangeResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e73e8f3e4a270190, []int{43} + return fileDescriptor_api_c364818c36b06139, []int{43} } func (m *AdminRelocateRangeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2224,7 +2224,7 @@ func (m *HeartbeatTxnRequest) Reset() { *m = HeartbeatTxnRequest{} } func (m *HeartbeatTxnRequest) String() string { return proto.CompactTextString(m) } func (*HeartbeatTxnRequest) ProtoMessage() {} func (*HeartbeatTxnRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e73e8f3e4a270190, []int{44} + return fileDescriptor_api_c364818c36b06139, []int{44} } func (m *HeartbeatTxnRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2261,7 +2261,7 @@ func (m *HeartbeatTxnResponse) Reset() { *m = HeartbeatTxnResponse{} } func (m *HeartbeatTxnResponse) String() string { return proto.CompactTextString(m) } func (*HeartbeatTxnResponse) ProtoMessage() {} func (*HeartbeatTxnResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e73e8f3e4a270190, []int{45} + return fileDescriptor_api_c364818c36b06139, []int{45} } func (m *HeartbeatTxnResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2299,7 +2299,7 @@ func (m *GCRequest) Reset() { *m = GCRequest{} } func (m *GCRequest) String() string { return proto.CompactTextString(m) } func (*GCRequest) ProtoMessage() {} func (*GCRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e73e8f3e4a270190, []int{46} + return fileDescriptor_api_c364818c36b06139, []int{46} } func (m *GCRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2333,7 +2333,7 @@ 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 fileDescriptor_api_e73e8f3e4a270190, []int{46, 0} + return fileDescriptor_api_c364818c36b06139, []int{46, 0} } func (m *GCRequest_GCKey) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2367,7 +2367,7 @@ func (m *GCResponse) Reset() { *m = GCResponse{} } func (m *GCResponse) String() string { return proto.CompactTextString(m) } func (*GCResponse) ProtoMessage() {} func (*GCResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e73e8f3e4a270190, []int{47} + return fileDescriptor_api_c364818c36b06139, []int{47} } func (m *GCResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2436,7 +2436,7 @@ func (m *PushTxnRequest) Reset() { *m = PushTxnRequest{} } func (m *PushTxnRequest) String() string { return proto.CompactTextString(m) } func (*PushTxnRequest) ProtoMessage() {} func (*PushTxnRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e73e8f3e4a270190, []int{48} + return fileDescriptor_api_c364818c36b06139, []int{48} } func (m *PushTxnRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2479,7 +2479,7 @@ func (m *PushTxnResponse) Reset() { *m = PushTxnResponse{} } func (m *PushTxnResponse) String() string { return proto.CompactTextString(m) } func (*PushTxnResponse) ProtoMessage() {} func (*PushTxnResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e73e8f3e4a270190, []int{49} + return fileDescriptor_api_c364818c36b06139, []int{49} } func (m *PushTxnResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2526,7 +2526,7 @@ func (m *RecoverTxnRequest) Reset() { *m = RecoverTxnRequest{} } func (m *RecoverTxnRequest) String() string { return proto.CompactTextString(m) } func (*RecoverTxnRequest) ProtoMessage() {} func (*RecoverTxnRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e73e8f3e4a270190, []int{50} + return fileDescriptor_api_c364818c36b06139, []int{50} } func (m *RecoverTxnRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2562,7 +2562,7 @@ func (m *RecoverTxnResponse) Reset() { *m = RecoverTxnResponse{} } func (m *RecoverTxnResponse) String() string { return proto.CompactTextString(m) } func (*RecoverTxnResponse) ProtoMessage() {} func (*RecoverTxnResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e73e8f3e4a270190, []int{51} + return fileDescriptor_api_c364818c36b06139, []int{51} } func (m *RecoverTxnResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2606,7 +2606,7 @@ func (m *QueryTxnRequest) Reset() { *m = QueryTxnRequest{} } func (m *QueryTxnRequest) String() string { return proto.CompactTextString(m) } func (*QueryTxnRequest) ProtoMessage() {} func (*QueryTxnRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e73e8f3e4a270190, []int{52} + return fileDescriptor_api_c364818c36b06139, []int{52} } func (m *QueryTxnRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2645,7 +2645,7 @@ func (m *QueryTxnResponse) Reset() { *m = QueryTxnResponse{} } func (m *QueryTxnResponse) String() string { return proto.CompactTextString(m) } func (*QueryTxnResponse) ProtoMessage() {} func (*QueryTxnResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e73e8f3e4a270190, []int{53} + return fileDescriptor_api_c364818c36b06139, []int{53} } func (m *QueryTxnResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2705,7 +2705,7 @@ func (m *QueryIntentRequest) Reset() { *m = QueryIntentRequest{} } func (m *QueryIntentRequest) String() string { return proto.CompactTextString(m) } func (*QueryIntentRequest) ProtoMessage() {} func (*QueryIntentRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e73e8f3e4a270190, []int{54} + return fileDescriptor_api_c364818c36b06139, []int{54} } func (m *QueryIntentRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2741,7 +2741,7 @@ func (m *QueryIntentResponse) Reset() { *m = QueryIntentResponse{} } func (m *QueryIntentResponse) String() string { return proto.CompactTextString(m) } func (*QueryIntentResponse) ProtoMessage() {} func (*QueryIntentResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e73e8f3e4a270190, []int{55} + return fileDescriptor_api_c364818c36b06139, []int{55} } func (m *QueryIntentResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2787,7 +2787,7 @@ func (m *ResolveIntentRequest) Reset() { *m = ResolveIntentRequest{} } func (m *ResolveIntentRequest) String() string { return proto.CompactTextString(m) } func (*ResolveIntentRequest) ProtoMessage() {} func (*ResolveIntentRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e73e8f3e4a270190, []int{56} + return fileDescriptor_api_c364818c36b06139, []int{56} } func (m *ResolveIntentRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2822,7 +2822,7 @@ func (m *ResolveIntentResponse) Reset() { *m = ResolveIntentResponse{} } func (m *ResolveIntentResponse) String() string { return proto.CompactTextString(m) } func (*ResolveIntentResponse) ProtoMessage() {} func (*ResolveIntentResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e73e8f3e4a270190, []int{57} + return fileDescriptor_api_c364818c36b06139, []int{57} } func (m *ResolveIntentResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2872,7 +2872,7 @@ func (m *ResolveIntentRangeRequest) Reset() { *m = ResolveIntentRangeReq func (m *ResolveIntentRangeRequest) String() string { return proto.CompactTextString(m) } func (*ResolveIntentRangeRequest) ProtoMessage() {} func (*ResolveIntentRangeRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e73e8f3e4a270190, []int{58} + return fileDescriptor_api_c364818c36b06139, []int{58} } func (m *ResolveIntentRangeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2907,7 +2907,7 @@ func (m *ResolveIntentRangeResponse) Reset() { *m = ResolveIntentRangeRe func (m *ResolveIntentRangeResponse) String() string { return proto.CompactTextString(m) } func (*ResolveIntentRangeResponse) ProtoMessage() {} func (*ResolveIntentRangeResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e73e8f3e4a270190, []int{59} + return fileDescriptor_api_c364818c36b06139, []int{59} } func (m *ResolveIntentRangeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2944,7 +2944,7 @@ func (m *MergeRequest) Reset() { *m = MergeRequest{} } func (m *MergeRequest) String() string { return proto.CompactTextString(m) } func (*MergeRequest) ProtoMessage() {} func (*MergeRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e73e8f3e4a270190, []int{60} + return fileDescriptor_api_c364818c36b06139, []int{60} } func (m *MergeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2978,7 +2978,7 @@ func (m *MergeResponse) Reset() { *m = MergeResponse{} } func (m *MergeResponse) String() string { return proto.CompactTextString(m) } func (*MergeResponse) ProtoMessage() {} func (*MergeResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e73e8f3e4a270190, []int{61} + return fileDescriptor_api_c364818c36b06139, []int{61} } func (m *MergeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3023,7 +3023,7 @@ func (m *TruncateLogRequest) Reset() { *m = TruncateLogRequest{} } func (m *TruncateLogRequest) String() string { return proto.CompactTextString(m) } func (*TruncateLogRequest) ProtoMessage() {} func (*TruncateLogRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e73e8f3e4a270190, []int{62} + return fileDescriptor_api_c364818c36b06139, []int{62} } func (m *TruncateLogRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3057,7 +3057,7 @@ func (m *TruncateLogResponse) Reset() { *m = TruncateLogResponse{} } func (m *TruncateLogResponse) String() string { return proto.CompactTextString(m) } func (*TruncateLogResponse) ProtoMessage() {} func (*TruncateLogResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e73e8f3e4a270190, []int{63} + return fileDescriptor_api_c364818c36b06139, []int{63} } func (m *TruncateLogResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3101,7 +3101,7 @@ func (m *RequestLeaseRequest) Reset() { *m = RequestLeaseRequest{} } func (m *RequestLeaseRequest) String() string { return proto.CompactTextString(m) } func (*RequestLeaseRequest) ProtoMessage() {} func (*RequestLeaseRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e73e8f3e4a270190, []int{64} + return fileDescriptor_api_c364818c36b06139, []int{64} } func (m *RequestLeaseRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3150,7 +3150,7 @@ func (m *TransferLeaseRequest) Reset() { *m = TransferLeaseRequest{} } func (m *TransferLeaseRequest) String() string { return proto.CompactTextString(m) } func (*TransferLeaseRequest) ProtoMessage() {} func (*TransferLeaseRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e73e8f3e4a270190, []int{65} + return fileDescriptor_api_c364818c36b06139, []int{65} } func (m *TransferLeaseRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3187,7 +3187,7 @@ func (m *LeaseInfoRequest) Reset() { *m = LeaseInfoRequest{} } func (m *LeaseInfoRequest) String() string { return proto.CompactTextString(m) } func (*LeaseInfoRequest) ProtoMessage() {} func (*LeaseInfoRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e73e8f3e4a270190, []int{66} + return fileDescriptor_api_c364818c36b06139, []int{66} } func (m *LeaseInfoRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3224,7 +3224,7 @@ func (m *LeaseInfoResponse) Reset() { *m = LeaseInfoResponse{} } func (m *LeaseInfoResponse) String() string { return proto.CompactTextString(m) } func (*LeaseInfoResponse) ProtoMessage() {} func (*LeaseInfoResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e73e8f3e4a270190, []int{67} + return fileDescriptor_api_c364818c36b06139, []int{67} } func (m *LeaseInfoResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3259,7 +3259,7 @@ func (m *RequestLeaseResponse) Reset() { *m = RequestLeaseResponse{} } func (m *RequestLeaseResponse) String() string { return proto.CompactTextString(m) } func (*RequestLeaseResponse) ProtoMessage() {} func (*RequestLeaseResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e73e8f3e4a270190, []int{68} + return fileDescriptor_api_c364818c36b06139, []int{68} } func (m *RequestLeaseResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3314,7 +3314,7 @@ func (m *ComputeChecksumRequest) Reset() { *m = ComputeChecksumRequest{} func (m *ComputeChecksumRequest) String() string { return proto.CompactTextString(m) } func (*ComputeChecksumRequest) ProtoMessage() {} func (*ComputeChecksumRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e73e8f3e4a270190, []int{69} + return fileDescriptor_api_c364818c36b06139, []int{69} } func (m *ComputeChecksumRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3351,7 +3351,7 @@ func (m *ComputeChecksumResponse) Reset() { *m = ComputeChecksumResponse func (m *ComputeChecksumResponse) String() string { return proto.CompactTextString(m) } func (*ComputeChecksumResponse) ProtoMessage() {} func (*ComputeChecksumResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e73e8f3e4a270190, []int{70} + return fileDescriptor_api_c364818c36b06139, []int{70} } func (m *ComputeChecksumResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3390,7 +3390,7 @@ func (m *ExternalStorage) Reset() { *m = ExternalStorage{} } func (m *ExternalStorage) String() string { return proto.CompactTextString(m) } func (*ExternalStorage) ProtoMessage() {} func (*ExternalStorage) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e73e8f3e4a270190, []int{71} + return fileDescriptor_api_c364818c36b06139, []int{71} } func (m *ExternalStorage) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3424,7 +3424,7 @@ func (m *ExternalStorage_LocalFilePath) Reset() { *m = ExternalStorage_L func (m *ExternalStorage_LocalFilePath) String() string { return proto.CompactTextString(m) } func (*ExternalStorage_LocalFilePath) ProtoMessage() {} func (*ExternalStorage_LocalFilePath) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e73e8f3e4a270190, []int{71, 0} + return fileDescriptor_api_c364818c36b06139, []int{71, 0} } func (m *ExternalStorage_LocalFilePath) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3457,7 +3457,7 @@ func (m *ExternalStorage_Http) Reset() { *m = ExternalStorage_Http{} } func (m *ExternalStorage_Http) String() string { return proto.CompactTextString(m) } func (*ExternalStorage_Http) ProtoMessage() {} func (*ExternalStorage_Http) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e73e8f3e4a270190, []int{71, 1} + return fileDescriptor_api_c364818c36b06139, []int{71, 1} } func (m *ExternalStorage_Http) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3497,7 +3497,7 @@ func (m *ExternalStorage_S3) Reset() { *m = ExternalStorage_S3{} } func (m *ExternalStorage_S3) String() string { return proto.CompactTextString(m) } func (*ExternalStorage_S3) ProtoMessage() {} func (*ExternalStorage_S3) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e73e8f3e4a270190, []int{71, 2} + return fileDescriptor_api_c364818c36b06139, []int{71, 2} } func (m *ExternalStorage_S3) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3536,7 +3536,7 @@ func (m *ExternalStorage_GCS) Reset() { *m = ExternalStorage_GCS{} } func (m *ExternalStorage_GCS) String() string { return proto.CompactTextString(m) } func (*ExternalStorage_GCS) ProtoMessage() {} func (*ExternalStorage_GCS) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e73e8f3e4a270190, []int{71, 3} + return fileDescriptor_api_c364818c36b06139, []int{71, 3} } func (m *ExternalStorage_GCS) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3572,7 +3572,7 @@ func (m *ExternalStorage_Azure) Reset() { *m = ExternalStorage_Azure{} } func (m *ExternalStorage_Azure) String() string { return proto.CompactTextString(m) } func (*ExternalStorage_Azure) ProtoMessage() {} func (*ExternalStorage_Azure) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e73e8f3e4a270190, []int{71, 4} + return fileDescriptor_api_c364818c36b06139, []int{71, 4} } func (m *ExternalStorage_Azure) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3611,7 +3611,7 @@ func (m *ExternalStorage_Workload) Reset() { *m = ExternalStorage_Worklo func (m *ExternalStorage_Workload) String() string { return proto.CompactTextString(m) } func (*ExternalStorage_Workload) ProtoMessage() {} func (*ExternalStorage_Workload) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e73e8f3e4a270190, []int{71, 5} + return fileDescriptor_api_c364818c36b06139, []int{71, 5} } func (m *ExternalStorage_Workload) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3651,7 +3651,7 @@ func (m *WriteBatchRequest) Reset() { *m = WriteBatchRequest{} } func (m *WriteBatchRequest) String() string { return proto.CompactTextString(m) } func (*WriteBatchRequest) ProtoMessage() {} func (*WriteBatchRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e73e8f3e4a270190, []int{72} + return fileDescriptor_api_c364818c36b06139, []int{72} } func (m *WriteBatchRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3685,7 +3685,7 @@ func (m *WriteBatchResponse) Reset() { *m = WriteBatchResponse{} } func (m *WriteBatchResponse) String() string { return proto.CompactTextString(m) } func (*WriteBatchResponse) ProtoMessage() {} func (*WriteBatchResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e73e8f3e4a270190, []int{73} + return fileDescriptor_api_c364818c36b06139, []int{73} } func (m *WriteBatchResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3747,7 +3747,7 @@ func (m *ExportRequest) Reset() { *m = ExportRequest{} } func (m *ExportRequest) String() string { return proto.CompactTextString(m) } func (*ExportRequest) ProtoMessage() {} func (*ExportRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e73e8f3e4a270190, []int{74} + return fileDescriptor_api_c364818c36b06139, []int{74} } func (m *ExportRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3783,7 +3783,7 @@ func (m *BulkOpSummary) Reset() { *m = BulkOpSummary{} } func (m *BulkOpSummary) String() string { return proto.CompactTextString(m) } func (*BulkOpSummary) ProtoMessage() {} func (*BulkOpSummary) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e73e8f3e4a270190, []int{75} + return fileDescriptor_api_c364818c36b06139, []int{75} } func (m *BulkOpSummary) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3819,7 +3819,7 @@ func (m *ExportResponse) Reset() { *m = ExportResponse{} } func (m *ExportResponse) String() string { return proto.CompactTextString(m) } func (*ExportResponse) ProtoMessage() {} func (*ExportResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e73e8f3e4a270190, []int{76} + return fileDescriptor_api_c364818c36b06139, []int{76} } func (m *ExportResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3859,7 +3859,7 @@ 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 fileDescriptor_api_e73e8f3e4a270190, []int{76, 0} + return fileDescriptor_api_c364818c36b06139, []int{76, 0} } func (m *ExportResponse_File) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3909,7 +3909,7 @@ func (m *ImportRequest) Reset() { *m = ImportRequest{} } func (m *ImportRequest) String() string { return proto.CompactTextString(m) } func (*ImportRequest) ProtoMessage() {} func (*ImportRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e73e8f3e4a270190, []int{77} + return fileDescriptor_api_c364818c36b06139, []int{77} } func (m *ImportRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3944,7 +3944,7 @@ 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 fileDescriptor_api_e73e8f3e4a270190, []int{77, 0} + return fileDescriptor_api_c364818c36b06139, []int{77, 0} } func (m *ImportRequest_File) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3980,7 +3980,7 @@ func (m *ImportRequest_TableRekey) Reset() { *m = ImportRequest_TableRek func (m *ImportRequest_TableRekey) String() string { return proto.CompactTextString(m) } func (*ImportRequest_TableRekey) ProtoMessage() {} func (*ImportRequest_TableRekey) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e73e8f3e4a270190, []int{77, 1} + return fileDescriptor_api_c364818c36b06139, []int{77, 1} } func (m *ImportRequest_TableRekey) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4015,7 +4015,7 @@ func (m *ImportResponse) Reset() { *m = ImportResponse{} } func (m *ImportResponse) String() string { return proto.CompactTextString(m) } func (*ImportResponse) ProtoMessage() {} func (*ImportResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e73e8f3e4a270190, []int{78} + return fileDescriptor_api_c364818c36b06139, []int{78} } func (m *ImportResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4053,7 +4053,7 @@ func (m *AdminScatterRequest) Reset() { *m = AdminScatterRequest{} } func (m *AdminScatterRequest) String() string { return proto.CompactTextString(m) } func (*AdminScatterRequest) ProtoMessage() {} func (*AdminScatterRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e73e8f3e4a270190, []int{79} + return fileDescriptor_api_c364818c36b06139, []int{79} } func (m *AdminScatterRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4088,7 +4088,7 @@ func (m *AdminScatterResponse) Reset() { *m = AdminScatterResponse{} } func (m *AdminScatterResponse) String() string { return proto.CompactTextString(m) } func (*AdminScatterResponse) ProtoMessage() {} func (*AdminScatterResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e73e8f3e4a270190, []int{80} + return fileDescriptor_api_c364818c36b06139, []int{80} } func (m *AdminScatterResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4121,7 +4121,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 fileDescriptor_api_e73e8f3e4a270190, []int{80, 0} + return fileDescriptor_api_c364818c36b06139, []int{80, 0} } func (m *AdminScatterResponse_Range) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4166,7 +4166,7 @@ func (m *AdminVerifyProtectedTimestampRequest) Reset() { *m = AdminVerif func (m *AdminVerifyProtectedTimestampRequest) String() string { return proto.CompactTextString(m) } func (*AdminVerifyProtectedTimestampRequest) ProtoMessage() {} func (*AdminVerifyProtectedTimestampRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e73e8f3e4a270190, []int{81} + return fileDescriptor_api_c364818c36b06139, []int{81} } func (m *AdminVerifyProtectedTimestampRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4204,7 +4204,7 @@ func (m *AdminVerifyProtectedTimestampResponse) Reset() { *m = AdminVeri func (m *AdminVerifyProtectedTimestampResponse) String() string { return proto.CompactTextString(m) } func (*AdminVerifyProtectedTimestampResponse) ProtoMessage() {} func (*AdminVerifyProtectedTimestampResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e73e8f3e4a270190, []int{82} + return fileDescriptor_api_c364818c36b06139, []int{82} } func (m *AdminVerifyProtectedTimestampResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4257,7 +4257,7 @@ func (m *AddSSTableRequest) Reset() { *m = AddSSTableRequest{} } func (m *AddSSTableRequest) String() string { return proto.CompactTextString(m) } func (*AddSSTableRequest) ProtoMessage() {} func (*AddSSTableRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e73e8f3e4a270190, []int{83} + return fileDescriptor_api_c364818c36b06139, []int{83} } func (m *AddSSTableRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4291,7 +4291,7 @@ func (m *AddSSTableResponse) Reset() { *m = AddSSTableResponse{} } func (m *AddSSTableResponse) String() string { return proto.CompactTextString(m) } func (*AddSSTableResponse) ProtoMessage() {} func (*AddSSTableResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e73e8f3e4a270190, []int{84} + return fileDescriptor_api_c364818c36b06139, []int{84} } func (m *AddSSTableResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4335,7 +4335,7 @@ func (m *RefreshRequest) Reset() { *m = RefreshRequest{} } func (m *RefreshRequest) String() string { return proto.CompactTextString(m) } func (*RefreshRequest) ProtoMessage() {} func (*RefreshRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e73e8f3e4a270190, []int{85} + return fileDescriptor_api_c364818c36b06139, []int{85} } func (m *RefreshRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4369,7 +4369,7 @@ func (m *RefreshResponse) Reset() { *m = RefreshResponse{} } func (m *RefreshResponse) String() string { return proto.CompactTextString(m) } func (*RefreshResponse) ProtoMessage() {} func (*RefreshResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e73e8f3e4a270190, []int{86} + return fileDescriptor_api_c364818c36b06139, []int{86} } func (m *RefreshResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4408,7 +4408,7 @@ func (m *RefreshRangeRequest) Reset() { *m = RefreshRangeRequest{} } func (m *RefreshRangeRequest) String() string { return proto.CompactTextString(m) } func (*RefreshRangeRequest) ProtoMessage() {} func (*RefreshRangeRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e73e8f3e4a270190, []int{87} + return fileDescriptor_api_c364818c36b06139, []int{87} } func (m *RefreshRangeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4442,7 +4442,7 @@ func (m *RefreshRangeResponse) Reset() { *m = RefreshRangeResponse{} } func (m *RefreshRangeResponse) String() string { return proto.CompactTextString(m) } func (*RefreshRangeResponse) ProtoMessage() {} func (*RefreshRangeResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e73e8f3e4a270190, []int{88} + return fileDescriptor_api_c364818c36b06139, []int{88} } func (m *RefreshRangeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4491,7 +4491,7 @@ func (m *SubsumeRequest) Reset() { *m = SubsumeRequest{} } func (m *SubsumeRequest) String() string { return proto.CompactTextString(m) } func (*SubsumeRequest) ProtoMessage() {} func (*SubsumeRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e73e8f3e4a270190, []int{89} + return fileDescriptor_api_c364818c36b06139, []int{89} } func (m *SubsumeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4540,7 +4540,7 @@ func (m *SubsumeResponse) Reset() { *m = SubsumeResponse{} } func (m *SubsumeResponse) String() string { return proto.CompactTextString(m) } func (*SubsumeResponse) ProtoMessage() {} func (*SubsumeResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e73e8f3e4a270190, []int{90} + return fileDescriptor_api_c364818c36b06139, []int{90} } func (m *SubsumeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4575,7 +4575,7 @@ func (m *RangeStatsRequest) Reset() { *m = RangeStatsRequest{} } func (m *RangeStatsRequest) String() string { return proto.CompactTextString(m) } func (*RangeStatsRequest) ProtoMessage() {} func (*RangeStatsRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e73e8f3e4a270190, []int{91} + return fileDescriptor_api_c364818c36b06139, []int{91} } func (m *RangeStatsRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4614,7 +4614,7 @@ func (m *RangeStatsResponse) Reset() { *m = RangeStatsResponse{} } func (m *RangeStatsResponse) String() string { return proto.CompactTextString(m) } func (*RangeStatsResponse) ProtoMessage() {} func (*RangeStatsResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e73e8f3e4a270190, []int{92} + return fileDescriptor_api_c364818c36b06139, []int{92} } func (m *RangeStatsResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4697,7 +4697,7 @@ func (m *RequestUnion) Reset() { *m = RequestUnion{} } func (m *RequestUnion) String() string { return proto.CompactTextString(m) } func (*RequestUnion) ProtoMessage() {} func (*RequestUnion) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e73e8f3e4a270190, []int{93} + return fileDescriptor_api_c364818c36b06139, []int{93} } func (m *RequestUnion) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -6147,7 +6147,7 @@ func (m *ResponseUnion) Reset() { *m = ResponseUnion{} } func (m *ResponseUnion) String() string { return proto.CompactTextString(m) } func (*ResponseUnion) ProtoMessage() {} func (*ResponseUnion) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e73e8f3e4a270190, []int{94} + return fileDescriptor_api_c364818c36b06139, []int{94} } func (m *ResponseUnion) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -7587,13 +7587,6 @@ type Header struct { // ability. Set this flag to request deferred errors, which can // improve performance under heavy contention when client-side // retries are already inevitable. - // - // Non-transactional requests are not allowed to set this flag (since there's - // nowhere to defer the error to). At the replica level, this flag is ignored - // when the batch contains an EndTxn (similarly to before, there's nowhere to - // defer the error to). However, the client is allowed to set the flag in a - // batch that contains the EndTxn, and it will have an effect for any - // sub-batches that are split off by the DistSender. DeferWriteTooOldError bool `protobuf:"varint,14,opt,name=defer_write_too_old_error,json=deferWriteTooOldError,proto3" json:"defer_write_too_old_error,omitempty"` } @@ -7601,7 +7594,7 @@ func (m *Header) Reset() { *m = Header{} } func (m *Header) String() string { return proto.CompactTextString(m) } func (*Header) ProtoMessage() {} func (*Header) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e73e8f3e4a270190, []int{95} + return fileDescriptor_api_c364818c36b06139, []int{95} } func (m *Header) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -7637,7 +7630,7 @@ type BatchRequest struct { func (m *BatchRequest) Reset() { *m = BatchRequest{} } func (*BatchRequest) ProtoMessage() {} func (*BatchRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e73e8f3e4a270190, []int{96} + return fileDescriptor_api_c364818c36b06139, []int{96} } func (m *BatchRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -7674,7 +7667,7 @@ type BatchResponse struct { func (m *BatchResponse) Reset() { *m = BatchResponse{} } func (*BatchResponse) ProtoMessage() {} func (*BatchResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e73e8f3e4a270190, []int{97} + return fileDescriptor_api_c364818c36b06139, []int{97} } func (m *BatchResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -7732,7 +7725,7 @@ 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 fileDescriptor_api_e73e8f3e4a270190, []int{97, 0} + return fileDescriptor_api_c364818c36b06139, []int{97, 0} } func (m *BatchResponse_Header) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -7771,7 +7764,7 @@ func (m *RangeFeedRequest) Reset() { *m = RangeFeedRequest{} } func (m *RangeFeedRequest) String() string { return proto.CompactTextString(m) } func (*RangeFeedRequest) ProtoMessage() {} func (*RangeFeedRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e73e8f3e4a270190, []int{98} + return fileDescriptor_api_c364818c36b06139, []int{98} } func (m *RangeFeedRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -7812,7 +7805,7 @@ func (m *RangeFeedValue) Reset() { *m = RangeFeedValue{} } func (m *RangeFeedValue) String() string { return proto.CompactTextString(m) } func (*RangeFeedValue) ProtoMessage() {} func (*RangeFeedValue) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e73e8f3e4a270190, []int{99} + return fileDescriptor_api_c364818c36b06139, []int{99} } func (m *RangeFeedValue) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -7853,7 +7846,7 @@ func (m *RangeFeedCheckpoint) Reset() { *m = RangeFeedCheckpoint{} } func (m *RangeFeedCheckpoint) String() string { return proto.CompactTextString(m) } func (*RangeFeedCheckpoint) ProtoMessage() {} func (*RangeFeedCheckpoint) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e73e8f3e4a270190, []int{100} + return fileDescriptor_api_c364818c36b06139, []int{100} } func (m *RangeFeedCheckpoint) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -7890,7 +7883,7 @@ func (m *RangeFeedError) Reset() { *m = RangeFeedError{} } func (m *RangeFeedError) String() string { return proto.CompactTextString(m) } func (*RangeFeedError) ProtoMessage() {} func (*RangeFeedError) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e73e8f3e4a270190, []int{101} + return fileDescriptor_api_c364818c36b06139, []int{101} } func (m *RangeFeedError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -7927,7 +7920,7 @@ func (m *RangeFeedEvent) Reset() { *m = RangeFeedEvent{} } func (m *RangeFeedEvent) String() string { return proto.CompactTextString(m) } func (*RangeFeedEvent) ProtoMessage() {} func (*RangeFeedEvent) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e73e8f3e4a270190, []int{102} + return fileDescriptor_api_c364818c36b06139, []int{102} } func (m *RangeFeedEvent) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -36854,9 +36847,9 @@ var ( ErrIntOverflowApi = fmt.Errorf("proto: integer overflow") ) -func init() { proto.RegisterFile("roachpb/api.proto", fileDescriptor_api_e73e8f3e4a270190) } +func init() { proto.RegisterFile("roachpb/api.proto", fileDescriptor_api_c364818c36b06139) } -var fileDescriptor_api_e73e8f3e4a270190 = []byte{ +var fileDescriptor_api_c364818c36b06139 = []byte{ // 7167 bytes of a gzipped FileDescriptorProto 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xe4, 0x7d, 0x5d, 0x6c, 0x24, 0xc7, 0x75, 0x2e, 0x7b, 0x66, 0x48, 0xce, 0x9c, 0xf9, 0xe1, 0xb0, 0xb8, 0x3f, 0xb3, 0x94, 0x44, 0x72, diff --git a/pkg/roachpb/api.proto b/pkg/roachpb/api.proto index 647c9f10742b..e9ac6cb5eda8 100644 --- a/pkg/roachpb/api.proto +++ b/pkg/roachpb/api.proto @@ -1791,13 +1791,6 @@ message Header { // ability. Set this flag to request deferred errors, which can // improve performance under heavy contention when client-side // retries are already inevitable. - // - // Non-transactional requests are not allowed to set this flag (since there's - // nowhere to defer the error to). At the replica level, this flag is ignored - // when the batch contains an EndTxn (similarly to before, there's nowhere to - // defer the error to). However, the client is allowed to set the flag in a - // batch that contains the EndTxn, and it will have an effect for any - // sub-batches that are split off by the DistSender. bool defer_write_too_old_error = 14; reserved 7,12; } diff --git a/pkg/roachpb/batch.go b/pkg/roachpb/batch.go index 8bb7e4940119..cb6d197df819 100644 --- a/pkg/roachpb/batch.go +++ b/pkg/roachpb/batch.go @@ -18,7 +18,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/storage/engine/enginepb" "github.com/cockroachdb/cockroach/pkg/util/hlc" - "github.com/cockroachdb/errors" + "github.com/pkg/errors" ) //go:generate go run -tags gen-batch gen_batch.go @@ -483,12 +483,6 @@ func (ba *BatchRequest) Methods() []Method { // special-cased: If false, an EndTxn request will never be split into a new // chunk (otherwise, it is treated according to its flags). This allows sending // a whole transaction in a single Batch when addressing a single range. -// -// NOTE: One reason for splitting reads from writes is that write-only batches -// can sometimes have their read timestamp bumped on the server, which doesn't -// work for read requests due to how the timestamp-aware latching works (i.e. a -// read that acquired a latch @ ts10 can't simply be bumped to ts 20 because -// there might have been overlapping writes in the 10..20 window). func (ba BatchRequest) Split(canSplitET bool) [][]RequestUnion { compatible := func(exFlags, newFlags int) bool { // isAlone requests are never compatible. @@ -601,26 +595,3 @@ func (ba BatchRequest) String() string { } return strings.Join(str, ", ") } - -// ValidateForEvaluation performs sanity checks on the batch when it's received -// by the "server" for evaluation. -func (ba BatchRequest) ValidateForEvaluation() error { - if ba.RangeID == 0 { - return errors.AssertionFailedf("batch request missing range ID") - } else if ba.Replica.StoreID == 0 { - return errors.AssertionFailedf("batch request missing store ID") - } - if ba.Header.DeferWriteTooOldError && ba.Txn == nil { - return errors.AssertionFailedf( - "DeferWriteTooOldError can't be set on non-transactional requests") - } - if _, ok := ba.GetArg(EndTxn); ok && ba.Txn == nil { - return errors.AssertionFailedf("EndTxn request without transaction") - } - if ba.Txn != nil { - if ba.Txn.WriteTooOld && (ba.Txn.ReadTimestamp.Equal(ba.Txn.WriteTimestamp)) { - return errors.AssertionFailedf("WriteTooOld set but no offset in timestamps. txn: %s", ba.Txn) - } - } - return nil -} diff --git a/pkg/roachpb/data.go b/pkg/roachpb/data.go index 26dbbbadb2a6..f79070ef5b3f 100644 --- a/pkg/roachpb/data.go +++ b/pkg/roachpb/data.go @@ -765,12 +765,6 @@ func (ts TransactionStatus) IsFinalized() bool { return ts == COMMITTED || ts == ABORTED } -// IsCommittedOrStaging determines if the transaction is morally committed (i.e. -// in the COMMITTED or STAGING state). -func (ts TransactionStatus) IsCommittedOrStaging() bool { - return ts == COMMITTED || ts == STAGING -} - var _ log.SafeMessager = Transaction{} // MakeTransaction creates a new transaction. The transaction key is diff --git a/pkg/storage/batcheval/cmd_end_transaction.go b/pkg/storage/batcheval/cmd_end_transaction.go index 53d39e4beee4..e6c4df208037 100644 --- a/pkg/storage/batcheval/cmd_end_transaction.go +++ b/pkg/storage/batcheval/cmd_end_transaction.go @@ -409,7 +409,12 @@ func IsEndTxnTriggeringRetryError( } } - // A transaction must obey its deadline, if set. + // A transaction can still avoid a retry under certain conditions. + if retry && CanForwardCommitTimestampWithoutRefresh(txn, args) { + retry, reason = false, 0 + } + + // However, a transaction must obey its deadline, if set. if !retry && IsEndTxnExceedingDeadline(txn.WriteTimestamp, args) { exceededBy := txn.WriteTimestamp.GoTime().Sub(args.Deadline.GoTime()) extraMsg = fmt.Sprintf( @@ -427,9 +432,6 @@ func IsEndTxnTriggeringRetryError( // has encountered no spans which require refreshing at the forwarded // timestamp. If either of those conditions are true, a client-side // retry is required. -// -// Note that when deciding whether a transaction can be bumped to a particular -// timestamp, the transaction's deadling must also be taken into account. func CanForwardCommitTimestampWithoutRefresh( txn *roachpb.Transaction, args *roachpb.EndTxnRequest, ) bool { diff --git a/pkg/storage/batcheval/cmd_end_transaction_test.go b/pkg/storage/batcheval/cmd_end_transaction_test.go index 0ae4e7a7b6eb..bd45ad6ce585 100644 --- a/pkg/storage/batcheval/cmd_end_transaction_test.go +++ b/pkg/storage/batcheval/cmd_end_transaction_test.go @@ -95,6 +95,7 @@ func TestEndTxnUpdatesTransactionRecord(t *testing.T) { noIntentSpans bool inFlightWrites []roachpb.SequencedWrite deadline *hlc.Timestamp + noRefreshSpans bool // Expected result. expError string expTxn *roachpb.TransactionRecord @@ -289,6 +290,43 @@ func TestEndTxnUpdatesTransactionRecord(t *testing.T) { return &record }(), }, + { + // The transaction's commit timestamp was increased during its + // lifetime and it has never read anything. The stage will succeed. + name: "record missing, can create, try stage at pushed timestamp, can forward timestamp", + // Replica state. + existingTxn: nil, + canCreateTxn: func() (bool, hlc.Timestamp) { return true, hlc.Timestamp{} }, + // Request state. + headerTxn: pushedHeaderTxn, + commit: true, + inFlightWrites: writes, + noRefreshSpans: true, + // Expected result. + expTxn: func() *roachpb.TransactionRecord { + record := *stagingRecord + record.WriteTimestamp.Forward(ts2) + return &record + }(), + }, + { + // The transaction's commit timestamp was increased during its + // lifetime and it has never read anything. The commit will succeed. + name: "record missing, can create, try commit at pushed timestamp, can forward timestamp", + // Replica state. + existingTxn: nil, + canCreateTxn: func() (bool, hlc.Timestamp) { return true, hlc.Timestamp{} }, + // Request state. + headerTxn: pushedHeaderTxn, + commit: true, + noRefreshSpans: true, + // Expected result. + expTxn: func() *roachpb.TransactionRecord { + record := *committedRecord + record.WriteTimestamp.Forward(ts2) + return &record + }(), + }, { // A PushTxn(TIMESTAMP) request bumped the minimum timestamp that the // transaction can be created with. This will trigger a retry error. @@ -353,6 +391,43 @@ func TestEndTxnUpdatesTransactionRecord(t *testing.T) { return &record }(), }, + { + // A PushTxn(TIMESTAMP) request bumped the minimum timestamp that the + // transaction can be created with. This will trigger a retry error. + name: "record missing, can create with min timestamp, try stage, can forward timestamp", + // Replica state. + existingTxn: nil, + canCreateTxn: func() (bool, hlc.Timestamp) { return true, ts2 }, + // Request state. + headerTxn: headerTxn, + commit: true, + inFlightWrites: writes, + noRefreshSpans: true, + // Expected result. + expTxn: func() *roachpb.TransactionRecord { + record := *stagingRecord + record.WriteTimestamp.Forward(ts2) + return &record + }(), + }, + { + // A PushTxn(TIMESTAMP) request bumped the minimum timestamp that the + // transaction can be created with. This will trigger a retry error. + name: "record missing, can create with min timestamp, try commit, can forward timestamp", + // Replica state. + existingTxn: nil, + canCreateTxn: func() (bool, hlc.Timestamp) { return true, ts2 }, + // Request state. + headerTxn: headerTxn, + commit: true, + noRefreshSpans: true, + // Expected result. + expTxn: func() *roachpb.TransactionRecord { + record := *committedRecord + record.WriteTimestamp.Forward(ts2) + return &record + }(), + }, { // The transaction has run into a WriteTooOld error during its // lifetime. The stage will be rejected. @@ -454,6 +529,41 @@ func TestEndTxnUpdatesTransactionRecord(t *testing.T) { // Expected result. expError: "TransactionRetryError: retry txn (RETRY_SERIALIZABLE)", }, + { + // The transaction's commit timestamp was increased during its + // lifetime and it has never read anything. The stage will succeed. + name: "record pending, try stage at pushed timestamp, can forward timestamp", + // Replica state. + existingTxn: pendingRecord, + // Request state. + headerTxn: pushedHeaderTxn, + commit: true, + inFlightWrites: writes, + noRefreshSpans: true, + // Expected result. + expTxn: func() *roachpb.TransactionRecord { + record := *stagingRecord + record.WriteTimestamp.Forward(ts2) + return &record + }(), + }, + { + // The transaction's commit timestamp was increased during its + // lifetime and it has never read anything. The commit will succeed. + name: "record pending, try commit at pushed timestamp, can forward timestamp", + // Replica state. + existingTxn: pendingRecord, + // Request state. + headerTxn: pushedHeaderTxn, + commit: true, + noRefreshSpans: true, + // Expected result. + expTxn: func() *roachpb.TransactionRecord { + record := *committedRecord + record.WriteTimestamp.Forward(ts2) + return &record + }(), + }, { // The transaction's commit timestamp was increased during its // lifetime and it has refreshed up to this timestamp. The stage @@ -887,6 +997,7 @@ func TestEndTxnUpdatesTransactionRecord(t *testing.T) { if !c.commit { require.Nil(t, c.inFlightWrites) require.Nil(t, c.deadline) + require.False(t, c.noRefreshSpans) } // Issue an EndTxn request. @@ -894,8 +1005,9 @@ func TestEndTxnUpdatesTransactionRecord(t *testing.T) { RequestHeader: roachpb.RequestHeader{Key: txn.Key}, Commit: c.commit, - InFlightWrites: c.inFlightWrites, - Deadline: c.deadline, + InFlightWrites: c.inFlightWrites, + Deadline: c.deadline, + CanCommitAtHigherTimestamp: c.noRefreshSpans, } if !c.noIntentSpans { req.IntentSpans = intents diff --git a/pkg/storage/batcheval/transaction_test.go b/pkg/storage/batcheval/transaction_test.go index 60758e9e4fe7..029d61006c12 100644 --- a/pkg/storage/batcheval/transaction_test.go +++ b/pkg/storage/batcheval/transaction_test.go @@ -145,7 +145,7 @@ func TestUpdateAbortSpan(t *testing.T) { expErr string // empty if no error expected }{ /////////////////////////////////////////////////////////////////////// - // EndTxnRequest // + // EndTransactionRequest // /////////////////////////////////////////////////////////////////////// { name: "end txn, rollback, no poison, intent missing, abort span missing", diff --git a/pkg/storage/client_lease_test.go b/pkg/storage/client_lease_test.go index 5a21b682b24b..8cd0f0dd5f16 100644 --- a/pkg/storage/client_lease_test.go +++ b/pkg/storage/client_lease_test.go @@ -429,7 +429,7 @@ func TestTimestampCacheErrorAfterLeaseTransfer(t *testing.T) { require.NoError(t, tc.TransferRangeLease(rangeDesc, tc.Target(0))) // Start a txn and perform a write, so that a txn record has to be created by - // the EndTxn. + // the EndTransaction. txn := tc.Servers[0].DB().NewTxn(ctx, "test") require.NoError(t, txn.Put(ctx, "a", "val")) // After starting the transaction, transfer the lease. This will wipe the diff --git a/pkg/storage/engine/mvcc.go b/pkg/storage/engine/mvcc.go index 90e1a08e032f..f419cd7cda3c 100644 --- a/pkg/storage/engine/mvcc.go +++ b/pkg/storage/engine/mvcc.go @@ -3480,12 +3480,10 @@ func ComputeStatsGo( ms.IntentAge += nowNanos/1e9 - meta.Timestamp.WallTime/1e9 } if meta.KeyBytes != MVCCVersionTimestampSize { - return ms, errors.Errorf("expected mvcc metadata key bytes to equal %d; got %d "+ - "(meta: %s)", MVCCVersionTimestampSize, meta.KeyBytes, &meta) + return ms, errors.Errorf("expected mvcc metadata key bytes to equal %d; got %d", MVCCVersionTimestampSize, meta.KeyBytes) } if meta.ValBytes != int64(len(unsafeValue)) { - return ms, errors.Errorf("expected mvcc metadata val bytes to equal %d; got %d "+ - "(meta: %s)", len(unsafeValue), meta.ValBytes, &meta) + return ms, errors.Errorf("expected mvcc metadata val bytes to equal %d; got %d", len(unsafeValue), meta.ValBytes) } accrueGCAgeNanos = meta.Timestamp.WallTime } else { diff --git a/pkg/storage/replica_batch_updates.go b/pkg/storage/replica_batch_updates.go index 7c78ebecfa7b..96e0c1eac21d 100644 --- a/pkg/storage/replica_batch_updates.go +++ b/pkg/storage/replica_batch_updates.go @@ -11,12 +11,7 @@ package storage import ( - "context" - "github.com/cockroachdb/cockroach/pkg/roachpb" - "github.com/cockroachdb/cockroach/pkg/storage/batcheval" - "github.com/cockroachdb/cockroach/pkg/util/hlc" - "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/pkg/errors" ) @@ -164,50 +159,3 @@ func maybeStripInFlightWrites(ba *roachpb.BatchRequest) (*roachpb.BatchRequest, } return ba, nil } - -// maybeBumpReadTimestampToWriteTimestamp bumps the batch's read timestamp to -// the write timestamp for transactional batches where these timestamp have -// diverged and where bumping is possible. When possible, this allows the -// transaction to commit without having to retry. -// -// Note that this, like all the server-side bumping of the read timestamp, only -// works for batches that exclusively contain writes; reads cannot be bumped -// like this because they've already acquired timestamp-aware latches. -func maybeBumpReadTimestampToWriteTimestamp(ctx context.Context, ba *roachpb.BatchRequest) { - if ba.Txn == nil { - return - } - if ba.Txn.ReadTimestamp.Equal(ba.Txn.WriteTimestamp) { - return - } - arg, ok := ba.GetArg(roachpb.EndTxn) - if !ok { - return - } - etArg := arg.(*roachpb.EndTxnRequest) - if batcheval.CanForwardCommitTimestampWithoutRefresh(ba.Txn, etArg) && - !batcheval.IsEndTxnExceedingDeadline(ba.Txn.WriteTimestamp, etArg) { - bumpBatchTimestamp(ctx, ba, ba.Txn.WriteTimestamp) - } -} - -// bumpBatchTimestamp bumps ba's read and write timestamps to ts. -func bumpBatchTimestamp(ctx context.Context, ba *roachpb.BatchRequest, ts hlc.Timestamp) { - if ts.Less(ba.Timestamp) { - log.Fatalf(ctx, "trying to bump to %s <= ba.Timestamp: %s", ts, ba.Timestamp) - } - ba.Timestamp = ts - if txn := ba.Txn; txn == nil { - return - } - if ts.Less(ba.Txn.ReadTimestamp) || ts.Less(ba.Txn.WriteTimestamp) { - log.Fatalf(ctx, "trying to bump to %s inconsistent with ba.Txn.ReadTimestamp: %s, "+ - "ba.Txn.WriteTimestamp: %s", ts, ba.Txn.ReadTimestamp, ba.Txn.WriteTimestamp) - } - log.VEventf(ctx, 2, "bumping batch timestamp to: %s from read: %s, write: %s)", - ts, ba.Txn.ReadTimestamp, ba.Txn.WriteTimestamp) - ba.Txn = ba.Txn.Clone() - ba.Txn.ReadTimestamp = ts - ba.Txn.WriteTimestamp = ba.Timestamp - ba.Txn.WriteTooOld = false -} diff --git a/pkg/storage/replica_evaluate.go b/pkg/storage/replica_evaluate.go index 35be170528dc..f89394934475 100644 --- a/pkg/storage/replica_evaluate.go +++ b/pkg/storage/replica_evaluate.go @@ -22,8 +22,8 @@ import ( "github.com/cockroachdb/cockroach/pkg/storage/engine/enginepb" "github.com/cockroachdb/cockroach/pkg/storage/storagebase" "github.com/cockroachdb/cockroach/pkg/util/log" - "github.com/cockroachdb/errors" "github.com/kr/pretty" + "github.com/pkg/errors" ) // optimizePuts searches for contiguous runs of Put & CPut commands in @@ -191,7 +191,7 @@ func evaluateBatch( } } - var mergedResult result.Result + var result result.Result // WriteTooOldErrors are unique: When one is returned, we also lay // down an intent at our new proposed timestamp. We have the option @@ -200,9 +200,8 @@ func evaluateBatch( // a RefreshSpan and possibly a client-side retry). // // Within a batch, there's no downside to continuing past the - // WriteTooOldError, so we at least defer returning the error to the end of - // the batch so that we lay down more intents and we find out the final - // timestamp for the batch. + // WriteTooOldError, so we at least defer returning the error to the + // end of the batch. // // Across batches, it's more complicated. We want to avoid // client-side retries whenever possible. However, if a client-side @@ -235,16 +234,12 @@ func evaluateBatch( // WriteTooOldError even if the SQL CanAutoRetry is false. As of // this writing, nearly all writes issued by SQL are preceded by // reads of the same key. - var writeTooOldState struct { - err *roachpb.WriteTooOldError - // cantDeferWTOE is set when a WriteTooOldError cannot be deferred. - cantDeferWTOE bool - } + var writeTooOldErr *roachpb.Error + mustReturnWriteTooOldErr := false for index, union := range baReqs { // Execute the command. args := union.GetInner() - if baHeader.Txn != nil { // Set the Request's sequence number on the TxnMeta for this // request. The MVCC layer (currently) uses TxnMeta to @@ -252,34 +247,18 @@ func evaluateBatch( // request operates. baHeader.Txn.Sequence = args.Header().Sequence } - // Note that responses are populated even when an error is returned. // TODO(tschottdorf): Change that. IIRC there is nontrivial use of it currently. reply := br.Responses[index].GetInner() + curResult, pErr := evaluateCommand(ctx, idKey, index, readWriter, rec, ms, baHeader, maxKeys, args, reply) - var curResult result.Result - var pErr *roachpb.Error - curResult, pErr = evaluateCommand( - ctx, idKey, index, readWriter, rec, ms, baHeader, maxKeys, args, reply) - - // If an EndTxn wants to restart because of a write too old, we - // might have a better error to return to the client. - retErr, ok := pErr.GetDetail().(*roachpb.TransactionRetryError) - if ok && retErr.Reason == roachpb.RETRY_WRITE_TOO_OLD && - args.Method() == roachpb.EndTxn && writeTooOldState.err != nil { - pErr.SetDetail(writeTooOldState.err) - // Remember not to defer this error. Since it came from an EndTransaction, - // there's nowhere to defer it to. - writeTooOldState.cantDeferWTOE = true - } - - if err := mergedResult.MergeAndDestroy(curResult); err != nil { + if err := result.MergeAndDestroy(curResult); err != nil { // TODO(tschottdorf): see whether we really need to pass nontrivial // Result up on error and if so, formalize that. log.Fatalf( ctx, "unable to absorb Result: %s\ndiff(new, old): %s", - err, pretty.Diff(curResult, mergedResult), + err, pretty.Diff(curResult, result), ) } @@ -296,11 +275,19 @@ func evaluateBatch( // other concurrent overlapping transactions are forced // through intent resolution and the chances of this batch // succeeding when it will be retried are increased. - if writeTooOldState.err != nil { - writeTooOldState.err.ActualTimestamp.Forward( - tErr.ActualTimestamp) + if writeTooOldErr != nil { + writeTooOldErr.GetDetail().(*roachpb.WriteTooOldError).ActualTimestamp.Forward(tErr.ActualTimestamp) } else { - writeTooOldState.err = tErr + writeTooOldErr = pErr + } + + // Requests which are both read and write are not currently + // accounted for in RefreshSpans, so they rely on eager + // returning of WriteTooOldErrors. + // TODO(bdarnell): add read+write requests to the read refresh spans + // in TxnCoordSender, and then I think this can go away. + if roachpb.IsReadAndWrite(args) { + mustReturnWriteTooOldErr = true } if baHeader.Txn != nil { @@ -316,7 +303,7 @@ func evaluateBatch( // transaction. pErr = nil default: - return nil, mergedResult, pErr + return nil, result, pErr } } @@ -342,16 +329,20 @@ func evaluateBatch( } } - if baHeader.Txn != nil && baHeader.Txn.Status.IsCommittedOrStaging() { - if writeTooOldState.err != nil { - log.Fatalf(ctx, "committed txn with writeTooOld err: %s", writeTooOldState.err) - } + // If there was an EndTxn in the batch that finalized the transaction, + // the WriteTooOld status has been fully processed and we can discard the error. + if baHeader.Txn != nil && baHeader.Txn.Status.IsFinalized() { + writeTooOldErr = nil + } else if baHeader.Txn == nil { + // Non-transactional requests are unable to defer WriteTooOldErrors + // because there is no where to defer them to. + mustReturnWriteTooOldErr = true } - // If there's a write too old error that we don't want to defer, return. - if writeTooOldState.err != nil && - (!baHeader.DeferWriteTooOldError || writeTooOldState.cantDeferWTOE) { - return nil, mergedResult, roachpb.NewErrorWithTxn(writeTooOldState.err, baHeader.Txn) + // If there's a write too old error, return now that we've found + // the high water timestamp for retries. + if writeTooOldErr != nil && (mustReturnWriteTooOldErr || !baHeader.DeferWriteTooOldError) { + return nil, result, writeTooOldErr } if baHeader.Txn != nil { @@ -359,16 +350,16 @@ func evaluateBatch( br.Txn = baHeader.Txn // Set br.Timestamp to the timestamp at which reads were evaluated. Note // that this might be higher than baHeader.Timestamp if we had an - // EndTxn that decided that it can essentially refresh to something + // EndTransaction that decided that it can essentially refresh to something // higher than baHeader.Timestamp because there were no refresh spans. - br.Timestamp.Forward(br.Txn.ReadTimestamp) + br.Timestamp.Forward(baHeader.Txn.ReadTimestamp) } else { // Always update the batch response timestamp field to the timestamp at // which the batch executed. br.Timestamp.Forward(baHeader.Timestamp) } - return br, mergedResult, nil + return br, result, nil } // evaluateCommand delegates to the eval method for the given @@ -398,9 +389,6 @@ func evaluateCommand( Hdr: h, } if pErr := filter(filterArgs); pErr != nil { - if pErr.GetTxn() == nil { - pErr.SetTxn(h.Txn) - } log.Infof(ctx, "test injecting error: %s", pErr) return result.Result{}, pErr } @@ -409,21 +397,21 @@ func evaluateCommand( var err error var pd result.Result - cArgs := batcheval.CommandArgs{ - EvalCtx: rec, - Header: h, - Args: args, - MaxKeys: maxKeys, - Stats: ms, - } if cmd, ok := batcheval.LookupCommand(args.Method()); ok { + cArgs := batcheval.CommandArgs{ + EvalCtx: rec, + Header: h, + Args: args, + MaxKeys: maxKeys, + Stats: ms, + } if cmd.EvalRW != nil { pd, err = cmd.EvalRW(ctx, readWriter, cArgs, reply) } else { pd, err = cmd.EvalRO(ctx, readWriter, cArgs, reply) } } else { - err = errors.AssertionFailedf("unrecognized command %s", args.Method()) + err = errors.Errorf("unrecognized command %s", args.Method()) } if h.ReturnRangeInfo { diff --git a/pkg/storage/replica_test.go b/pkg/storage/replica_test.go index 1469430c32cb..e33f01e28c08 100644 --- a/pkg/storage/replica_test.go +++ b/pkg/storage/replica_test.go @@ -440,7 +440,6 @@ func createReplicaSets(replicaNumbers []roachpb.StoreID) []roachpb.ReplicaDescri // transactional batch can be committed as an atomic write. func TestIsOnePhaseCommit(t *testing.T) { defer leaktest.AfterTest(t)() - ctx := context.Background() withSeq := func(req roachpb.Request, seq enginepb.TxnSeq) roachpb.Request { h := req.Header() h.Sequence = seq @@ -519,38 +518,27 @@ func TestIsOnePhaseCommit(t *testing.T) { clock := hlc.NewClock(hlc.UnixNano, time.Nanosecond) for i, c := range testCases { - t.Run( - fmt.Sprintf("%d:isTxn:%t,isRestarted:%t,isWTO:%t,isTSOff:%t", - i, c.isTxn, c.isRestarted, c.isWTO, c.isTSOff), - func(t *testing.T) { - ba := roachpb.BatchRequest{Requests: c.ru} - if c.isTxn { - ba.Txn = newTransaction("txn", roachpb.Key("a"), 1, clock) - if c.isRestarted { - ba.Txn.Restart(-1, 0, clock.Now()) - } - if c.isWTO { - ba.Txn.WriteTooOld = true - c.isTSOff = true - } - if c.isTSOff { - ba.Txn.WriteTimestamp = ba.Txn.ReadTimestamp.Add(1, 0) - } - } else { - require.False(t, c.isRestarted) - require.False(t, c.isWTO) - require.False(t, c.isTSOff) - } - - // Emulate what a server actually does and bump the write timestamp when - // possible. This makes some batches with diverged read and write - // timestamps to still pass isOnePhaseCommit(). - maybeBumpReadTimestampToWriteTimestamp(ctx, &ba) - - if is1PC := isOnePhaseCommit(&ba); is1PC != c.exp1PC { - t.Errorf("expected 1pc=%t; got %t", c.exp1PC, is1PC) - } - }) + ba := roachpb.BatchRequest{Requests: c.ru} + if c.isTxn { + ba.Txn = newTransaction("txn", roachpb.Key("a"), 1, clock) + if c.isRestarted { + ba.Txn.Restart(-1, 0, clock.Now()) + } + if c.isWTO { + ba.Txn.WriteTooOld = true + c.isTSOff = true + } + if c.isTSOff { + ba.Txn.WriteTimestamp = ba.Txn.ReadTimestamp.Add(1, 0) + } + } else { + require.False(t, c.isRestarted) + require.False(t, c.isWTO) + require.False(t, c.isTSOff) + } + if is1PC := isOnePhaseCommit(&ba); is1PC != c.exp1PC { + t.Errorf("%d: expected 1pc=%t; got %t", i, c.exp1PC, is1PC) + } } } @@ -4343,8 +4331,9 @@ func TestRPCRetryProtectionInTxn(t *testing.T) { if pErr == nil { t.Fatalf("expected error, got nil") } - require.Error(t, pErr.GetDetail(), - "TransactionAbortedError(ABORT_REASON_ALREADY_COMMITTED_OR_ROLLED_BACK_POSSIBLE_REPLAY)") + if _, ok := pErr.GetDetail().(*roachpb.TransactionAbortedError); !ok { + t.Fatalf("expected TransactionAbortedError; got %s", pErr) + } } // TestReplicaLaziness verifies that Raft Groups are brought up lazily. @@ -9582,10 +9571,10 @@ func TestConsistenctQueueErrorFromCheckConsistency(t *testing.T) { } } -// TestReplicaServersideRefreshes verifies local retry logic for transactional -// and non transactional batches. Verifies the timestamp cache is updated to -// reflect the timestamp at which retried batches are executed. -func TestReplicaServersideRefreshes(t *testing.T) { +// TestReplicaLocalRetries verifies local retry logic for transactional +// and non transactional batches. Verifies the timestamp cache is updated +// to reflect the timestamp at which retried batches are executed. +func TestReplicaLocalRetries(t *testing.T) { defer leaktest.AfterTest(t)() tc := testContext{} stopper := stop.NewStopper() @@ -9744,7 +9733,7 @@ func TestReplicaServersideRefreshes(t *testing.T) { assignSeqNumsForReqs(ba.Txn, &cput, &et) return }, - expErr: "WriteTooOldError", + expErr: "RETRY_WRITE_TOO_OLD", }, // 1PC serializable transaction will retry locally. { @@ -9893,11 +9882,6 @@ func TestReplicaServersideRefreshes(t *testing.T) { return }, }, - // TODO(andrei): We should also have a test similar to the one above, but - // with the WriteTooOld flag set by a different batch than the one with the - // EndTransaction. This is hard to do at the moment, though, because we - // never defer the handling of the write too old conditions to the end of - // the transaction (but we might in the future). } for _, test := range testCases { @@ -9957,7 +9941,7 @@ func TestReplicaPushed1PC(t *testing.T) { txn.WriteTimestamp.Forward(ts3) // Execute the write phase of the transaction as a single batch, - // which must return a WriteTooOldError. + // which must return a WRITE_TOO_OLD TransactionRetryError. // // TODO(bdarnell): When this test was written, in SNAPSHOT // isolation we would attempt to execute the transaction on the @@ -9976,8 +9960,10 @@ func TestReplicaPushed1PC(t *testing.T) { assignSeqNumsForReqs(&txn, &put, &et) if br, pErr := tc.Sender().Send(ctx, ba); pErr == nil { t.Errorf("did not get expected error. resp=%s", br) - } else if wtoe, ok := pErr.GetDetail().(*roachpb.WriteTooOldError); !ok { - t.Errorf("expected WriteTooOldError, got %s", wtoe) + } else if trErr, ok := pErr.GetDetail().(*roachpb.TransactionRetryError); !ok { + t.Errorf("expected TransactionRetryError, got %s", pErr) + } else if trErr.Reason != roachpb.RETRY_WRITE_TOO_OLD { + t.Errorf("expected RETRY_WRITE_TOO_OLD, got %s", trErr) } } diff --git a/pkg/storage/replica_write.go b/pkg/storage/replica_write.go index 389d594091f1..7565677e3415 100644 --- a/pkg/storage/replica_write.go +++ b/pkg/storage/replica_write.go @@ -25,7 +25,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/storage/storagebase" "github.com/cockroachdb/cockroach/pkg/storage/storagepb" "github.com/cockroachdb/cockroach/pkg/util" - "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/timeutil" "github.com/pkg/errors" @@ -253,65 +252,31 @@ func (r *Replica) evaluateWriteBatch( ctx context.Context, idKey storagebase.CmdIDKey, ba *roachpb.BatchRequest, spans *spanset.SpanSet, ) (engine.Batch, enginepb.MVCCStats, *roachpb.BatchResponse, result.Result, *roachpb.Error) { ms := enginepb.MVCCStats{} - - // If the transaction has been pushed but it can commit at the higher - // timestamp, let's evaluate the batch at the bumped timestamp. This will - // allow it commit, and also it'll allow us to attempt the 1PC code path. - maybeBumpReadTimestampToWriteTimestamp(ctx, ba) - - // Attempt 1PC execution, if applicable. If not transactional or there are - // indications that the batch's txn will require retry, execute as normal. + // If not transactional or there are indications that the batch's txn will + // require restart or retry, execute as normal. if isOnePhaseCommit(ba) { - log.VEventf(ctx, 2, "attempting 1PC execution") arg, _ := ba.GetArg(roachpb.EndTxn) etArg := arg.(*roachpb.EndTxnRequest) - if ba.Timestamp != ba.Txn.WriteTimestamp { - log.Fatalf(ctx, "unexpected 1PC execution with diverged timestamp. %s != %s", - ba.Timestamp, ba.Txn.WriteTimestamp) - } - - // Try executing with transaction stripped. + // Try executing with transaction stripped. We use the transaction timestamp + // to write any values as it may have been advanced by the timestamp cache. strippedBa := *ba + strippedBa.Timestamp = strippedBa.Txn.WriteTimestamp strippedBa.Txn = nil - // strippedBa is non-transactional, so DeferWriteTooOldError cannot be set. - strippedBa.DeferWriteTooOldError = false strippedBa.Requests = ba.Requests[:len(ba.Requests)-1] // strip end txn req + // Is the transaction allowed to retry locally in the event of + // write too old errors? This is only allowed if it is able to + // forward its commit timestamp without a read refresh. + canForwardTimestamp := batcheval.CanForwardCommitTimestampWithoutRefresh(ba.Txn, etArg) + + // If all writes occurred at the intended timestamp, we've succeeded on the fast path. rec := NewReplicaEvalContext(r, spans) - batch, br, res, pErr := r.evaluateWriteBatchWithServersideRefreshes( - ctx, idKey, rec, &ms, &strippedBa, spans, + batch, br, res, pErr := r.evaluateWriteBatchWithLocalRetries( + ctx, idKey, rec, &ms, &strippedBa, spans, canForwardTimestamp, ) - - type onePCResult struct { - success bool - // pErr is set if success == false and regular transactional execution - // should not be attempted. Conversely, if success is not set and pErr is - // not set, then transactional execution should be attempted. - pErr *roachpb.Error - - // The fields below are only set when success is set. - stats enginepb.MVCCStats - br *roachpb.BatchResponse - res result.Result - } - synthesizeEndTxnResponse := func() onePCResult { - if pErr != nil { - return onePCResult{success: false} - } - commitTS := br.Timestamp - - // If we were pushed ... - if ba.Timestamp != commitTS && - // ... and the batch can't commit at the pushed timestamp ... - (!batcheval.CanForwardCommitTimestampWithoutRefresh(ba.Txn, etArg) || - batcheval.IsEndTxnExceedingDeadline(commitTS, etArg)) { - // ... then the 1PC execution was not successful. - return onePCResult{success: false} - } - - // 1PC execution was successful, let's synthesize an EndTxnResponse. - + if pErr == nil && (ba.Timestamp == br.Timestamp || + (canForwardTimestamp && !batcheval.IsEndTxnExceedingDeadline(br.Timestamp, etArg))) { clonedTxn := ba.Txn.Clone() clonedTxn.Status = roachpb.COMMITTED // Make sure the returned txn has the actual commit @@ -329,54 +294,32 @@ func (r *Replica) evaluateWriteBatch( // Run commit trigger manually. innerResult, err := batcheval.RunCommitTrigger(ctx, rec, batch, &ms, etArg, clonedTxn) if err != nil { - return onePCResult{ - success: false, - pErr: roachpb.NewErrorf("failed to run commit trigger: %s", err), - } + return batch, ms, br, res, roachpb.NewErrorf("failed to run commit trigger: %s", err) } if err := res.MergeAndDestroy(innerResult); err != nil { - return onePCResult{ - success: false, - pErr: roachpb.NewError(err), - } + return batch, ms, br, res, roachpb.NewError(err) } } // Add placeholder responses for end transaction requests. br.Add(&roachpb.EndTxnResponse{OnePhaseCommit: true}) br.Txn = clonedTxn - return onePCResult{ - success: true, - stats: ms, - br: br, - res: res, - } - } - onePCRes := synthesizeEndTxnResponse() - if onePCRes.success { - return batch, onePCRes.stats, onePCRes.br, onePCRes.res, nil - } - if onePCRes.pErr != nil { - return batch, enginepb.MVCCStats{}, nil, result.Result{}, onePCRes.pErr + return batch, ms, br, res, nil } + ms = enginepb.MVCCStats{} + // Handle the case of a required one phase commit transaction. if etArg.Require1PC { - // Make sure that there's a pErr returned. if pErr != nil { - return batch, enginepb.MVCCStats{}, nil, result.Result{}, pErr - } - if ba.Timestamp != br.Timestamp { - onePCRes.pErr = roachpb.NewError( - roachpb.NewTransactionRetryError( - roachpb.RETRY_SERIALIZABLE, "Require1PC batch pushed")) - return batch, enginepb.MVCCStats{}, nil, result.Result{}, pErr + return batch, ms, nil, result.Result{}, pErr + } else if ba.Timestamp != br.Timestamp { + err := roachpb.NewTransactionRetryError(roachpb.RETRY_REASON_UNKNOWN, "Require1PC batch pushed") + return batch, ms, nil, result.Result{}, roachpb.NewError(err) } log.Fatal(ctx, "unreachable") } - ms = enginepb.MVCCStats{} - batch.Close() if log.ExpensiveLogEnabled(ctx, 2) { log.VEventf(ctx, 2, @@ -385,27 +328,26 @@ func (r *Replica) evaluateWriteBatch( } rec := NewReplicaEvalContext(r, spans) - batch, br, res, pErr := r.evaluateWriteBatchWithServersideRefreshes( - ctx, idKey, rec, &ms, ba, spans) + // We can retry locally if this is a non-transactional request. + canRetry := ba.Txn == nil + batch, br, res, pErr := r.evaluateWriteBatchWithLocalRetries(ctx, idKey, rec, &ms, ba, spans, canRetry) return batch, ms, br, res, pErr } -// evaluateWriteBatchWithServersideRefreshes invokes evaluateBatch and retries -// at a higher timestamp in the event of some retriable errors if allowed by the -// batch/txn. -func (r *Replica) evaluateWriteBatchWithServersideRefreshes( +// evaluateWriteBatchWithLocalRetries invokes evaluateBatch and +// retries in the event of a WriteTooOldError at a higher timestamp if +// canRetry is true. +func (r *Replica) evaluateWriteBatchWithLocalRetries( ctx context.Context, idKey storagebase.CmdIDKey, rec batcheval.EvalContext, ms *enginepb.MVCCStats, ba *roachpb.BatchRequest, spans *spanset.SpanSet, + canRetry bool, ) (batch engine.Batch, br *roachpb.BatchResponse, res result.Result, pErr *roachpb.Error) { goldenMS := *ms for retries := 0; ; retries++ { - if retries > 0 { - log.VEventf(ctx, 2, "server-side retry of batch") - } if batch != nil { *ms = goldenMS batch.Close() @@ -458,69 +400,27 @@ func (r *Replica) evaluateWriteBatchWithServersideRefreshes( } br, res, pErr = evaluateBatch(ctx, idKey, batch, rec, ms, ba, false /* readOnly */) - if pErr == nil { - if opLogger != nil { - res.LogicalOpLog = &storagepb.LogicalOpLog{ - Ops: opLogger.LogicalOps(), - } + // If we can retry, set a higher batch timestamp and continue. + if wtoErr, ok := pErr.GetDetail().(*roachpb.WriteTooOldError); ok && canRetry { + // Allow one retry only; a non-txn batch containing overlapping + // spans will always experience WriteTooOldError. + if retries == 1 { + break } + log.Infof(ctx, "!!! local retries updating ba.Timestamp %s->%s", ba.Timestamp, wtoErr.ActualTimestamp) + ba.Timestamp = wtoErr.ActualTimestamp + continue } - // If we can retry, set a higher batch timestamp and continue. - // Allow one retry only; a non-txn batch containing overlapping - // spans will always experience WriteTooOldError. - if pErr == nil || retries > 0 || !canDoServersideRetry(ctx, pErr, ba) { - break + if opLogger != nil { + res.LogicalOpLog = &storagepb.LogicalOpLog{ + Ops: opLogger.LogicalOps(), + } } + break } return } -// canDoServersideRetry looks at the error produced by evaluating ba and decides -// if it's possible to retry the batch evaluation at a higher timestamp. -// Retrying is sometimes possible in case of some retriable errors which ask for -// higher timestamps : for transactional requests, retrying is possible if the -// transaction had not performed any prior reads that need refreshing. -// -// If true is returned, ba and ba.Txn will have been updated with the new -// timestamp. -func canDoServersideRetry(ctx context.Context, pErr *roachpb.Error, ba *roachpb.BatchRequest) bool { - var deadline *hlc.Timestamp - if ba.Txn != nil { - // Transaction requests can only be retried if there's an EndTransaction - // telling us that there's been no prior reads in the transaction. - etArg, ok := ba.GetArg(roachpb.EndTxn) - if !ok { - return false - } - et := etArg.(*roachpb.EndTxnRequest) - if !batcheval.CanForwardCommitTimestampWithoutRefresh(ba.Txn, et) { - return false - } - deadline = et.Deadline - } - var newTimestamp hlc.Timestamp - switch tErr := pErr.GetDetail().(type) { - case *roachpb.WriteTooOldError: - newTimestamp = tErr.ActualTimestamp - case *roachpb.TransactionRetryError: - if ba.Txn == nil { - // TODO(andrei): I don't know if TransactionRetryError is possible for - // non-transactional batches, but some tests inject them for 1PC - // transactions. I'm not sure how to deal with them, so let's not retry. - return false - } - newTimestamp = pErr.GetTxn().WriteTimestamp - default: - // TODO(andrei): Handle other retriable errors too. - return false - } - if deadline != nil && deadline.LessEq(newTimestamp) { - return false - } - bumpBatchTimestamp(ctx, ba, newTimestamp) - return true -} - // isOnePhaseCommit returns true iff the BatchRequest contains all writes in the // transaction and ends with an EndTxn. One phase commits are disallowed if any // of the following conditions are true: diff --git a/pkg/storage/store_test.go b/pkg/storage/store_test.go index befb364656a2..846289aaf473 100644 --- a/pkg/storage/store_test.go +++ b/pkg/storage/store_test.go @@ -2511,6 +2511,8 @@ func TestStoreScanMultipleIntents(t *testing.T) { // TestStoreBadRequests verifies that Send returns errors for // bad requests that do not pass key verification. +// +// TODO(kkaneda): Add more test cases. func TestStoreBadRequests(t *testing.T) { defer leaktest.AfterTest(t)() stopper := stop.NewStopper() @@ -2530,7 +2532,8 @@ func TestStoreBadRequests(t *testing.T) { args5 := scanArgs(roachpb.RKeyMin, roachpb.Key("a")) args6 := scanArgs(keys.RangeDescriptorKey(roachpb.RKey(keys.MinKey)), roachpb.Key("a")) - tArgs0, _ := heartbeatArgs(txn, hlc.Timestamp{}) + tArgs0, _ := endTxnArgs(txn, false /* commit */) + tArgs1, _ := heartbeatArgs(txn, hlc.Timestamp{}) tArgs2, tHeader2 := endTxnArgs(txn, false /* commit */) tHeader2.Txn.Key = roachpb.Key(tHeader2.Txn.Key).Next() @@ -2558,23 +2561,22 @@ func TestStoreBadRequests(t *testing.T) { {&args6, nil, "is range-local, but"}, // Txn must be specified in Header. {&tArgs0, nil, "no transaction specified"}, + {&tArgs1, nil, "no transaction specified"}, // Txn key must be same as the request key. {&tArgs2, &tHeader2, "request key .* should match txn key .*"}, {&tArgs3, &tHeader3, "request key .* should match txn key .*"}, {&tArgs4, nil, "request key .* should match pushee"}, } for i, test := range testCases { - t.Run("", func(t *testing.T) { - if test.header == nil { - test.header = &roachpb.Header{} - } - if test.header.Txn != nil { - assignSeqNumsForReqs(test.header.Txn, test.args) - } - if _, pErr := client.SendWrappedWith(context.Background(), store.TestSender(), *test.header, test.args); !testutils.IsPError(pErr, test.err) { - t.Errorf("%d expected error %q, got error %v", i, test.err, pErr) - } - }) + if test.header == nil { + test.header = &roachpb.Header{} + } + if test.header.Txn != nil { + assignSeqNumsForReqs(test.header.Txn, test.args) + } + if _, pErr := client.SendWrappedWith(context.Background(), store.TestSender(), *test.header, test.args); !testutils.IsPError(pErr, test.err) { + t.Errorf("%d expected error %q, got error %v", i, test.err, pErr) + } } } diff --git a/pkg/storage/stores.go b/pkg/storage/stores.go index 8f74fbdf0c6b..b887b232d487 100644 --- a/pkg/storage/stores.go +++ b/pkg/storage/stores.go @@ -163,8 +163,14 @@ func (ls *Stores) GetReplicaForRangeID(rangeID roachpb.RangeID) (*Replica, error func (ls *Stores) Send( ctx context.Context, ba roachpb.BatchRequest, ) (*roachpb.BatchResponse, *roachpb.Error) { - if err := ba.ValidateForEvaluation(); err != nil { - log.Fatalf(ctx, "invalid batch (%s): %s", ba, err) + // To simplify tests, this function used to perform its own range routing if + // the request was missing its range or store IDs. It was too easy to rely on + // this in production code paths, though, so it's now a fatal error if either + // the range or store ID is missing. + if ba.RangeID == 0 { + log.Fatal(ctx, "batch request missing range ID") + } else if ba.Replica.StoreID == 0 { + log.Fatal(ctx, "batch request missing store ID") } store, err := ls.GetStore(ba.Replica.StoreID)