From 7c967abfdaf3fa0300ff98135ef8afd0064d3f52 Mon Sep 17 00:00:00 2001 From: Nathan VanBenschoten Date: Fri, 4 Jan 2019 20:06:28 -0500 Subject: [PATCH 1/3] storage: remove clockless mode support in updateTimestampCache Release note: None --- pkg/storage/replica_tscache.go | 21 +++++---------------- 1 file changed, 5 insertions(+), 16 deletions(-) diff --git a/pkg/storage/replica_tscache.go b/pkg/storage/replica_tscache.go index 564d9d59099b..ea2d5b0a7997 100644 --- a/pkg/storage/replica_tscache.go +++ b/pkg/storage/replica_tscache.go @@ -21,7 +21,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/log" - "github.com/cockroachdb/cockroach/pkg/util/timeutil" "github.com/cockroachdb/cockroach/pkg/util/uuid" ) @@ -31,12 +30,6 @@ import ( func (r *Replica) updateTimestampCache( ba *roachpb.BatchRequest, br *roachpb.BatchResponse, pErr *roachpb.Error, ) { - readOnlyUseReadCache := true - if r.store.Clock().MaxOffset() == timeutil.ClocklessMaxOffset { - // Clockless mode: all reads count as writes. - readOnlyUseReadCache = false - } - tc := r.store.tsCache // Update the timestamp cache using the timestamp at which the batch // was executed. Note this may have moved forward from ba.Timestamp, @@ -82,7 +75,7 @@ func (r *Replica) updateTimestampCache( continue } } - tc.Add(start, end, ts, txnID, readOnlyUseReadCache) + tc.Add(start, end, ts, txnID, true /* readCache */) case *roachpb.ScanRequest: resp := br.Responses[i].GetInner().(*roachpb.ScanResponse) if resp.ResumeSpan != nil { @@ -91,7 +84,7 @@ func (r *Replica) updateTimestampCache( // end key for the span to update the timestamp cache. end = resp.ResumeSpan.Key } - tc.Add(start, end, ts, txnID, readOnlyUseReadCache) + tc.Add(start, end, ts, txnID, true /* readCache */) case *roachpb.ReverseScanRequest: resp := br.Responses[i].GetInner().(*roachpb.ReverseScanResponse) if resp.ResumeSpan != nil { @@ -101,7 +94,7 @@ func (r *Replica) updateTimestampCache( // the span to update the timestamp cache. start = resp.ResumeSpan.EndKey } - tc.Add(start, end, ts, txnID, readOnlyUseReadCache) + tc.Add(start, end, ts, txnID, true /* readCache */) case *roachpb.QueryIntentRequest: if t.IfMissing == roachpb.QueryIntentRequest_PREVENT { resp := br.Responses[i].GetInner().(*roachpb.QueryIntentResponse) @@ -114,7 +107,7 @@ func (r *Replica) updateTimestampCache( // transaction ID so that we block the intent regardless // of whether it is part of the current batch's transaction // or not. - tc.Add(start, end, t.Txn.Timestamp, uuid.UUID{}, readOnlyUseReadCache) + tc.Add(start, end, t.Txn.Timestamp, uuid.UUID{}, true /* readCache */) } } case *roachpb.RefreshRequest: @@ -122,11 +115,7 @@ func (r *Replica) updateTimestampCache( case *roachpb.RefreshRangeRequest: tc.Add(start, end, ts, txnID, !t.Write /* readCache */) default: - readCache := readOnlyUseReadCache - if roachpb.UpdatesWriteTimestampCache(args) { - readCache = false - } - tc.Add(start, end, ts, txnID, readCache) + tc.Add(start, end, ts, txnID, !roachpb.UpdatesWriteTimestampCache(args)) } } } From 73397b45014a0a4a9c887fa5da097a31a1af0ed1 Mon Sep 17 00:00:00 2001 From: Nathan VanBenschoten Date: Mon, 7 Jan 2019 19:22:39 -0500 Subject: [PATCH 2/3] txnwait: don't reset pusheeTxnTimer when pusher is updated This shouldn't cause any issues with a functioning clock, but it means that a repeatedly updated pusher can result in a large number of timer resets. In tests without well functioning clocks, this could result in starvation because `queryPusherCh` could repeatedly broadcast updates (which it does every `maxWaitForQueryTxn`) and the timer could continuously be reset without ever making progress. Release note: None --- pkg/storage/txnwait/txnqueue.go | 28 +++++++++++----------------- 1 file changed, 11 insertions(+), 17 deletions(-) diff --git a/pkg/storage/txnwait/txnqueue.go b/pkg/storage/txnwait/txnqueue.go index cfad2e0f17a1..dfd98ba137ba 100644 --- a/pkg/storage/txnwait/txnqueue.go +++ b/pkg/storage/txnwait/txnqueue.go @@ -460,27 +460,17 @@ func (q *Queue) MaybeWaitForPush( } }() } - var pusheeTxnTimer timeutil.Timer - defer pusheeTxnTimer.Stop() pusherPriority := req.PusherTxn.Priority pusheePriority := req.PusheeTxn.Priority - first := true + var pusheeTxnTimer timeutil.Timer + defer pusheeTxnTimer.Stop() + // The first time we want to check the pushee's txn record immediately: + // the pushee might be gone by the time the pusher gets here if it cleaned + // itself up after the pusher saw an intent but before it entered this + // queue. + pusheeTxnTimer.Reset(0) for { - // Set the timer to check for the pushee txn's expiration. - if !first { - expiration := TxnExpiration(pending.txn.Load().(*roachpb.Transaction)).GoTime() - now := q.store.Clock().Now().GoTime() - pusheeTxnTimer.Reset(expiration.Sub(now)) - } else { - // The first time we want to check the pushee's txn record immediately: - // the pushee might be gone by the time the pusher gets here if it cleaned - // itself up after the pusher saw an intent but before it entered this - // queue. - pusheeTxnTimer.Reset(0) - first = false - } - select { case <-ctx.Done(): // Caller has given up. @@ -530,6 +520,10 @@ func (q *Queue) MaybeWaitForPush( log.VEventf(ctx, 1, "pushing expired txn %s", req.PusheeTxn.ID.Short()) return nil, nil } + // Set the timer to check for the pushee txn's expiration. + expiration := TxnExpiration(updatedPushee).GoTime() + now := q.store.Clock().Now().GoTime() + pusheeTxnTimer.Reset(expiration.Sub(now)) case updatedPusher := <-queryPusherCh: switch updatedPusher.Status { From 3ac97e67880c4fc8bae5526fd90f90afc72b283b Mon Sep 17 00:00:00 2001 From: Nathan VanBenschoten Date: Fri, 4 Jan 2019 23:02:13 -0500 Subject: [PATCH 3/3] storage: tolerate missing transaction records when pushing Informs #25437. Informs #32971. This is the second part of addressing #32971. The final part will be updating the txn client to stop sending BeginTxn requests. The change closely resembles what is laid out in the corresponding RFC sections (`PushTxn` and `QueryTxn / txnwait.Queue`). Both `PushTxn` requests and `QueryTxn` requests are adjusted to tolerate missing transaction records and to synthesize them based on the information pulled from intents if necessary. By hiding these details behind the request API abstraction, we don't actually need to modify the `txnwait.Queue` at all! The change does diverge from the RFC in two distinct ways. The first is that it introduces a new invariant about transaction record creation. Transaction can now only ever be created by requests originating from their own coordinator (`BeginTxn`, `HeartbeatTxn`, and `EndTxn`). They can never be created in any state (not even ABORTED) by concurrent actors. This is actually a much stronger invariant than what previously existed (and even what was proposed in the RFC), and it simplifies a lot of logic by dramatically refining the transaction state machine. We now know that for a transaction record to exist, it must have been created by its own coordinator and it must have checked with `CanCreateTxnRecord`. Making this new invariant work required the second major divergence from the RFC. The RFC suggested that we use the read timestamp cache for successful transaction timestamp pushes before a transaction record was written. This PR takes this a step further and uses the write timestamp cache for successful transaction aborts before a transaction record was written. In doing this, we emerge with a very sane timestamp cache policy - the read timestamp cache is used to push transaction timestamps and the write timestamp cache is used to abort them entirely (which it was already essentially being used for). The txnID marker on these timestamp cache entries then becomes the transaction that initiated the push/abort. Transactions then consult both of these sources before creating their transaction record in `CanCreateTxnRecord`. In doing this, we avoid an entire class of situations where concurrent transactions created and abandoned transaction records for another transaction, requiring eventual GC. Release note: None --- .../RFCS/20181209_lazy_txn_record_creation.md | 2 +- pkg/roachpb/api.go | 9 +- pkg/roachpb/api.pb.go | 240 +++++----- pkg/roachpb/api.proto | 4 +- pkg/roachpb/errors.pb.go | 102 ++--- pkg/roachpb/errors.proto | 22 +- .../batcheval/cmd_begin_transaction.go | 4 +- pkg/storage/batcheval/cmd_end_transaction.go | 16 +- pkg/storage/batcheval/cmd_heartbeat_txn.go | 4 +- pkg/storage/batcheval/cmd_push_txn.go | 214 +++++---- pkg/storage/batcheval/cmd_query_txn.go | 13 +- .../batcheval/cmd_resolve_intent_test.go | 4 +- pkg/storage/batcheval/eval_context.go | 17 +- pkg/storage/batcheval/transaction.go | 59 +++ pkg/storage/gc_queue_test.go | 3 - pkg/storage/replica_eval_context_span.go | 16 +- pkg/storage/replica_test.go | 412 ++++++++++-------- pkg/storage/replica_tscache.go | 196 +++++++-- pkg/storage/txnwait/txnqueue.go | 9 +- 19 files changed, 828 insertions(+), 518 deletions(-) diff --git a/docs/RFCS/20181209_lazy_txn_record_creation.md b/docs/RFCS/20181209_lazy_txn_record_creation.md index 1c2540f94b67..34cf39938ceb 100644 --- a/docs/RFCS/20181209_lazy_txn_record_creation.md +++ b/docs/RFCS/20181209_lazy_txn_record_creation.md @@ -1,5 +1,5 @@ - Feature Name: Lazy Transaction Record Creation (a.k.a Deprecate BeginTransaction) -- Status: draft +- Status: in-progress - Start Date: 2018-12-09 - Authors: Nathan VanBenschoten - RFC PR: #32971 diff --git a/pkg/roachpb/api.go b/pkg/roachpb/api.go index f50ffbb3befc..dafdf4ada9dc 100644 --- a/pkg/roachpb/api.go +++ b/pkg/roachpb/api.go @@ -1028,8 +1028,13 @@ func (*AdminChangeReplicasRequest) flags() int { return isAdmin | isAlone } func (*AdminRelocateRangeRequest) flags() int { return isAdmin | isAlone } func (*HeartbeatTxnRequest) flags() int { return isWrite | isTxn } func (*GCRequest) flags() int { return isWrite | isRange } -func (*PushTxnRequest) flags() int { return isWrite | isAlone } -func (*QueryTxnRequest) flags() int { return isRead | isAlone } + +// PushTxnRequest updates the read timestamp cache when pushing a transaction's +// timestamp and updates the write timestamp cache when aborting a transaction. +func (*PushTxnRequest) flags() int { + return isWrite | isAlone | updatesReadTSCache | updatesWriteTSCache +} +func (*QueryTxnRequest) flags() int { return isRead | isAlone } // QueryIntent only updates the read timestamp cache when attempting // to prevent an intent that is found missing from ever being written diff --git a/pkg/roachpb/api.pb.go b/pkg/roachpb/api.pb.go index 3be1b7c32ee6..280291eb500c 100644 --- a/pkg/roachpb/api.pb.go +++ b/pkg/roachpb/api.pb.go @@ -69,7 +69,7 @@ func (x ReadConsistencyType) String() string { return proto.EnumName(ReadConsistencyType_name, int32(x)) } func (ReadConsistencyType) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_72f8be059f5b2203, []int{0} + return fileDescriptor_api_a47949baa0439bdd, []int{0} } // ScanFormat is an enumeration of the available response formats for MVCCScan @@ -97,7 +97,7 @@ func (x ScanFormat) String() string { return proto.EnumName(ScanFormat_name, int32(x)) } func (ScanFormat) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_72f8be059f5b2203, []int{1} + return fileDescriptor_api_a47949baa0439bdd, []int{1} } // PushTxnType determines what action to take when pushing a transaction. @@ -128,7 +128,7 @@ func (x PushTxnType) String() string { return proto.EnumName(PushTxnType_name, int32(x)) } func (PushTxnType) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_72f8be059f5b2203, []int{2} + return fileDescriptor_api_a47949baa0439bdd, []int{2} } type ExportStorageProvider int32 @@ -166,7 +166,7 @@ func (x ExportStorageProvider) String() string { return proto.EnumName(ExportStorageProvider_name, int32(x)) } func (ExportStorageProvider) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_72f8be059f5b2203, []int{3} + return fileDescriptor_api_a47949baa0439bdd, []int{3} } type MVCCFilter int32 @@ -189,7 +189,7 @@ func (x MVCCFilter) String() string { return proto.EnumName(MVCCFilter_name, int32(x)) } func (MVCCFilter) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_72f8be059f5b2203, []int{4} + return fileDescriptor_api_a47949baa0439bdd, []int{4} } type ResponseHeader_ResumeReason int32 @@ -221,7 +221,7 @@ func (x ResponseHeader_ResumeReason) String() string { return proto.EnumName(ResponseHeader_ResumeReason_name, int32(x)) } func (ResponseHeader_ResumeReason) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_72f8be059f5b2203, []int{2, 0} + return fileDescriptor_api_a47949baa0439bdd, []int{2, 0} } type QueryIntentRequest_IfMissingBehavior int32 @@ -257,7 +257,7 @@ func (x QueryIntentRequest_IfMissingBehavior) String() string { return proto.EnumName(QueryIntentRequest_IfMissingBehavior_name, int32(x)) } func (QueryIntentRequest_IfMissingBehavior) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_72f8be059f5b2203, []int{50, 0} + return fileDescriptor_api_a47949baa0439bdd, []int{50, 0} } // RangeInfo describes a range which executed a request. It contains @@ -273,7 +273,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_72f8be059f5b2203, []int{0} + return fileDescriptor_api_a47949baa0439bdd, []int{0} } func (m *RangeInfo) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -318,7 +318,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_72f8be059f5b2203, []int{1} + return fileDescriptor_api_a47949baa0439bdd, []int{1} } func (m *RequestHeader) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -378,7 +378,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_72f8be059f5b2203, []int{2} + return fileDescriptor_api_a47949baa0439bdd, []int{2} } func (m *ResponseHeader) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -414,7 +414,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_72f8be059f5b2203, []int{3} + return fileDescriptor_api_a47949baa0439bdd, []int{3} } func (m *GetRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -454,7 +454,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_72f8be059f5b2203, []int{4} + return fileDescriptor_api_a47949baa0439bdd, []int{4} } func (m *GetResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -499,7 +499,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_72f8be059f5b2203, []int{5} + return fileDescriptor_api_a47949baa0439bdd, []int{5} } func (m *PutRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -535,7 +535,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_72f8be059f5b2203, []int{6} + return fileDescriptor_api_a47949baa0439bdd, []int{6} } func (m *PutResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -586,7 +586,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_72f8be059f5b2203, []int{7} + return fileDescriptor_api_a47949baa0439bdd, []int{7} } func (m *ConditionalPutRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -623,7 +623,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_72f8be059f5b2203, []int{8} + return fileDescriptor_api_a47949baa0439bdd, []int{8} } func (m *ConditionalPutResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -671,7 +671,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_72f8be059f5b2203, []int{9} + return fileDescriptor_api_a47949baa0439bdd, []int{9} } func (m *InitPutRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -707,7 +707,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_72f8be059f5b2203, []int{10} + return fileDescriptor_api_a47949baa0439bdd, []int{10} } func (m *InitPutResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -749,7 +749,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_72f8be059f5b2203, []int{11} + return fileDescriptor_api_a47949baa0439bdd, []int{11} } func (m *IncrementRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -788,7 +788,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_72f8be059f5b2203, []int{12} + return fileDescriptor_api_a47949baa0439bdd, []int{12} } func (m *IncrementResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -824,7 +824,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_72f8be059f5b2203, []int{13} + return fileDescriptor_api_a47949baa0439bdd, []int{13} } func (m *DeleteRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -860,7 +860,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_72f8be059f5b2203, []int{14} + return fileDescriptor_api_a47949baa0439bdd, []int{14} } func (m *DeleteResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -911,7 +911,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_72f8be059f5b2203, []int{15} + return fileDescriptor_api_a47949baa0439bdd, []int{15} } func (m *DeleteRangeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -950,7 +950,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_72f8be059f5b2203, []int{16} + return fileDescriptor_api_a47949baa0439bdd, []int{16} } func (m *DeleteRangeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -999,7 +999,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_72f8be059f5b2203, []int{17} + return fileDescriptor_api_a47949baa0439bdd, []int{17} } func (m *ClearRangeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1035,7 +1035,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_72f8be059f5b2203, []int{18} + return fileDescriptor_api_a47949baa0439bdd, []int{18} } func (m *ClearRangeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1106,7 +1106,7 @@ func (m *ScanOptions) Reset() { *m = ScanOptions{} } func (m *ScanOptions) String() string { return proto.CompactTextString(m) } func (*ScanOptions) ProtoMessage() {} func (*ScanOptions) Descriptor() ([]byte, []int) { - return fileDescriptor_api_72f8be059f5b2203, []int{19} + return fileDescriptor_api_a47949baa0439bdd, []int{19} } func (m *ScanOptions) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1148,7 +1148,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_72f8be059f5b2203, []int{20} + return fileDescriptor_api_a47949baa0439bdd, []int{20} } func (m *ScanRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1197,7 +1197,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_72f8be059f5b2203, []int{21} + return fileDescriptor_api_a47949baa0439bdd, []int{21} } func (m *ScanResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1239,7 +1239,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_72f8be059f5b2203, []int{22} + return fileDescriptor_api_a47949baa0439bdd, []int{22} } func (m *ReverseScanRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1288,7 +1288,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_72f8be059f5b2203, []int{23} + return fileDescriptor_api_a47949baa0439bdd, []int{23} } func (m *ReverseScanResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1329,7 +1329,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_72f8be059f5b2203, []int{24} + return fileDescriptor_api_a47949baa0439bdd, []int{24} } func (m *CheckConsistencyRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1366,7 +1366,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_72f8be059f5b2203, []int{25} + return fileDescriptor_api_a47949baa0439bdd, []int{25} } func (m *CheckConsistencyResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1416,7 +1416,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_72f8be059f5b2203, []int{26} + return fileDescriptor_api_a47949baa0439bdd, []int{26} } func (m *RecomputeStatsRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1454,7 +1454,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_72f8be059f5b2203, []int{27} + return fileDescriptor_api_a47949baa0439bdd, []int{27} } func (m *RecomputeStatsResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1490,7 +1490,7 @@ func (m *BeginTransactionRequest) Reset() { *m = BeginTransactionRequest func (m *BeginTransactionRequest) String() string { return proto.CompactTextString(m) } func (*BeginTransactionRequest) ProtoMessage() {} func (*BeginTransactionRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_72f8be059f5b2203, []int{28} + return fileDescriptor_api_a47949baa0439bdd, []int{28} } func (m *BeginTransactionRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1526,7 +1526,7 @@ func (m *BeginTransactionResponse) Reset() { *m = BeginTransactionRespon func (m *BeginTransactionResponse) String() string { return proto.CompactTextString(m) } func (*BeginTransactionResponse) ProtoMessage() {} func (*BeginTransactionResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_72f8be059f5b2203, []int{29} + return fileDescriptor_api_a47949baa0439bdd, []int{29} } func (m *BeginTransactionResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1600,7 +1600,7 @@ func (m *EndTransactionRequest) Reset() { *m = EndTransactionRequest{} } func (m *EndTransactionRequest) String() string { return proto.CompactTextString(m) } func (*EndTransactionRequest) ProtoMessage() {} func (*EndTransactionRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_72f8be059f5b2203, []int{30} + return fileDescriptor_api_a47949baa0439bdd, []int{30} } func (m *EndTransactionRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1646,7 +1646,7 @@ func (m *EndTransactionResponse) Reset() { *m = EndTransactionResponse{} func (m *EndTransactionResponse) String() string { return proto.CompactTextString(m) } func (*EndTransactionResponse) ProtoMessage() {} func (*EndTransactionResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_72f8be059f5b2203, []int{31} + return fileDescriptor_api_a47949baa0439bdd, []int{31} } func (m *EndTransactionResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1702,7 +1702,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_72f8be059f5b2203, []int{32} + return fileDescriptor_api_a47949baa0439bdd, []int{32} } func (m *AdminSplitRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1739,7 +1739,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_72f8be059f5b2203, []int{33} + return fileDescriptor_api_a47949baa0439bdd, []int{33} } func (m *AdminSplitResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1783,7 +1783,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_72f8be059f5b2203, []int{34} + return fileDescriptor_api_a47949baa0439bdd, []int{34} } func (m *AdminMergeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1820,7 +1820,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_72f8be059f5b2203, []int{35} + return fileDescriptor_api_a47949baa0439bdd, []int{35} } func (m *AdminMergeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1860,7 +1860,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_72f8be059f5b2203, []int{36} + return fileDescriptor_api_a47949baa0439bdd, []int{36} } func (m *AdminTransferLeaseRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1895,7 +1895,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_72f8be059f5b2203, []int{37} + return fileDescriptor_api_a47949baa0439bdd, []int{37} } func (m *AdminTransferLeaseResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1935,7 +1935,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_72f8be059f5b2203, []int{38} + return fileDescriptor_api_a47949baa0439bdd, []int{38} } func (m *AdminChangeReplicasRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1970,7 +1970,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_72f8be059f5b2203, []int{39} + return fileDescriptor_api_a47949baa0439bdd, []int{39} } func (m *AdminChangeReplicasResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2009,7 +2009,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_72f8be059f5b2203, []int{40} + return fileDescriptor_api_a47949baa0439bdd, []int{40} } func (m *AdminRelocateRangeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2044,7 +2044,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_72f8be059f5b2203, []int{41} + return fileDescriptor_api_a47949baa0439bdd, []int{41} } func (m *AdminRelocateRangeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2085,7 +2085,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_72f8be059f5b2203, []int{42} + return fileDescriptor_api_a47949baa0439bdd, []int{42} } func (m *HeartbeatTxnRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2124,7 +2124,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_72f8be059f5b2203, []int{43} + return fileDescriptor_api_a47949baa0439bdd, []int{43} } func (m *HeartbeatTxnResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2167,7 +2167,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_72f8be059f5b2203, []int{44} + return fileDescriptor_api_a47949baa0439bdd, []int{44} } func (m *GCRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2203,7 +2203,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_72f8be059f5b2203, []int{44, 0} + return fileDescriptor_api_a47949baa0439bdd, []int{44, 0} } func (m *GCRequest_GCKey) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2239,7 +2239,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_72f8be059f5b2203, []int{45} + return fileDescriptor_api_a47949baa0439bdd, []int{45} } func (m *GCResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2304,8 +2304,8 @@ type PushTxnRequest struct { // this to PUSH_TOUCH to determine whether the pushee can be aborted // due to inactivity (based on the now field). PushType PushTxnType `protobuf:"varint,6,opt,name=push_type,json=pushType,proto3,enum=cockroach.roachpb.PushTxnType" json:"push_type,omitempty"` - // Forces the push by overriding the normal checks in PushTxn to - // either abort or push the timestamp. + // Forces the push by overriding the normal expiration and priority checks + // in PushTxn to either abort or push the timestamp. Force bool `protobuf:"varint,7,opt,name=force,proto3" json:"force,omitempty"` XXX_NoUnkeyedLiteral struct{} `json:"-"` XXX_sizecache int32 `json:"-"` @@ -2315,7 +2315,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_72f8be059f5b2203, []int{46} + return fileDescriptor_api_a47949baa0439bdd, []int{46} } func (m *PushTxnRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2360,7 +2360,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_72f8be059f5b2203, []int{47} + return fileDescriptor_api_a47949baa0439bdd, []int{47} } func (m *PushTxnResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2406,7 +2406,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_72f8be059f5b2203, []int{48} + return fileDescriptor_api_a47949baa0439bdd, []int{48} } func (m *QueryTxnRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2447,7 +2447,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_72f8be059f5b2203, []int{49} + return fileDescriptor_api_a47949baa0439bdd, []int{49} } func (m *QueryTxnResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2509,7 +2509,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_72f8be059f5b2203, []int{50} + return fileDescriptor_api_a47949baa0439bdd, []int{50} } func (m *QueryIntentRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2547,7 +2547,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_72f8be059f5b2203, []int{51} + return fileDescriptor_api_a47949baa0439bdd, []int{51} } func (m *QueryIntentResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2593,7 +2593,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_72f8be059f5b2203, []int{52} + return fileDescriptor_api_a47949baa0439bdd, []int{52} } func (m *ResolveIntentRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2630,7 +2630,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_72f8be059f5b2203, []int{53} + return fileDescriptor_api_a47949baa0439bdd, []int{53} } func (m *ResolveIntentResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2680,7 +2680,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_72f8be059f5b2203, []int{54} + return fileDescriptor_api_a47949baa0439bdd, []int{54} } func (m *ResolveIntentRangeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2717,7 +2717,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_72f8be059f5b2203, []int{55} + return fileDescriptor_api_a47949baa0439bdd, []int{55} } func (m *ResolveIntentRangeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2756,7 +2756,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_72f8be059f5b2203, []int{56} + return fileDescriptor_api_a47949baa0439bdd, []int{56} } func (m *MergeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2792,7 +2792,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_72f8be059f5b2203, []int{57} + return fileDescriptor_api_a47949baa0439bdd, []int{57} } func (m *MergeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2839,7 +2839,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_72f8be059f5b2203, []int{58} + return fileDescriptor_api_a47949baa0439bdd, []int{58} } func (m *TruncateLogRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2875,7 +2875,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_72f8be059f5b2203, []int{59} + return fileDescriptor_api_a47949baa0439bdd, []int{59} } func (m *TruncateLogResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2921,7 +2921,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_72f8be059f5b2203, []int{60} + return fileDescriptor_api_a47949baa0439bdd, []int{60} } func (m *RequestLeaseRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2972,7 +2972,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_72f8be059f5b2203, []int{61} + return fileDescriptor_api_a47949baa0439bdd, []int{61} } func (m *TransferLeaseRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3011,7 +3011,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_72f8be059f5b2203, []int{62} + return fileDescriptor_api_a47949baa0439bdd, []int{62} } func (m *LeaseInfoRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3050,7 +3050,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_72f8be059f5b2203, []int{63} + return fileDescriptor_api_a47949baa0439bdd, []int{63} } func (m *LeaseInfoResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3087,7 +3087,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_72f8be059f5b2203, []int{64} + return fileDescriptor_api_a47949baa0439bdd, []int{64} } func (m *RequestLeaseResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3132,7 +3132,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_72f8be059f5b2203, []int{65} + return fileDescriptor_api_a47949baa0439bdd, []int{65} } func (m *ComputeChecksumRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3171,7 +3171,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_72f8be059f5b2203, []int{66} + return fileDescriptor_api_a47949baa0439bdd, []int{66} } func (m *ComputeChecksumResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3212,7 +3212,7 @@ func (m *ExportStorage) Reset() { *m = ExportStorage{} } func (m *ExportStorage) String() string { return proto.CompactTextString(m) } func (*ExportStorage) ProtoMessage() {} func (*ExportStorage) Descriptor() ([]byte, []int) { - return fileDescriptor_api_72f8be059f5b2203, []int{67} + return fileDescriptor_api_a47949baa0439bdd, []int{67} } func (m *ExportStorage) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3247,7 +3247,7 @@ func (m *ExportStorage_LocalFilePath) Reset() { *m = ExportStorage_Local func (m *ExportStorage_LocalFilePath) String() string { return proto.CompactTextString(m) } func (*ExportStorage_LocalFilePath) ProtoMessage() {} func (*ExportStorage_LocalFilePath) Descriptor() ([]byte, []int) { - return fileDescriptor_api_72f8be059f5b2203, []int{67, 0} + return fileDescriptor_api_a47949baa0439bdd, []int{67, 0} } func (m *ExportStorage_LocalFilePath) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3282,7 +3282,7 @@ func (m *ExportStorage_Http) Reset() { *m = ExportStorage_Http{} } func (m *ExportStorage_Http) String() string { return proto.CompactTextString(m) } func (*ExportStorage_Http) ProtoMessage() {} func (*ExportStorage_Http) Descriptor() ([]byte, []int) { - return fileDescriptor_api_72f8be059f5b2203, []int{67, 1} + return fileDescriptor_api_a47949baa0439bdd, []int{67, 1} } func (m *ExportStorage_Http) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3323,7 +3323,7 @@ func (m *ExportStorage_S3) Reset() { *m = ExportStorage_S3{} } func (m *ExportStorage_S3) String() string { return proto.CompactTextString(m) } func (*ExportStorage_S3) ProtoMessage() {} func (*ExportStorage_S3) Descriptor() ([]byte, []int) { - return fileDescriptor_api_72f8be059f5b2203, []int{67, 2} + return fileDescriptor_api_a47949baa0439bdd, []int{67, 2} } func (m *ExportStorage_S3) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3364,7 +3364,7 @@ func (m *ExportStorage_GCS) Reset() { *m = ExportStorage_GCS{} } func (m *ExportStorage_GCS) String() string { return proto.CompactTextString(m) } func (*ExportStorage_GCS) ProtoMessage() {} func (*ExportStorage_GCS) Descriptor() ([]byte, []int) { - return fileDescriptor_api_72f8be059f5b2203, []int{67, 3} + return fileDescriptor_api_a47949baa0439bdd, []int{67, 3} } func (m *ExportStorage_GCS) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3402,7 +3402,7 @@ func (m *ExportStorage_Azure) Reset() { *m = ExportStorage_Azure{} } func (m *ExportStorage_Azure) String() string { return proto.CompactTextString(m) } func (*ExportStorage_Azure) ProtoMessage() {} func (*ExportStorage_Azure) Descriptor() ([]byte, []int) { - return fileDescriptor_api_72f8be059f5b2203, []int{67, 4} + return fileDescriptor_api_a47949baa0439bdd, []int{67, 4} } func (m *ExportStorage_Azure) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3443,7 +3443,7 @@ func (m *ExportStorage_Workload) Reset() { *m = ExportStorage_Workload{} func (m *ExportStorage_Workload) String() string { return proto.CompactTextString(m) } func (*ExportStorage_Workload) ProtoMessage() {} func (*ExportStorage_Workload) Descriptor() ([]byte, []int) { - return fileDescriptor_api_72f8be059f5b2203, []int{67, 5} + return fileDescriptor_api_a47949baa0439bdd, []int{67, 5} } func (m *ExportStorage_Workload) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3485,7 +3485,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_72f8be059f5b2203, []int{68} + return fileDescriptor_api_a47949baa0439bdd, []int{68} } func (m *WriteBatchRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3521,7 +3521,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_72f8be059f5b2203, []int{69} + return fileDescriptor_api_a47949baa0439bdd, []int{69} } func (m *WriteBatchResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3568,7 +3568,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_72f8be059f5b2203, []int{70} + return fileDescriptor_api_a47949baa0439bdd, []int{70} } func (m *ExportRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3606,7 +3606,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_72f8be059f5b2203, []int{71} + return fileDescriptor_api_a47949baa0439bdd, []int{71} } func (m *BulkOpSummary) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3644,7 +3644,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_72f8be059f5b2203, []int{72} + return fileDescriptor_api_a47949baa0439bdd, []int{72} } func (m *ExportResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3685,7 +3685,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_72f8be059f5b2203, []int{72, 0} + return fileDescriptor_api_a47949baa0439bdd, []int{72, 0} } func (m *ExportResponse_File) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3737,7 +3737,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_72f8be059f5b2203, []int{73} + return fileDescriptor_api_a47949baa0439bdd, []int{73} } func (m *ImportRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3774,7 +3774,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_72f8be059f5b2203, []int{73, 0} + return fileDescriptor_api_a47949baa0439bdd, []int{73, 0} } func (m *ImportRequest_File) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3812,7 +3812,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_72f8be059f5b2203, []int{73, 1} + return fileDescriptor_api_a47949baa0439bdd, []int{73, 1} } func (m *ImportRequest_TableRekey) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3849,7 +3849,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_72f8be059f5b2203, []int{74} + return fileDescriptor_api_a47949baa0439bdd, []int{74} } func (m *ImportResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3889,7 +3889,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_72f8be059f5b2203, []int{75} + return fileDescriptor_api_a47949baa0439bdd, []int{75} } func (m *AdminScatterRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3926,7 +3926,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_72f8be059f5b2203, []int{76} + return fileDescriptor_api_a47949baa0439bdd, []int{76} } func (m *AdminScatterResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3961,7 +3961,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_72f8be059f5b2203, []int{76, 0} + return fileDescriptor_api_a47949baa0439bdd, []int{76, 0} } func (m *AdminScatterResponse_Range) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3999,7 +3999,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_72f8be059f5b2203, []int{77} + return fileDescriptor_api_a47949baa0439bdd, []int{77} } func (m *AddSSTableRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4035,7 +4035,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_72f8be059f5b2203, []int{78} + return fileDescriptor_api_a47949baa0439bdd, []int{78} } func (m *AddSSTableResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4081,7 +4081,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_72f8be059f5b2203, []int{79} + return fileDescriptor_api_a47949baa0439bdd, []int{79} } func (m *RefreshRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4117,7 +4117,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_72f8be059f5b2203, []int{80} + return fileDescriptor_api_a47949baa0439bdd, []int{80} } func (m *RefreshResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4159,7 +4159,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_72f8be059f5b2203, []int{81} + return fileDescriptor_api_a47949baa0439bdd, []int{81} } func (m *RefreshRangeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4195,7 +4195,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_72f8be059f5b2203, []int{82} + return fileDescriptor_api_a47949baa0439bdd, []int{82} } func (m *RefreshRangeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4243,7 +4243,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_72f8be059f5b2203, []int{83} + return fileDescriptor_api_a47949baa0439bdd, []int{83} } func (m *SubsumeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4290,7 +4290,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_72f8be059f5b2203, []int{84} + return fileDescriptor_api_a47949baa0439bdd, []int{84} } func (m *SubsumeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4327,7 +4327,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_72f8be059f5b2203, []int{85} + return fileDescriptor_api_a47949baa0439bdd, []int{85} } func (m *RangeStatsRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4368,7 +4368,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_72f8be059f5b2203, []int{86} + return fileDescriptor_api_a47949baa0439bdd, []int{86} } func (m *RangeStatsResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4450,7 +4450,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_72f8be059f5b2203, []int{87} + return fileDescriptor_api_a47949baa0439bdd, []int{87} } func (m *RequestUnion) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -5809,7 +5809,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_72f8be059f5b2203, []int{88} + return fileDescriptor_api_a47949baa0439bdd, []int{88} } func (m *ResponseUnion) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -7158,7 +7158,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_72f8be059f5b2203, []int{89} + return fileDescriptor_api_a47949baa0439bdd, []int{89} } func (m *Header) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -7196,7 +7196,7 @@ type BatchRequest struct { func (m *BatchRequest) Reset() { *m = BatchRequest{} } func (*BatchRequest) ProtoMessage() {} func (*BatchRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_72f8be059f5b2203, []int{90} + return fileDescriptor_api_a47949baa0439bdd, []int{90} } func (m *BatchRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -7235,7 +7235,7 @@ type BatchResponse struct { func (m *BatchResponse) Reset() { *m = BatchResponse{} } func (*BatchResponse) ProtoMessage() {} func (*BatchResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_72f8be059f5b2203, []int{91} + return fileDescriptor_api_a47949baa0439bdd, []int{91} } func (m *BatchResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -7296,7 +7296,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_72f8be059f5b2203, []int{91, 0} + return fileDescriptor_api_a47949baa0439bdd, []int{91, 0} } func (m *BatchResponse_Header) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -7334,7 +7334,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_72f8be059f5b2203, []int{92} + return fileDescriptor_api_a47949baa0439bdd, []int{92} } func (m *RangeFeedRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -7372,7 +7372,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_72f8be059f5b2203, []int{93} + return fileDescriptor_api_a47949baa0439bdd, []int{93} } func (m *RangeFeedValue) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -7412,7 +7412,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_72f8be059f5b2203, []int{94} + return fileDescriptor_api_a47949baa0439bdd, []int{94} } func (m *RangeFeedCheckpoint) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -7451,7 +7451,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_72f8be059f5b2203, []int{95} + return fileDescriptor_api_a47949baa0439bdd, []int{95} } func (m *RangeFeedError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -7490,7 +7490,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_72f8be059f5b2203, []int{96} + return fileDescriptor_api_a47949baa0439bdd, []int{96} } func (m *RangeFeedEvent) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -33470,9 +33470,9 @@ var ( ErrIntOverflowApi = fmt.Errorf("proto: integer overflow") ) -func init() { proto.RegisterFile("roachpb/api.proto", fileDescriptor_api_72f8be059f5b2203) } +func init() { proto.RegisterFile("roachpb/api.proto", fileDescriptor_api_a47949baa0439bdd) } -var fileDescriptor_api_72f8be059f5b2203 = []byte{ +var fileDescriptor_api_a47949baa0439bdd = []byte{ // 6156 bytes of a gzipped FileDescriptorProto 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xe4, 0x7d, 0x5b, 0x6c, 0x1c, 0xc9, 0x75, 0x28, 0x7b, 0x66, 0x48, 0xce, 0x9c, 0x79, 0xb0, 0x59, 0xd4, 0x63, 0x44, 0x69, 0x45, 0x6a, diff --git a/pkg/roachpb/api.proto b/pkg/roachpb/api.proto index 994f39858384..ffcbbe1cab01 100644 --- a/pkg/roachpb/api.proto +++ b/pkg/roachpb/api.proto @@ -742,8 +742,8 @@ message PushTxnRequest { // this to PUSH_TOUCH to determine whether the pushee can be aborted // due to inactivity (based on the now field). PushTxnType push_type = 6; - // Forces the push by overriding the normal checks in PushTxn to - // either abort or push the timestamp. + // Forces the push by overriding the normal expiration and priority checks + // in PushTxn to either abort or push the timestamp. bool force = 7; reserved 8; diff --git a/pkg/roachpb/errors.pb.go b/pkg/roachpb/errors.pb.go index 0c0bc03b3187..7260b6447540 100644 --- a/pkg/roachpb/errors.pb.go +++ b/pkg/roachpb/errors.pb.go @@ -35,12 +35,12 @@ type TransactionAbortedReason int32 const ( // For backwards compatibility. ABORT_REASON_UNKNOWN TransactionAbortedReason = 0 - // A BeginTransaction or EndTransaction(commit=true) request found an aborted - // transaction record. Another txn must have written this record - that other - // txn probably ran into one of our intents written before BeginTransaction - // (on a different range) and pushed it successfully. Or, a high-priority - // transaction simply pushed us, or we failed to heartbeat for a while and - // another txn (of any priority) considered us abandoned and pushed us. + // A BeginTransaction, HeartbeatTxn, or EndTransaction(commit=true) request + // found an aborted transaction record. Another txn must have written this + // record - that other txn probably ran into one of our intents and pushed our + // transaction record successfully. Either a high-priority transaction simply + // pushed us or we failed to heartbeat for a while and another txn (of any + // priority) considered us abandoned and pushed us. ABORT_REASON_ABORTED_RECORD_FOUND TransactionAbortedReason = 1 // The request attempting to create a transaction record has a timestamp below // the TxnSpanGCThreshold, so there might have been an ABORTED txn record that @@ -59,11 +59,11 @@ const ( // where it had previously laid down intents that have been cleaned up in the // meantime because the transaction was aborted. ABORT_REASON_ABORT_SPAN TransactionAbortedReason = 5 - // An EndTransaction encountered a timestamp cache entry for the txn key, and - // the entry identifies this transaction. This means that the transaction - // definitely committed or rolled back before. - // So, this EndTransaction is either a delayed replay of some sort, or it - // raced with an async abort and lost. If a client gets this + // A request attempting to create a transaction record encountered a write + // timestamp cache entry for the txn key, and the entry identifies this + // transaction. This means that the transaction definitely committed or rolled + // back before. So, this request is either a delayed replay of some sort, or + // it raced with an async abort and lost. If a client gets this // TransactionAbortedError (without it being wrapped in an ambiguous error), // it must be the latter case, and the transaction can be retried. ABORT_REASON_ALREADY_COMMITTED_OR_ROLLED_BACK_POSSIBLE_REPLAY TransactionAbortedReason = 6 @@ -112,7 +112,7 @@ func (x *TransactionAbortedReason) UnmarshalJSON(data []byte) error { return nil } func (TransactionAbortedReason) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_errors_be1e4db9e00c7ccb, []int{0} + return fileDescriptor_errors_15887ffd51edf29a, []int{0} } // TransactionRetryReason specifies what caused a transaction retry. @@ -165,7 +165,7 @@ func (x *TransactionRetryReason) UnmarshalJSON(data []byte) error { return nil } func (TransactionRetryReason) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_errors_be1e4db9e00c7ccb, []int{1} + return fileDescriptor_errors_15887ffd51edf29a, []int{1} } // TransactionRestart indicates how an error should be handled in a @@ -216,7 +216,7 @@ func (x *TransactionRestart) UnmarshalJSON(data []byte) error { return nil } func (TransactionRestart) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_errors_be1e4db9e00c7ccb, []int{2} + return fileDescriptor_errors_15887ffd51edf29a, []int{2} } // Reason specifies what caused the error. @@ -259,7 +259,7 @@ func (x *TransactionStatusError_Reason) UnmarshalJSON(data []byte) error { return nil } func (TransactionStatusError_Reason) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_errors_be1e4db9e00c7ccb, []int{9, 0} + return fileDescriptor_errors_15887ffd51edf29a, []int{9, 0} } // Reason specifies what caused the error. @@ -310,7 +310,7 @@ func (x *RangeFeedRetryError_Reason) UnmarshalJSON(data []byte) error { return nil } func (RangeFeedRetryError_Reason) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_errors_be1e4db9e00c7ccb, []int{29, 0} + return fileDescriptor_errors_15887ffd51edf29a, []int{29, 0} } // A NotLeaseHolderError indicates that the current range is not the lease @@ -339,7 +339,7 @@ func (m *NotLeaseHolderError) Reset() { *m = NotLeaseHolderError{} } func (m *NotLeaseHolderError) String() string { return proto.CompactTextString(m) } func (*NotLeaseHolderError) ProtoMessage() {} func (*NotLeaseHolderError) Descriptor() ([]byte, []int) { - return fileDescriptor_errors_be1e4db9e00c7ccb, []int{0} + return fileDescriptor_errors_15887ffd51edf29a, []int{0} } func (m *NotLeaseHolderError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -376,7 +376,7 @@ func (m *NodeUnavailableError) Reset() { *m = NodeUnavailableError{} } func (m *NodeUnavailableError) String() string { return proto.CompactTextString(m) } func (*NodeUnavailableError) ProtoMessage() {} func (*NodeUnavailableError) Descriptor() ([]byte, []int) { - return fileDescriptor_errors_be1e4db9e00c7ccb, []int{1} + return fileDescriptor_errors_15887ffd51edf29a, []int{1} } func (m *NodeUnavailableError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -412,7 +412,7 @@ func (m *UnsupportedRequestError) Reset() { *m = UnsupportedRequestError func (m *UnsupportedRequestError) String() string { return proto.CompactTextString(m) } func (*UnsupportedRequestError) ProtoMessage() {} func (*UnsupportedRequestError) Descriptor() ([]byte, []int) { - return fileDescriptor_errors_be1e4db9e00c7ccb, []int{2} + return fileDescriptor_errors_15887ffd51edf29a, []int{2} } func (m *UnsupportedRequestError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -451,7 +451,7 @@ func (m *RangeNotFoundError) Reset() { *m = RangeNotFoundError{} } func (m *RangeNotFoundError) String() string { return proto.CompactTextString(m) } func (*RangeNotFoundError) ProtoMessage() {} func (*RangeNotFoundError) Descriptor() ([]byte, []int) { - return fileDescriptor_errors_be1e4db9e00c7ccb, []int{3} + return fileDescriptor_errors_15887ffd51edf29a, []int{3} } func (m *RangeNotFoundError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -497,7 +497,7 @@ func (m *RangeKeyMismatchError) Reset() { *m = RangeKeyMismatchError{} } func (m *RangeKeyMismatchError) String() string { return proto.CompactTextString(m) } func (*RangeKeyMismatchError) ProtoMessage() {} func (*RangeKeyMismatchError) Descriptor() ([]byte, []int) { - return fileDescriptor_errors_be1e4db9e00c7ccb, []int{4} + return fileDescriptor_errors_15887ffd51edf29a, []int{4} } func (m *RangeKeyMismatchError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -545,7 +545,7 @@ func (m *ReadWithinUncertaintyIntervalError) Reset() { *m = ReadWithinUn func (m *ReadWithinUncertaintyIntervalError) String() string { return proto.CompactTextString(m) } func (*ReadWithinUncertaintyIntervalError) ProtoMessage() {} func (*ReadWithinUncertaintyIntervalError) Descriptor() ([]byte, []int) { - return fileDescriptor_errors_be1e4db9e00c7ccb, []int{5} + return fileDescriptor_errors_15887ffd51edf29a, []int{5} } func (m *ReadWithinUncertaintyIntervalError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -590,7 +590,7 @@ func (m *TransactionAbortedError) Reset() { *m = TransactionAbortedError func (m *TransactionAbortedError) String() string { return proto.CompactTextString(m) } func (*TransactionAbortedError) ProtoMessage() {} func (*TransactionAbortedError) Descriptor() ([]byte, []int) { - return fileDescriptor_errors_be1e4db9e00c7ccb, []int{6} + return fileDescriptor_errors_15887ffd51edf29a, []int{6} } func (m *TransactionAbortedError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -628,7 +628,7 @@ func (m *TransactionPushError) Reset() { *m = TransactionPushError{} } func (m *TransactionPushError) String() string { return proto.CompactTextString(m) } func (*TransactionPushError) ProtoMessage() {} func (*TransactionPushError) Descriptor() ([]byte, []int) { - return fileDescriptor_errors_be1e4db9e00c7ccb, []int{7} + return fileDescriptor_errors_15887ffd51edf29a, []int{7} } func (m *TransactionPushError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -665,7 +665,7 @@ func (m *TransactionRetryError) Reset() { *m = TransactionRetryError{} } func (m *TransactionRetryError) String() string { return proto.CompactTextString(m) } func (*TransactionRetryError) ProtoMessage() {} func (*TransactionRetryError) Descriptor() ([]byte, []int) { - return fileDescriptor_errors_be1e4db9e00c7ccb, []int{8} + return fileDescriptor_errors_15887ffd51edf29a, []int{8} } func (m *TransactionRetryError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -707,7 +707,7 @@ func (m *TransactionStatusError) Reset() { *m = TransactionStatusError{} func (m *TransactionStatusError) String() string { return proto.CompactTextString(m) } func (*TransactionStatusError) ProtoMessage() {} func (*TransactionStatusError) Descriptor() ([]byte, []int) { - return fileDescriptor_errors_be1e4db9e00c7ccb, []int{9} + return fileDescriptor_errors_15887ffd51edf29a, []int{9} } func (m *TransactionStatusError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -747,7 +747,7 @@ func (m *WriteIntentError) Reset() { *m = WriteIntentError{} } func (m *WriteIntentError) String() string { return proto.CompactTextString(m) } func (*WriteIntentError) ProtoMessage() {} func (*WriteIntentError) Descriptor() ([]byte, []int) { - return fileDescriptor_errors_be1e4db9e00c7ccb, []int{10} + return fileDescriptor_errors_15887ffd51edf29a, []int{10} } func (m *WriteIntentError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -787,7 +787,7 @@ func (m *WriteTooOldError) Reset() { *m = WriteTooOldError{} } func (m *WriteTooOldError) String() string { return proto.CompactTextString(m) } func (*WriteTooOldError) ProtoMessage() {} func (*WriteTooOldError) Descriptor() ([]byte, []int) { - return fileDescriptor_errors_be1e4db9e00c7ccb, []int{11} + return fileDescriptor_errors_15887ffd51edf29a, []int{11} } func (m *WriteTooOldError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -825,7 +825,7 @@ func (m *OpRequiresTxnError) Reset() { *m = OpRequiresTxnError{} } func (m *OpRequiresTxnError) String() string { return proto.CompactTextString(m) } func (*OpRequiresTxnError) ProtoMessage() {} func (*OpRequiresTxnError) Descriptor() ([]byte, []int) { - return fileDescriptor_errors_be1e4db9e00c7ccb, []int{12} + return fileDescriptor_errors_15887ffd51edf29a, []int{12} } func (m *OpRequiresTxnError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -864,7 +864,7 @@ func (m *ConditionFailedError) Reset() { *m = ConditionFailedError{} } func (m *ConditionFailedError) String() string { return proto.CompactTextString(m) } func (*ConditionFailedError) ProtoMessage() {} func (*ConditionFailedError) Descriptor() ([]byte, []int) { - return fileDescriptor_errors_be1e4db9e00c7ccb, []int{13} + return fileDescriptor_errors_15887ffd51edf29a, []int{13} } func (m *ConditionFailedError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -903,7 +903,7 @@ func (m *LeaseRejectedError) Reset() { *m = LeaseRejectedError{} } func (m *LeaseRejectedError) String() string { return proto.CompactTextString(m) } func (*LeaseRejectedError) ProtoMessage() {} func (*LeaseRejectedError) Descriptor() ([]byte, []int) { - return fileDescriptor_errors_be1e4db9e00c7ccb, []int{14} + return fileDescriptor_errors_15887ffd51edf29a, []int{14} } func (m *LeaseRejectedError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -940,7 +940,7 @@ func (m *SendError) Reset() { *m = SendError{} } func (m *SendError) String() string { return proto.CompactTextString(m) } func (*SendError) ProtoMessage() {} func (*SendError) Descriptor() ([]byte, []int) { - return fileDescriptor_errors_be1e4db9e00c7ccb, []int{15} + return fileDescriptor_errors_15887ffd51edf29a, []int{15} } func (m *SendError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -980,7 +980,7 @@ func (m *AmbiguousResultError) Reset() { *m = AmbiguousResultError{} } func (m *AmbiguousResultError) String() string { return proto.CompactTextString(m) } func (*AmbiguousResultError) ProtoMessage() {} func (*AmbiguousResultError) Descriptor() ([]byte, []int) { - return fileDescriptor_errors_be1e4db9e00c7ccb, []int{16} + return fileDescriptor_errors_15887ffd51edf29a, []int{16} } func (m *AmbiguousResultError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1016,7 +1016,7 @@ func (m *RaftGroupDeletedError) Reset() { *m = RaftGroupDeletedError{} } func (m *RaftGroupDeletedError) String() string { return proto.CompactTextString(m) } func (*RaftGroupDeletedError) ProtoMessage() {} func (*RaftGroupDeletedError) Descriptor() ([]byte, []int) { - return fileDescriptor_errors_be1e4db9e00c7ccb, []int{17} + return fileDescriptor_errors_15887ffd51edf29a, []int{17} } func (m *RaftGroupDeletedError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1056,7 +1056,7 @@ func (m *ReplicaCorruptionError) Reset() { *m = ReplicaCorruptionError{} func (m *ReplicaCorruptionError) String() string { return proto.CompactTextString(m) } func (*ReplicaCorruptionError) ProtoMessage() {} func (*ReplicaCorruptionError) Descriptor() ([]byte, []int) { - return fileDescriptor_errors_be1e4db9e00c7ccb, []int{18} + return fileDescriptor_errors_15887ffd51edf29a, []int{18} } func (m *ReplicaCorruptionError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1095,7 +1095,7 @@ func (m *ReplicaTooOldError) Reset() { *m = ReplicaTooOldError{} } func (m *ReplicaTooOldError) String() string { return proto.CompactTextString(m) } func (*ReplicaTooOldError) ProtoMessage() {} func (*ReplicaTooOldError) Descriptor() ([]byte, []int) { - return fileDescriptor_errors_be1e4db9e00c7ccb, []int{19} + return fileDescriptor_errors_15887ffd51edf29a, []int{19} } func (m *ReplicaTooOldError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1132,7 +1132,7 @@ func (m *StoreNotFoundError) Reset() { *m = StoreNotFoundError{} } func (m *StoreNotFoundError) String() string { return proto.CompactTextString(m) } func (*StoreNotFoundError) ProtoMessage() {} func (*StoreNotFoundError) Descriptor() ([]byte, []int) { - return fileDescriptor_errors_be1e4db9e00c7ccb, []int{20} + return fileDescriptor_errors_15887ffd51edf29a, []int{20} } func (m *StoreNotFoundError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1180,7 +1180,7 @@ func (m *UnhandledRetryableError) Reset() { *m = UnhandledRetryableError func (m *UnhandledRetryableError) String() string { return proto.CompactTextString(m) } func (*UnhandledRetryableError) ProtoMessage() {} func (*UnhandledRetryableError) Descriptor() ([]byte, []int) { - return fileDescriptor_errors_be1e4db9e00c7ccb, []int{21} + return fileDescriptor_errors_15887ffd51edf29a, []int{21} } func (m *UnhandledRetryableError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1234,7 +1234,7 @@ func (m *TransactionRetryWithProtoRefreshError) Reset() { *m = Transacti func (m *TransactionRetryWithProtoRefreshError) String() string { return proto.CompactTextString(m) } func (*TransactionRetryWithProtoRefreshError) ProtoMessage() {} func (*TransactionRetryWithProtoRefreshError) Descriptor() ([]byte, []int) { - return fileDescriptor_errors_be1e4db9e00c7ccb, []int{22} + return fileDescriptor_errors_15887ffd51edf29a, []int{22} } func (m *TransactionRetryWithProtoRefreshError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1271,7 +1271,7 @@ func (m *TxnAlreadyEncounteredErrorError) Reset() { *m = TxnAlreadyEncou func (m *TxnAlreadyEncounteredErrorError) String() string { return proto.CompactTextString(m) } func (*TxnAlreadyEncounteredErrorError) ProtoMessage() {} func (*TxnAlreadyEncounteredErrorError) Descriptor() ([]byte, []int) { - return fileDescriptor_errors_be1e4db9e00c7ccb, []int{23} + return fileDescriptor_errors_15887ffd51edf29a, []int{23} } func (m *TxnAlreadyEncounteredErrorError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1310,7 +1310,7 @@ func (m *IntegerOverflowError) Reset() { *m = IntegerOverflowError{} } func (m *IntegerOverflowError) String() string { return proto.CompactTextString(m) } func (*IntegerOverflowError) ProtoMessage() {} func (*IntegerOverflowError) Descriptor() ([]byte, []int) { - return fileDescriptor_errors_be1e4db9e00c7ccb, []int{24} + return fileDescriptor_errors_15887ffd51edf29a, []int{24} } func (m *IntegerOverflowError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1348,7 +1348,7 @@ func (m *MixedSuccessError) Reset() { *m = MixedSuccessError{} } func (m *MixedSuccessError) String() string { return proto.CompactTextString(m) } func (*MixedSuccessError) ProtoMessage() {} func (*MixedSuccessError) Descriptor() ([]byte, []int) { - return fileDescriptor_errors_be1e4db9e00c7ccb, []int{25} + return fileDescriptor_errors_15887ffd51edf29a, []int{25} } func (m *MixedSuccessError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1386,7 +1386,7 @@ func (m *BatchTimestampBeforeGCError) Reset() { *m = BatchTimestampBefor func (m *BatchTimestampBeforeGCError) String() string { return proto.CompactTextString(m) } func (*BatchTimestampBeforeGCError) ProtoMessage() {} func (*BatchTimestampBeforeGCError) Descriptor() ([]byte, []int) { - return fileDescriptor_errors_be1e4db9e00c7ccb, []int{26} + return fileDescriptor_errors_15887ffd51edf29a, []int{26} } func (m *BatchTimestampBeforeGCError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1425,7 +1425,7 @@ func (m *IntentMissingError) Reset() { *m = IntentMissingError{} } func (m *IntentMissingError) String() string { return proto.CompactTextString(m) } func (*IntentMissingError) ProtoMessage() {} func (*IntentMissingError) Descriptor() ([]byte, []int) { - return fileDescriptor_errors_be1e4db9e00c7ccb, []int{27} + return fileDescriptor_errors_15887ffd51edf29a, []int{27} } func (m *IntentMissingError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1464,7 +1464,7 @@ func (m *MergeInProgressError) Reset() { *m = MergeInProgressError{} } func (m *MergeInProgressError) String() string { return proto.CompactTextString(m) } func (*MergeInProgressError) ProtoMessage() {} func (*MergeInProgressError) Descriptor() ([]byte, []int) { - return fileDescriptor_errors_be1e4db9e00c7ccb, []int{28} + return fileDescriptor_errors_15887ffd51edf29a, []int{28} } func (m *MergeInProgressError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1501,7 +1501,7 @@ func (m *RangeFeedRetryError) Reset() { *m = RangeFeedRetryError{} } func (m *RangeFeedRetryError) String() string { return proto.CompactTextString(m) } func (*RangeFeedRetryError) ProtoMessage() {} func (*RangeFeedRetryError) Descriptor() ([]byte, []int) { - return fileDescriptor_errors_be1e4db9e00c7ccb, []int{29} + return fileDescriptor_errors_15887ffd51edf29a, []int{29} } func (m *RangeFeedRetryError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1567,7 +1567,7 @@ func (m *ErrorDetail) Reset() { *m = ErrorDetail{} } func (m *ErrorDetail) String() string { return proto.CompactTextString(m) } func (*ErrorDetail) ProtoMessage() {} func (*ErrorDetail) Descriptor() ([]byte, []int) { - return fileDescriptor_errors_be1e4db9e00c7ccb, []int{30} + return fileDescriptor_errors_15887ffd51edf29a, []int{30} } func (m *ErrorDetail) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2527,7 +2527,7 @@ func (m *ErrPosition) Reset() { *m = ErrPosition{} } func (m *ErrPosition) String() string { return proto.CompactTextString(m) } func (*ErrPosition) ProtoMessage() {} func (*ErrPosition) Descriptor() ([]byte, []int) { - return fileDescriptor_errors_be1e4db9e00c7ccb, []int{31} + return fileDescriptor_errors_15887ffd51edf29a, []int{31} } func (m *ErrPosition) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2583,7 +2583,7 @@ type Error struct { func (m *Error) Reset() { *m = Error{} } func (*Error) ProtoMessage() {} func (*Error) Descriptor() ([]byte, []int) { - return fileDescriptor_errors_be1e4db9e00c7ccb, []int{32} + return fileDescriptor_errors_15887ffd51edf29a, []int{32} } func (m *Error) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -10551,9 +10551,9 @@ var ( ErrIntOverflowErrors = fmt.Errorf("proto: integer overflow") ) -func init() { proto.RegisterFile("roachpb/errors.proto", fileDescriptor_errors_be1e4db9e00c7ccb) } +func init() { proto.RegisterFile("roachpb/errors.proto", fileDescriptor_errors_15887ffd51edf29a) } -var fileDescriptor_errors_be1e4db9e00c7ccb = []byte{ +var fileDescriptor_errors_15887ffd51edf29a = []byte{ // 2787 bytes of a gzipped FileDescriptorProto 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x9c, 0x58, 0xcd, 0x73, 0xdb, 0xc6, 0x15, 0x27, 0x24, 0xca, 0x94, 0x9e, 0xbe, 0xa0, 0xb5, 0x22, 0xc3, 0x72, 0x4c, 0xc9, 0x72, 0x9c, diff --git a/pkg/roachpb/errors.proto b/pkg/roachpb/errors.proto index 1a178131fbb7..966750f13285 100644 --- a/pkg/roachpb/errors.proto +++ b/pkg/roachpb/errors.proto @@ -123,12 +123,12 @@ enum TransactionAbortedReason { // For backwards compatibility. ABORT_REASON_UNKNOWN = 0; - // A BeginTransaction or EndTransaction(commit=true) request found an aborted - // transaction record. Another txn must have written this record - that other - // txn probably ran into one of our intents written before BeginTransaction - // (on a different range) and pushed it successfully. Or, a high-priority - // transaction simply pushed us, or we failed to heartbeat for a while and - // another txn (of any priority) considered us abandoned and pushed us. + // A BeginTransaction, HeartbeatTxn, or EndTransaction(commit=true) request + // found an aborted transaction record. Another txn must have written this + // record - that other txn probably ran into one of our intents and pushed our + // transaction record successfully. Either a high-priority transaction simply + // pushed us or we failed to heartbeat for a while and another txn (of any + // priority) considered us abandoned and pushed us. ABORT_REASON_ABORTED_RECORD_FOUND = 1; // The request attempting to create a transaction record has a timestamp below @@ -152,11 +152,11 @@ enum TransactionAbortedReason { // meantime because the transaction was aborted. ABORT_REASON_ABORT_SPAN = 5; - // An EndTransaction encountered a timestamp cache entry for the txn key, and - // the entry identifies this transaction. This means that the transaction - // definitely committed or rolled back before. - // So, this EndTransaction is either a delayed replay of some sort, or it - // raced with an async abort and lost. If a client gets this + // A request attempting to create a transaction record encountered a write + // timestamp cache entry for the txn key, and the entry identifies this + // transaction. This means that the transaction definitely committed or rolled + // back before. So, this request is either a delayed replay of some sort, or + // it raced with an async abort and lost. If a client gets this // TransactionAbortedError (without it being wrapped in an ambiguous error), // it must be the latter case, and the transaction can be retried. ABORT_REASON_ALREADY_COMMITTED_OR_ROLLED_BACK_POSSIBLE_REPLAY = 6; diff --git a/pkg/storage/batcheval/cmd_begin_transaction.go b/pkg/storage/batcheval/cmd_begin_transaction.go index 9799528c6359..561b398e2d6c 100644 --- a/pkg/storage/batcheval/cmd_begin_transaction.go +++ b/pkg/storage/batcheval/cmd_begin_transaction.go @@ -118,8 +118,8 @@ func BeginTransaction( } // Verify that it is safe to create the transaction record. - if ok, reason := cArgs.EvalCtx.CanCreateTxnRecord(reply.Txn); !ok { - return result.Result{}, roachpb.NewTransactionAbortedError(reason) + if err := CanCreateTxnRecord(cArgs.EvalCtx, reply.Txn); err != nil { + return result.Result{}, err } // Write the txn record. diff --git a/pkg/storage/batcheval/cmd_end_transaction.go b/pkg/storage/batcheval/cmd_end_transaction.go index 12590db987bc..1f6d558f901b 100644 --- a/pkg/storage/batcheval/cmd_end_transaction.go +++ b/pkg/storage/batcheval/cmd_end_transaction.go @@ -186,8 +186,8 @@ func evalEndTransaction( // to perform this verification for commits. Rollbacks can always write // an aborted txn record. if args.Commit { - if ok, reason := cArgs.EvalCtx.CanCreateTxnRecord(reply.Txn); !ok { - return result.Result{}, roachpb.NewTransactionAbortedError(reason) + if err := CanCreateTxnRecord(cArgs.EvalCtx, reply.Txn); err != nil { + return result.Result{}, err } } } else { @@ -350,12 +350,12 @@ func evalEndTransaction( // subsequent replay of this EndTransaction call because the txn timestamp // will be too old. Replays of requests which attempt to create a new txn // record (BeginTransaction, HeartbeatTxn, or EndTransaction) never succeed - // because EndTransaction inserts in the write timestamp cache, forcing the - // call to CanCreateTxnRecord to return false, resulting in a transaction - // retry error. If the replay didn't attempt to create a txn record, any - // push will immediately succeed as a missing txn record on push where - // CanCreateTxnRecord returns false succeeds. In both cases, the txn will - // be GC'd on the slow path. + // because EndTransaction inserts in the write timestamp cache in Replica's + // updateTimestampCache method, forcing the call to CanCreateTxnRecord to + // return false, resulting in a transaction retry error. If the replay + // didn't attempt to create a txn record, any push will immediately succeed + // as a missing txn record on push where CanCreateTxnRecord returns false + // succeeds. In both cases, the txn will be GC'd on the slow path. // // We specify alwaysReturn==false because if the commit fails below Raft, we // don't want the intents to be up for resolution. That should happen only diff --git a/pkg/storage/batcheval/cmd_heartbeat_txn.go b/pkg/storage/batcheval/cmd_heartbeat_txn.go index ee8d6ea7475d..141bc0079ca5 100644 --- a/pkg/storage/batcheval/cmd_heartbeat_txn.go +++ b/pkg/storage/batcheval/cmd_heartbeat_txn.go @@ -72,8 +72,8 @@ func HeartbeatTxn( } // Verify that it is safe to create the transaction record. - if ok, reason := cArgs.EvalCtx.CanCreateTxnRecord(&txn); !ok { - return result.Result{}, roachpb.NewTransactionAbortedError(reason) + if err := CanCreateTxnRecord(cArgs.EvalCtx, &txn); err != nil { + return result.Result{}, err } } diff --git a/pkg/storage/batcheval/cmd_push_txn.go b/pkg/storage/batcheval/cmd_push_txn.go index fa2ecdec07e1..f3ff60c9194a 100644 --- a/pkg/storage/batcheval/cmd_push_txn.go +++ b/pkg/storage/batcheval/cmd_push_txn.go @@ -39,58 +39,76 @@ func declareKeysPushTransaction( pr := req.(*roachpb.PushTxnRequest) spans.Add(spanset.SpanReadWrite, roachpb.Span{Key: keys.TransactionKey(pr.PusheeTxn.Key, pr.PusheeTxn.ID)}) spans.Add(spanset.SpanReadWrite, roachpb.Span{Key: keys.AbortSpanKey(header.RangeID, pr.PusheeTxn.ID)}) + spans.Add(spanset.SpanReadOnly, roachpb.Span{Key: keys.RangeTxnSpanGCThresholdKey(header.RangeID)}) } -// PushTxn resolves conflicts between concurrent txns (or -// between a non-transactional reader or writer and a txn) in several -// ways depending on the statuses and priorities of the conflicting -// transactions. The PushTxn operation is invoked by a -// "pusher" (the writer trying to abort a conflicting txn or the -// reader trying to push a conflicting txn's commit timestamp +// PushTxn resolves conflicts between concurrent txns (or between +// a non-transactional reader or writer and a txn) in several ways, +// depending on the statuses and priorities of the conflicting +// transactions. The PushTxn operation is invoked by a "pusher" +// (args.PusherTxn -- the writer trying to abort a conflicting txn +// or the reader trying to push a conflicting txn's commit timestamp // forward), who attempts to resolve a conflict with a "pushee" -// (args.PushTxn -- the pushee txn whose intent(s) caused the +// (args.PusheeTxn -- the pushee txn whose intent(s) caused the // conflict). A pusher is either transactional, in which case -// PushTxn is completely initialized, or not, in which case the -// PushTxn has only the priority set. +// PusherTxn is completely initialized, or not, in which case the +// PusherTxn has only the priority set. // -// Txn already committed/aborted: If pushee txn is committed or -// aborted return success. +// The request arrives and immediately tries to determine the current +// disposition of the pushee transaction by reading its transaction +// record. If it finds one, it continues with the push. If not, it +// uses knowledge from the existence of the conflicting intent to +// determine the current state of the pushee. It's possible that the +// transaction record is missing either because it hasn't been written +// yet or because it has already been GCed after being finalized. Once +// the request determines which case its in, it decides whether to +// continue with the push. There are a number of different outcomes +// that a push can result in, based on the state that the pushee's +// transaction record is found in: // -// Txn Timeout: If pushee txn entry isn't present or its LastHeartbeat -// timestamp isn't set, use its as LastHeartbeat. If current time - -// LastHeartbeat > 2 * DefaultHeartbeatInterval, then the pushee txn -// should be either pushed forward, aborted, or confirmed not pending, -// depending on value of Request.PushType. +// Txn already committed/aborted: If the pushee txn is committed or +// aborted return success. // -// Old Txn Epoch: If persisted pushee txn entry has a newer Epoch than -// PushTxn.Epoch, return success, as older epoch may be removed. +// Txn record expired: If the pushee txn is pending, its last +// heartbeat timestamp is observed to determine the latest client +// activity. This heartbeat is forwarded by the conflicting intent's +// timestamp because that timestamp also indicates definitive client +// activity. This time of "last activity" is compared against the +// current time to determine whether the transaction has expired. +// If so, it is aborted. NOTE: the intent timestamp used is not +// updated on intent pushes. This is important because it allows us +// to use its timestamp as an indication of recent activity. If this +// is ever changed, we don't run the risk of any correctness violations, +// but we do make it possible for intent pushes to look like client +// activity and extend the waiting period until a transaction is +// considered expired. This waiting period is a "courtesy" - if we +// simply aborted txns right away then we would see worse performance +// under contention, but everything would still be correct. // -// Lower Txn Priority: If pushee txn has a lower priority than pusher, -// adjust pushee's persisted txn depending on value of -// args.PushType. If args.PushType is PUSH_ABORT, set txn.Status to -// ABORTED, and priority to one less than the pusher's priority and -// return success. If args.PushType is PUSH_TIMESTAMP, set -// txn.Timestamp to just after PushTo. +// Txn record not expired: If the pushee txn is not expired, its +// priority is compared against the pusher's (see CanPushWithPriority). // -// Higher Txn Priority: If pushee txn has a higher priority than -// pusher, return TransactionPushError. Transaction will be retried -// with priority one less than the pushee's higher priority. +// Push cannot proceed: a TransactionPushError is returned. // -// If the pusher is non-transactional, args.PusherTxn is an empty -// proto with only the priority set. +// Push can proceed: the pushee's transaction record is modified and +// rewritten, based on the value of args.PushType. If args.PushType +// is PUSH_ABORT, txn.Status is set to ABORTED. If args.PushType is +// PUSH_TIMESTAMP, txn.Timestamp is set to just after args.PushTo. // +// TODO(nvanbenschoten): I don't understand this below... // If the pushee is aborted, its timestamp will be forwarded to match its last // client activity timestamp (i.e. last heartbeat), if available. This is done // so that the updated timestamp populates the AbortSpan, allowing the GC -// queue to purge entries for which the transaction coordinator must have found +// queue to purge records for which the transaction coordinator must have found // out via its heartbeats that the transaction has failed. func PushTxn( ctx context.Context, batch engine.ReadWriter, cArgs CommandArgs, resp roachpb.Response, ) (result.Result, error) { args := cArgs.Args.(*roachpb.PushTxnRequest) + h := cArgs.Header reply := resp.(*roachpb.PushTxnResponse) - if cArgs.Header.Txn != nil { + if h.Txn != nil { return result.Result{}, ErrTransactionUnsupported } if args.Now == (hlc.Timestamp{}) { @@ -107,47 +125,56 @@ func PushTxn( ok, err := engine.MVCCGetProto(ctx, batch, key, hlc.Timestamp{}, existTxn, engine.MVCCGetOptions{}) if err != nil { return result.Result{}, err + } else if !ok { + // There are three cases in which there is no transaction record: + // + // * the pushee is still active but its transaction record has not + // been written yet. This is fairly common because transactions + // do not eagerly write their transaction record before writing + // intents, which another reader or writer might stumble upon and + // be forced to push. + // * the pushee resolved its intents synchronously on successful commit; + // in this case, the transaction record of the pushee is also removed. + // Note that in this case, the intent which prompted this PushTxn + // doesn't exist any more. + // * the pushee timed out or was aborted and the intent not cleaned up, + // but the transaction record was garbage collected. + // + // To determine which case we're in, we check whether the transaction could + // ever write a transaction record. We do this by using the metadata from + // the intent and attempting to synthesize a transaction record while + // verifying that it would be possible for the transaction record to ever be + // written. If a transaction record for the transaction could be written in + // the future then we must be in the first case. If one could not be written + // then we know we're in either the second or the third case. + // + // Performing this detection could have false positives where we determine + // that a record could still be written and conclude that we're in the first + // case. However, it cannot have false negatives where we determine that a + // record can not be written and conclude that we're in the second or third + // case. This is important, because it means that we may end up failing to + // push a finalized transaction but will never determine that a transaction + // is finalized when it still could end up committing. + reply.PusheeTxn = SynthesizeTxnFromMeta(cArgs.EvalCtx, args.PusheeTxn) + if reply.PusheeTxn.Status == roachpb.ABORTED { + // If the transaction is uncommittable, we don't even need to + // persist an ABORTED transaction record, we can just consider it + // aborted. This is good because it allows us to obey the invariant + // that only the transaction's own coordinator can create its + // transaction record. + result := result.Result{} + result.Local.UpdatedTxns = &[]*roachpb.Transaction{&reply.PusheeTxn} + return result, nil + } + } else { + // Start with the persisted transaction record. + reply.PusheeTxn = existTxn.Clone() + + // Forward the last heartbeat time of the transaction record by + // the timestamp of the intent. This is another indication of + // client activity. + reply.PusheeTxn.LastHeartbeat.Forward(args.PusheeTxn.Timestamp) } - // There are three cases in which there is no transaction entry: - // - // * the pushee is still active but the BeginTransaction was delayed - // for long enough that a write intent from this txn to another - // range is causing another reader or writer to push. - // * the pushee resolved its intents synchronously on successful commit; - // in this case, the transaction record of the pushee is also removed. - // Note that in this case, the intent which prompted this PushTxn - // doesn't exist any more. - // * the pushee timed out or was aborted and the intent not cleaned up, - // but the transaction record was garbage collected. - // - // We currently make no attempt at guessing which one it is, though we - // could (see #1939). Instead, a new aborted entry is always written. - // - // TODO(tschottdorf): we should actually improve this when we - // garbage-collect aborted transactions, or we run the risk of a push - // recreating a GC'ed transaction as PENDING, which is an error if it - // has open intents (which is likely if someone pushes it). - if !ok { - // The transaction doesn't exist on disk; we're allowed to abort it. - // TODO(tschottdorf): especially for SNAPSHOT transactions, there's - // something to win here by not aborting, but instead pushing the - // timestamp. For SERIALIZABLE it's less important, but still better - // to have them restart than abort. See #3344. - // TODO(tschottdorf): double-check for problems emanating from - // using a trivial Transaction proto here. Maybe some fields ought - // to receive dummy values. - reply.PusheeTxn.Status = roachpb.ABORTED - reply.PusheeTxn.TxnMeta = args.PusheeTxn - reply.PusheeTxn.Timestamp = args.Now // see method comment - // Setting OrigTimestamp bumps LastActive(); see #9265. - reply.PusheeTxn.OrigTimestamp = args.Now - result := result.Result{} - result.Local.UpdatedTxns = &[]*roachpb.Transaction{&reply.PusheeTxn} - txnRecord := reply.PusheeTxn.AsRecord() - return result, engine.MVCCPutProto(ctx, batch, cArgs.Stats, key, hlc.Timestamp{}, nil, &txnRecord) - } - // Start with the persisted transaction record as final transaction. - reply.PusheeTxn = existTxn.Clone() // If already committed or aborted, return success. if reply.PusheeTxn.Status != roachpb.PENDING { @@ -212,23 +239,48 @@ func PushTxn( // Upgrade priority of pushed transaction to one less than pusher's. reply.PusheeTxn.UpgradePriority(args.PusherTxn.Priority - 1) - // If aborting transaction, set new status and return success. - if args.PushType == roachpb.PUSH_ABORT { + // Determine what to do with the pushee, based on the push type. + switch args.PushType { + case roachpb.PUSH_ABORT: + // If aborting the transaction, set the new status. reply.PusheeTxn.Status = roachpb.ABORTED // Forward the timestamp to accommodate AbortSpan GC. See method // comment for details. reply.PusheeTxn.Timestamp.Forward(reply.PusheeTxn.LastActive()) - } else if args.PushType == roachpb.PUSH_TIMESTAMP { - // Otherwise, update timestamp to be one greater than the request's timestamp. - reply.PusheeTxn.Timestamp = args.PushTo - reply.PusheeTxn.Timestamp.Logical++ + case roachpb.PUSH_TIMESTAMP: + // Otherwise, update timestamp to be one greater than the request's + // timestamp. This new timestamp will be use to update the read + // timestamp cache. If the transaction record was not already present + // then we rely on the read timestamp cache to prevent the record from + // ever being written with a timestamp beneath this timestamp. + // + // TODO(nvanbenschoten): Remove this comment - the migration path for + // this is subtle. We rely on the read timestamp cache for timestamp + // pushes of missing transaction records. However, in mixed version + // clusters we can't be sure that a future leaseholder will consult the + // read timestamp cache. Luckily, we can be sure that it will consult + // its write timestamp cache, whose low water mark will necessarily be + // equal to or greater than this timestamp. If it finds a transaction + // with an original timestamp below this time, it will abort it. + reply.PusheeTxn.Timestamp = args.PushTo.Next() + default: + return result.Result{}, errors.Errorf("unexpected push type: %v", args.PushType) } - // Persist the pushed transaction using zero timestamp for inline value. - txnRecord := reply.PusheeTxn.AsRecord() - if err := engine.MVCCPutProto(ctx, batch, cArgs.Stats, key, hlc.Timestamp{}, nil, &txnRecord); err != nil { - return result.Result{}, err + // If the transaction record was already present, persist the updates to it. + // If not, then we don't want to create it. This could allow for finalized + // transactions to be revived. Instead, we obey the invariant that only the + // transaction's own coordinator can issue requests that create its + // transaction record. To ensure that a timestamp push or an abort is + // respected for transactions without transaction records, we rely on the + // read and write timestamp cache, respectively. + if ok { + txnRecord := reply.PusheeTxn.AsRecord() + if err := engine.MVCCPutProto(ctx, batch, cArgs.Stats, key, hlc.Timestamp{}, nil, &txnRecord); err != nil { + return result.Result{}, err + } } + result := result.Result{} result.Local.UpdatedTxns = &[]*roachpb.Transaction{&reply.PusheeTxn} return result, nil diff --git a/pkg/storage/batcheval/cmd_query_txn.go b/pkg/storage/batcheval/cmd_query_txn.go index af62dc076141..b2d0968f76bf 100644 --- a/pkg/storage/batcheval/cmd_query_txn.go +++ b/pkg/storage/batcheval/cmd_query_txn.go @@ -36,6 +36,7 @@ func declareKeysQueryTransaction( ) { qr := req.(*roachpb.QueryTxnRequest) spans.Add(spanset.SpanReadOnly, roachpb.Span{Key: keys.TransactionKey(qr.Txn.Key, qr.Txn.ID)}) + spans.Add(spanset.SpanReadOnly, roachpb.Span{Key: keys.RangeTxnSpanGCThresholdKey(header.RangeID)}) } // QueryTxn fetches the current state of a transaction. @@ -59,11 +60,15 @@ func QueryTxn( } key := keys.TransactionKey(args.Txn.Key, args.Txn.ID) - // Fetch transaction record; if missing, return empty txn. - ok, err := engine.MVCCGetProto(ctx, batch, key, hlc.Timestamp{}, &reply.QueriedTxn, - engine.MVCCGetOptions{}) - if err != nil || !ok { + // Fetch transaction record; if missing, attempt to synthesize one. + if ok, err := engine.MVCCGetProto( + ctx, batch, key, hlc.Timestamp{}, &reply.QueriedTxn, engine.MVCCGetOptions{}, + ); err != nil { return result.Result{}, err + } else if !ok { + // The transaction hasn't written a transaction record yet. + // Attempt to synthesize it from the provided TxnMeta. + reply.QueriedTxn = SynthesizeTxnFromMeta(cArgs.EvalCtx, args.Txn) } // Get the list of txns waiting on this txn. reply.WaitingTxns = cArgs.EvalCtx.GetTxnWaitQueue().GetDependents(args.Txn.ID) diff --git a/pkg/storage/batcheval/cmd_resolve_intent_test.go b/pkg/storage/batcheval/cmd_resolve_intent_test.go index 6c56b602e893..505eafe10119 100644 --- a/pkg/storage/batcheval/cmd_resolve_intent_test.go +++ b/pkg/storage/batcheval/cmd_resolve_intent_test.go @@ -106,8 +106,8 @@ func (m *mockEvalCtx) GetSplitQPS() float64 { return m.qps } func (m *mockEvalCtx) CanCreateTxnRecord( - txn *roachpb.Transaction, -) (bool, roachpb.TransactionAbortedReason) { + uuid.UUID, []byte, hlc.Timestamp, +) (bool, hlc.Timestamp, roachpb.TransactionAbortedReason) { panic("unimplemented") } func (m *mockEvalCtx) GetGCThreshold() hlc.Timestamp { diff --git a/pkg/storage/batcheval/eval_context.go b/pkg/storage/batcheval/eval_context.go index 7c088a072817..a38fcdd8479f 100644 --- a/pkg/storage/batcheval/eval_context.go +++ b/pkg/storage/batcheval/eval_context.go @@ -28,6 +28,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/storage/txnwait" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/limit" + "github.com/cockroachdb/cockroach/pkg/util/uuid" "golang.org/x/time/rate" ) @@ -64,15 +65,15 @@ type EvalContext interface { Desc() *roachpb.RangeDescriptor ContainsKey(key roachpb.Key) bool - // CanCreateTxnRecord determines whether a transaction record can be - // created for the provided transaction. If not, it returns the reason - // that transaction record was rejected. If the method ever determines - // that a transaction record must be rejected, it will continue to - // reject that transaction going forwards. + // CanCreateTxnRecord determines whether a transaction record can be created + // for the provided transaction information. See Replica.CanCreateTxnRecord + // for details about its arguments, return values, and preconditions. // - // NOTE: To call this method, a command must delare (at least) a read - // on both the transaction's key and on the txn span GC threshold key. - CanCreateTxnRecord(*roachpb.Transaction) (bool, roachpb.TransactionAbortedReason) + // NOTE: To call this method, a command must declare (at least) a read on + // both the transaction's key and on the txn span GC threshold key. + CanCreateTxnRecord( + txnID uuid.UUID, txnKey []byte, txnMinTSUpperBound hlc.Timestamp, + ) (ok bool, minCommitTS hlc.Timestamp, reason roachpb.TransactionAbortedReason) // GetMVCCStats returns a snapshot of the MVCC stats for the range. // If called from a command that declares a read/write span on the diff --git a/pkg/storage/batcheval/transaction.go b/pkg/storage/batcheval/transaction.go index 2b37498f61cb..adecafdf1658 100644 --- a/pkg/storage/batcheval/transaction.go +++ b/pkg/storage/batcheval/transaction.go @@ -91,3 +91,62 @@ func CanPushWithPriority(pusher, pushee *roachpb.Transaction) bool { return (pusher.Priority > roachpb.MinTxnPriority && pushee.Priority == roachpb.MinTxnPriority) || (pusher.Priority == roachpb.MaxTxnPriority && pushee.Priority < pusher.Priority) } + +// CanCreateTxnRecord determines whether a transaction record can be created for +// the provided transaction. If not, the function will return an error. If so, +// the function may modify the provided transaction. +func CanCreateTxnRecord(rec EvalContext, txn *roachpb.Transaction) error { + // Provide the transaction's epoch zero original timestamp as its minimum + // timestamp. The transaction could not have written a transaction record + // previously with a timestamp below this. + epochZeroOrigTS, _ := txn.InclusiveTimeBounds() + ok, minTS, reason := rec.CanCreateTxnRecord(txn.ID, txn.Key, epochZeroOrigTS) + if !ok { + return roachpb.NewTransactionAbortedError(reason) + } + txn.Timestamp.Forward(minTS) + return nil +} + +// SynthesizeTxnFromMeta creates a synthetic transaction object from the +// provided transaction metadata. The synthetic transaction is not meant to be +// persisted, but can serve as a representation of the transaction for outside +// observation. The function also checks whether it is possible for the +// transaction to ever create a transaction record in the future. If not, the +// returned transaction will be marked as ABORTED and it is safe to assume that +// the transaction record will never be written in the future. +func SynthesizeTxnFromMeta(rec EvalContext, txn enginepb.TxnMeta) roachpb.Transaction { + // Construct the transaction object. + synthTxnRecord := roachpb.TransactionRecord{ + TxnMeta: txn, + Status: roachpb.PENDING, + // Set the LastHeartbeat timestamp to the intent's timestamp. + // We use this as an indication of client activity. + LastHeartbeat: txn.Timestamp, + // We set the OrigTimestamp to avoid triggering an assertion + // in txn.AssertInitialized on 2.1 nodes. This value may not + // be accurate, but it won't cause issues anywhere that it + // can leak to. + // TODO(nvanbenschoten): Remove this in 2.3. + OrigTimestamp: txn.Timestamp, + } + + // Determine whether the transaction record could ever actually be written + // in the future. We provide the TxnMeta's timestamp (which we read from an + // intent) as the upper bound on the transaction's minimum timestamp. This + // may be greater than the transaction's actually original epoch-zero + // timestamp, in which case we're subjecting ourselves to false positives + // where we don't discover that a transaction is uncommittable, but never + // false negatives where we think that a transaction is uncommittable even + // when it's not and could later complete. + ok, minTS, _ := rec.CanCreateTxnRecord(txn.ID, txn.Key, txn.Timestamp) + if ok { + // Forward the provisional commit timestamp by the minimum timestamp that + // the transaction would be able to create a transaction record at. + synthTxnRecord.Timestamp.Forward(minTS) + } else { + // Mark the transaction as ABORTED because it is uncommittable. + synthTxnRecord.Status = roachpb.ABORTED + } + return synthTxnRecord.AsTransaction() +} diff --git a/pkg/storage/gc_queue_test.go b/pkg/storage/gc_queue_test.go index f48d7ac98b39..8c5ec3a77db2 100644 --- a/pkg/storage/gc_queue_test.go +++ b/pkg/storage/gc_queue_test.go @@ -715,9 +715,6 @@ func TestGCQueueTransactionTable(t *testing.T) { if test.hb > 0 { txn.LastHeartbeat = hlc.Timestamp{WallTime: test.hb} } - // Set a high Timestamp to make sure it does not matter. Only - // OrigTimestamp (and heartbeat) are used for GC decisions. - txn.Timestamp.Forward(hlc.MaxTimestamp) txns[strKey] = *txn for _, addrKey := range []roachpb.Key{baseKey, outsideKey} { key := keys.TransactionKey(addrKey, txn.ID) diff --git a/pkg/storage/replica_eval_context_span.go b/pkg/storage/replica_eval_context_span.go index 010d957d412c..61ee9bf55539 100644 --- a/pkg/storage/replica_eval_context_span.go +++ b/pkg/storage/replica_eval_context_span.go @@ -29,6 +29,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/storage/storagebase" "github.com/cockroachdb/cockroach/pkg/storage/txnwait" "github.com/cockroachdb/cockroach/pkg/util/hlc" + "github.com/cockroachdb/cockroach/pkg/util/uuid" ) // SpanSetReplicaEvalContext is a testing-only implementation of @@ -142,20 +143,19 @@ func (rec SpanSetReplicaEvalContext) GetSplitQPS() float64 { return rec.i.GetSplitQPS() } -// CanCreateTxnRecord determines whether a transaction record can be created for -// the provided transaction. If not, it returns the reason that transaction -// record was rejected. If the method ever determines that a transaction record -// must be rejected, it will continue to reject that transaction going forwards. +// CanCreateTxnRecord determines whether a transaction record can be created +// for the provided transaction information. See Replica.CanCreateTxnRecord +// for details about its arguments, return values, and preconditions. func (rec SpanSetReplicaEvalContext) CanCreateTxnRecord( - txn *roachpb.Transaction, -) (bool, roachpb.TransactionAbortedReason) { + txnID uuid.UUID, txnKey []byte, txnMinTSUpperBound hlc.Timestamp, +) (bool, hlc.Timestamp, roachpb.TransactionAbortedReason) { rec.ss.AssertAllowed(spanset.SpanReadOnly, - roachpb.Span{Key: keys.TransactionKey(txn.Key, txn.ID)}, + roachpb.Span{Key: keys.TransactionKey(txnKey, txnID)}, ) rec.ss.AssertAllowed(spanset.SpanReadOnly, roachpb.Span{Key: keys.RangeTxnSpanGCThresholdKey(rec.GetRangeID())}, ) - return rec.i.CanCreateTxnRecord(txn) + return rec.i.CanCreateTxnRecord(txnID, txnKey, txnMinTSUpperBound) } // GetGCThreshold returns the GC threshold of the Range, typically updated when diff --git a/pkg/storage/replica_test.go b/pkg/storage/replica_test.go index d4fa94308f2e..b40f81f40e75 100644 --- a/pkg/storage/replica_test.go +++ b/pkg/storage/replica_test.go @@ -3190,14 +3190,14 @@ func TestSerializableDeadline(t *testing.T) { } } -// TestTxnSpanGCThreshold verifies that aborting transactions which haven't -// written their initial txn record yet does not lead to anomalies. Precisely, -// verify that if the GC queue could potentially have removed a txn record -// created through a successful push (by a concurrent actor), the original -// transaction's subsequent attempt to create its initial record fails. +// TestTxnRecordUnderTxnSpanGCThreshold verifies that aborting transactions does +// not lead to anomalies even after the aborted transaction record is cleaned up. +// Precisely, verify that if the GC queue could potentially have removed a txn +// record created through a successful push (by a concurrent actor), the +// original transaction's subsequent attempt to create its initial record fails. // // See #9265 for context. -func TestEndTransactionTxnSpanGCThreshold(t *testing.T) { +func TestTxnRecordUnderTxnSpanGCThreshold(t *testing.T) { defer leaktest.AfterTest(t)() tc := testContext{} stopper := stop.NewStopper() @@ -3209,13 +3209,14 @@ func TestEndTransactionTxnSpanGCThreshold(t *testing.T) { // This test avoids a zero-timestamp regression (see LastActive() below), // so avoid zero timestamps. tc.manualClock.Increment(123) - pusher := &roachpb.Transaction{} // non-transactional pusher is enough + pusher := newTransaction("pusher", key, 1, tc.Clock()) // This pushee should never be allowed to write a txn record because it // will be aborted before it even tries. - pushee := newTransaction("foo", key, 1, tc.Clock()) - pushReq := pushTxnArgs(pusher, pushee, roachpb.PUSH_TOUCH) + pushee := newTransaction("pushee", key, 1, tc.Clock()) + pushReq := pushTxnArgs(pusher, pushee, roachpb.PUSH_ABORT) pushReq.Now = tc.Clock().Now() + pushReq.Force = true resp, pErr := tc.SendWrapped(&pushReq) if pErr != nil { t.Fatal(pErr) @@ -3224,10 +3225,6 @@ func TestEndTransactionTxnSpanGCThreshold(t *testing.T) { if abortedPushee.Status != roachpb.ABORTED { t.Fatalf("expected push to abort pushee, got %+v", abortedPushee) } - if lastActive := abortedPushee.LastActive(); lastActive.Less(pushReq.Now) { - t.Fatalf("pushee has no recent activity: %s (expected >= %s): %+v", - lastActive, pushReq.Now, abortedPushee) - } gcHeader := roachpb.RequestHeader{ Key: desc.StartKey.AsRawKey(), @@ -3252,13 +3249,11 @@ func TestEndTransactionTxnSpanGCThreshold(t *testing.T) { // Try to let our transaction write its initial record. If this succeeds, // we're in trouble because other written intents may have been aborted, - // i.e. the transaction might commit but lose some of its writes. - // - // It should not succeed because all request types that can create txn - // records check the transaction's original timestamp against the persisted - // TxnSpanGCThreshold. + // i.e. the transaction might commit but lose some of its writes. It should + // not succeed because the abort is reflected in the write timestamp cache, + // which is consulted when attempting to create the transaction record. { - expErr := "TransactionAbortedError(ABORT_REASON_NEW_TXN_RECORD_TOO_OLD)" + expErr := "TransactionAbortedError(ABORT_REASON_ABORTED_RECORD_FOUND)" // BeginTransaction. bt, btH := beginTxnArgs(key, pushee) @@ -3288,27 +3283,73 @@ func TestEndTransactionTxnSpanGCThreshold(t *testing.T) { } } + // A second transaction which begins at the same timestamp should also be + // rejected. This time it wasn't explicit aborted, so it won't run into the + // write timestamp cache. Instead, it should not succeed because all request + // types that can create txn records check the transaction's original + // timestamp against the persisted TxnSpanGCThreshold. + // + // TODO(nvanbenschoten): This changes to this test demonstrate that checking + // the TxnSpanGCThreshold isn't necessary anymore because for the GC to remove + // a transaction record, it must first abort it, which is recorded in the + // write timestamp cache. Should we remove this logic or do we still like it? + // Removing it would allow new transactions beneath the GC threshold, but + // would not allow transactions to be revived. See the corresponding TODO in + // Replica.CanCreateTxnRecord. + { + expErr := "TransactionAbortedError(ABORT_REASON_NEW_TXN_RECORD_TOO_OLD)" + oldTxn := newTransaction("old", key, 1, tc.Clock()) + oldTxn.OrigTimestamp = pushee.OrigTimestamp + + // BeginTransaction. + bt, btH := beginTxnArgs(key, oldTxn) + resp, pErr := tc.SendWrappedWith(btH, &bt) + if pErr == nil { + t.Fatalf("unexpected success: %+v", resp) + } else if !testutils.IsPError(pErr, regexp.QuoteMeta(expErr)) { + t.Fatalf("expected %s, got %v and response %+v", expErr, pErr, resp) + } + + // HeartbeatTxn. + hb, hbH := heartbeatArgs(oldTxn, tc.Clock().Now()) + resp, pErr = tc.SendWrappedWith(hbH, &hb) + if pErr == nil { + t.Fatalf("unexpected success: %+v", resp) + } else if !testutils.IsPError(pErr, regexp.QuoteMeta(expErr)) { + t.Fatalf("expected %s, got %v and response %+v", expErr, pErr, resp) + } + + // EndTransaction. + et, etH := endTxnArgs(oldTxn, true) + resp, pErr = tc.SendWrappedWith(etH, &et) + if pErr == nil { + t.Fatalf("unexpected success: %+v", resp) + } else if !testutils.IsPError(pErr, regexp.QuoteMeta(expErr)) { + t.Fatalf("expected %s, got %v and response %+v", expErr, pErr, resp) + } + } + // A transaction which starts later (i.e. at a higher timestamp) should not // be prevented from writing its record. // See #9522. { // BeginTransaction. - txn := newTransaction("foo", key, 1, tc.Clock()) - bt, btH := beginTxnArgs(key, txn) + newTxn := newTransaction("foo", key, 1, tc.Clock()) + bt, btH := beginTxnArgs(key, newTxn) if _, pErr := tc.SendWrappedWith(btH, &bt); pErr != nil { t.Fatal(pErr) } // HeartbeatTxn. - txn2 := newTransaction("foo", key, 1, tc.Clock()) - hb, hbH := heartbeatArgs(txn2, tc.Clock().Now()) + newTxn2 := newTransaction("foo", key, 1, tc.Clock()) + hb, hbH := heartbeatArgs(newTxn2, tc.Clock().Now()) if _, pErr := tc.SendWrappedWith(hbH, &hb); pErr != nil { t.Fatal(pErr) } // EndTransaction. - txn3 := newTransaction("foo", key, 1, tc.Clock()) - et, etH := endTxnArgs(txn3, true) + newTxn3 := newTransaction("foo", key, 1, tc.Clock()) + et, etH := endTxnArgs(newTxn3, true) if _, pErr := tc.SendWrappedWith(etH, &et); pErr != nil { t.Fatal(pErr) } @@ -4687,53 +4728,75 @@ func TestPushTxnBadKey(t *testing.T) { // (noop) in event that pushee is already committed or aborted. func TestPushTxnAlreadyCommittedOrAborted(t *testing.T) { defer leaktest.AfterTest(t)() + // This test simulates running into an open intent and resolving it using - // the transaction record. If we auto-gc'ed entries here, the entry would - // be deleted and the intents resolved instantaneously on successful commit - // (since they're on the same Range). Could split the range and have - // non-local intents if we ever wanted to get rid of this. - defer setTxnAutoGC(false)() - tc := testContext{} - stopper := stop.NewStopper() - defer stopper.Stop(context.TODO()) - tc.Start(t, stopper) + // the transaction record. We test this in two ways: + // 1. The first prevents the transaction record from being GCed by the + // EndTxn request. The effect of this is that the pusher finds the + // transaction record in a finalized status and returns it directly. + // 2. The second allows the transaction record to be GCed by the EndTxn + // request. The effect of this is that the pusher finds no transaction + // record but discovers that the transaction has already been finalized + // using the write timestamp cache. It doesn't know whether the transaction + // was COMMITTED or ABORTED, so it is forced to be conservative and return + // an ABORTED transaction. + testutils.RunTrueAndFalse(t, "auto-gc", func(t *testing.T, autoGC bool) { + defer setTxnAutoGC(autoGC)() + + // Test for COMMITTED and ABORTED transactions. + testutils.RunTrueAndFalse(t, "commit", func(t *testing.T, commit bool) { + tc := testContext{} + stopper := stop.NewStopper() + defer stopper.Stop(context.TODO()) + tc.Start(t, stopper) - for i, status := range []roachpb.TransactionStatus{roachpb.COMMITTED, roachpb.ABORTED} { - key := roachpb.Key(fmt.Sprintf("key-%d", i)) - pusher := newTransaction("test", key, 1, tc.Clock()) - pushee := newTransaction("test", key, 1, tc.Clock()) + key := roachpb.Key(fmt.Sprintf("key-%t-%t", autoGC, commit)) + pusher := newTransaction("test", key, 1, tc.Clock()) + pushee := newTransaction("test", key, 1, tc.Clock()) - // Begin the pushee's transaction. - _, btH := beginTxnArgs(key, pushee) - put := putArgs(key, key) - assignSeqNumsForReqs(pushee, &put) - if _, pErr := maybeWrapWithBeginTransaction(context.Background(), tc.Sender(), btH, &put); pErr != nil { - t.Fatal(pErr) - } - // End the pushee's transaction. - etArgs, h := endTxnArgs(pushee, status == roachpb.COMMITTED) - assignSeqNumsForReqs(pushee, &etArgs) - if _, pErr := tc.SendWrappedWith(h, &etArgs); pErr != nil { - t.Fatal(pErr) - } + // Begin the pushee's transaction. + _, btH := beginTxnArgs(key, pushee) + put := putArgs(key, key) + assignSeqNumsForReqs(pushee, &put) + if _, pErr := maybeWrapWithBeginTransaction(context.Background(), tc.Sender(), btH, &put); pErr != nil { + t.Fatal(pErr) + } + // End the pushee's transaction. + etArgs, h := endTxnArgs(pushee, commit) + assignSeqNumsForReqs(pushee, &etArgs) + if _, pErr := tc.SendWrappedWith(h, &etArgs); pErr != nil { + t.Fatal(pErr) + } - // Now try to push what's already committed or aborted. - args := pushTxnArgs(pusher, pushee, roachpb.PUSH_ABORT) - resp, pErr := tc.SendWrapped(&args) - if pErr != nil { - t.Fatal(pErr) - } - reply := resp.(*roachpb.PushTxnResponse) - if reply.PusheeTxn.Status != status { - t.Errorf("expected push txn to return with status == %s; got %+v", status, reply.PusheeTxn) - } - } + // Now try to push what's already committed or aborted. + args := pushTxnArgs(pusher, pushee, roachpb.PUSH_ABORT) + resp, pErr := tc.SendWrapped(&args) + if pErr != nil { + t.Fatal(pErr) + } + reply := resp.(*roachpb.PushTxnResponse) + + // We expect the push to return an ABORTED transaction record for all + // cases except when the transaction is COMMITTED and its record is not + // GCed. The case where it is COMMITTED and its record is GCed can be + // surprising, but doesn't result in problems because a transaction must + // resolve all of its intents before garbage collecting its record, so + // the pusher won't end up removing a still-pending intent for a + // COMMITTED transaction. + expStatus := roachpb.ABORTED + if commit && !autoGC { + expStatus = roachpb.COMMITTED + } + if reply.PusheeTxn.Status != expStatus { + t.Errorf("expected push txn to return with status == %s; got %+v", expStatus, reply.PusheeTxn) + } + }) + }) } -// TestPushTxnUpgradeExistingTxn verifies that pushing -// a transaction record with a new epoch upgrades the pushee's -// epoch and timestamp if greater. In all test cases, the -// priorities are set such that the push will succeed. +// TestPushTxnUpgradeExistingTxn verifies that pushing a transaction record +// with a new timestamp upgrades the pushee's timestamp if greater. In all +// test cases, the priorities are set such that the push will succeed. func TestPushTxnUpgradeExistingTxn(t *testing.T) { defer leaktest.AfterTest(t)() tc := testContext{} @@ -4761,11 +4824,11 @@ func TestPushTxnUpgradeExistingTxn(t *testing.T) { pushee := newTransaction("test", key, 1, tc.Clock()) pushee.Epoch = 12345 pusher.Priority = roachpb.MaxTxnPriority // Pusher will win - pusher.Writing = true // expected when a txn is heartbeat // First, establish "start" of existing pushee's txn via BeginTransaction. pushee.Timestamp = test.startTS pushee.LastHeartbeat = test.startTS + pushee.OrigTimestamp = test.startTS _, btH := beginTxnArgs(key, pushee) put := putArgs(key, key) assignSeqNumsForReqs(pushee, &put) @@ -4787,8 +4850,8 @@ func TestPushTxnUpgradeExistingTxn(t *testing.T) { expTxn.Priority = roachpb.MaxTxnPriority - 1 expTxn.Epoch = pushee.Epoch // no change expTxn.Timestamp = test.expTS + expTxn.LastHeartbeat = test.expTS expTxn.Status = roachpb.ABORTED - expTxn.LastHeartbeat = test.startTS if !reflect.DeepEqual(expTxn, reply.PusheeTxn) { t.Fatalf("unexpected push txn in trial %d: %s", i, pretty.Diff(expTxn, reply.PusheeTxn)) @@ -4848,32 +4911,58 @@ func TestPushTxnHeartbeatTimeout(t *testing.T) { defer stopper.Stop(context.TODO()) tc.Start(t, stopper) - now := tc.Clock().Now() - ts := now.Add(1, 0) ns := base.DefaultHeartbeatInterval.Nanoseconds() testCases := []struct { - heartbeat hlc.Timestamp // zero value indicates no heartbeat - timeOffset int64 // nanoseconds - pushType roachpb.PushTxnType - expSuccess bool + txnRecord bool // was the record ever written? + heartbeatOffset int64 // nanoseconds from original timestamp, 0 for no heartbeat + timeOffset int64 // nanoseconds from original timestamp + pushType roachpb.PushTxnType + expSuccess bool }{ // Avoid using 0 as timeOffset to avoid having outcomes depend on random // logical ticks. - {hlc.Timestamp{}, 1, roachpb.PUSH_TIMESTAMP, false}, - {hlc.Timestamp{}, 1, roachpb.PUSH_ABORT, false}, - {hlc.Timestamp{}, 1, roachpb.PUSH_TOUCH, false}, - {hlc.Timestamp{}, ns, roachpb.PUSH_TIMESTAMP, false}, - {hlc.Timestamp{}, ns, roachpb.PUSH_ABORT, false}, - {hlc.Timestamp{}, ns, roachpb.PUSH_TOUCH, false}, - {hlc.Timestamp{}, ns*2 - 1, roachpb.PUSH_TIMESTAMP, false}, - {hlc.Timestamp{}, ns*2 - 1, roachpb.PUSH_ABORT, false}, - {hlc.Timestamp{}, ns*2 - 1, roachpb.PUSH_TOUCH, false}, - {hlc.Timestamp{}, ns * 2, roachpb.PUSH_TIMESTAMP, false}, - {hlc.Timestamp{}, ns * 2, roachpb.PUSH_ABORT, false}, - {hlc.Timestamp{}, ns * 2, roachpb.PUSH_TOUCH, false}, - {ts, ns*2 + 1, roachpb.PUSH_TIMESTAMP, true}, - {ts, ns*2 + 1, roachpb.PUSH_ABORT, true}, - {ts, ns*2 + 1, roachpb.PUSH_TOUCH, true}, + {true, 0, 1, roachpb.PUSH_TIMESTAMP, false}, + {true, 0, 1, roachpb.PUSH_ABORT, false}, + {true, 0, 1, roachpb.PUSH_TOUCH, false}, + {true, 0, ns, roachpb.PUSH_TIMESTAMP, false}, + {true, 0, ns, roachpb.PUSH_ABORT, false}, + {true, 0, ns, roachpb.PUSH_TOUCH, false}, + {true, 0, ns*2 - 1, roachpb.PUSH_TIMESTAMP, false}, + {true, 0, ns*2 - 1, roachpb.PUSH_ABORT, false}, + {true, 0, ns*2 - 1, roachpb.PUSH_TOUCH, false}, + {true, 0, ns * 2, roachpb.PUSH_TIMESTAMP, false}, + {true, 0, ns * 2, roachpb.PUSH_ABORT, false}, + {true, 0, ns * 2, roachpb.PUSH_TOUCH, false}, + {true, 0, ns*2 + 1, roachpb.PUSH_TIMESTAMP, true}, + {true, 0, ns*2 + 1, roachpb.PUSH_ABORT, true}, + {true, 0, ns*2 + 1, roachpb.PUSH_TOUCH, true}, + {true, ns, ns*2 + 1, roachpb.PUSH_TIMESTAMP, false}, + {true, ns, ns*2 + 1, roachpb.PUSH_ABORT, false}, + {true, ns, ns*2 + 1, roachpb.PUSH_TOUCH, false}, + {true, ns, ns * 3, roachpb.PUSH_TIMESTAMP, false}, + {true, ns, ns * 3, roachpb.PUSH_ABORT, false}, + {true, ns, ns * 3, roachpb.PUSH_TOUCH, false}, + {true, ns, ns*3 + 1, roachpb.PUSH_TIMESTAMP, true}, + {true, ns, ns*3 + 1, roachpb.PUSH_ABORT, true}, + {true, ns, ns*3 + 1, roachpb.PUSH_TOUCH, true}, + // Even when a transaction record doesn't exist, if the timestamp + // from the PushTxn request indicates sufficiently recent client + // activity, the push will fail. + {false, 0, 1, roachpb.PUSH_TIMESTAMP, false}, + {false, 0, 1, roachpb.PUSH_ABORT, false}, + {false, 0, 1, roachpb.PUSH_TOUCH, false}, + {false, 0, ns, roachpb.PUSH_TIMESTAMP, false}, + {false, 0, ns, roachpb.PUSH_ABORT, false}, + {false, 0, ns, roachpb.PUSH_TOUCH, false}, + {false, 0, ns*2 - 1, roachpb.PUSH_TIMESTAMP, false}, + {false, 0, ns*2 - 1, roachpb.PUSH_ABORT, false}, + {false, 0, ns*2 - 1, roachpb.PUSH_TOUCH, false}, + {false, 0, ns * 2, roachpb.PUSH_TIMESTAMP, false}, + {false, 0, ns * 2, roachpb.PUSH_ABORT, false}, + {false, 0, ns * 2, roachpb.PUSH_TOUCH, false}, + {false, 0, ns*2 + 1, roachpb.PUSH_TIMESTAMP, true}, + {false, 0, ns*2 + 1, roachpb.PUSH_ABORT, true}, + {false, 0, ns*2 + 1, roachpb.PUSH_TOUCH, true}, } for i, test := range testCases { @@ -4881,21 +4970,29 @@ func TestPushTxnHeartbeatTimeout(t *testing.T) { pushee := newTransaction(fmt.Sprintf("test-%d", i), key, 1, tc.Clock()) pusher := newTransaction("pusher", key, 1, tc.Clock()) - // First, establish "start" of existing pushee's txn via BeginTransaction. - if test.heartbeat != (hlc.Timestamp{}) { - pushee.LastHeartbeat = test.heartbeat + // Add the pushee's heartbeat offset. + if test.heartbeatOffset != 0 { + if !test.txnRecord { + t.Fatal("cannot heartbeat transaction record if it doesn't exist") + } + pushee.LastHeartbeat = pushee.OrigTimestamp.Add(test.heartbeatOffset, 0) } - _, btH := beginTxnArgs(key, pushee) - btH.Timestamp = pushee.Timestamp - put := putArgs(key, key) - assignSeqNumsForReqs(pushee, &put) - if _, pErr := maybeWrapWithBeginTransaction(context.Background(), tc.Sender(), btH, &put); pErr != nil { - t.Fatalf("%d: %s", i, pErr) + + // Establish "start" of existing pushee's txn via BeginTransaction request + // if the test case wants an existing transaction record. + if test.txnRecord { + _, btH := beginTxnArgs(key, pushee) + btH.Timestamp = pushee.Timestamp + put := putArgs(key, key) + assignSeqNumsForReqs(pushee, &put) + if _, pErr := maybeWrapWithBeginTransaction(context.Background(), tc.Sender(), btH, &put); pErr != nil { + t.Fatalf("%d: %s", i, pErr) + } } // Now, attempt to push the transaction with Now set to the txn start time + offset. args := pushTxnArgs(pusher, pushee, test.pushType) - args.Now = pushee.Timestamp.Add(test.timeOffset, 0) + args.Now = pushee.OrigTimestamp.Add(test.timeOffset, 0) args.PushTo = args.Now reply, pErr := tc.SendWrapped(&args) @@ -4916,8 +5013,9 @@ func TestPushTxnHeartbeatTimeout(t *testing.T) { } } -// TestResolveIntentPushTxnReplyTxn makes sure that no Txn is returned from PushTxn and that -// it and ResolveIntent{,Range} can not be carried out in a transaction. +// TestResolveIntentPushTxnReplyTxn makes sure that no Txn is returned from +// PushTxn and that it and ResolveIntent{,Range} can not be carried out in a +// transaction. func TestResolveIntentPushTxnReplyTxn(t *testing.T) { defer leaktest.AfterTest(t)() tc := testContext{} @@ -4931,6 +5029,7 @@ func TestResolveIntentPushTxnReplyTxn(t *testing.T) { txn := newTransaction("test", roachpb.Key("test"), 1, tc.Clock()) txnPushee := txn.Clone() pa := pushTxnArgs(txn, &txnPushee, roachpb.PUSH_ABORT) + pa.Force = true var ms enginepb.MVCCStats var ra roachpb.ResolveIntentRequest var rra roachpb.ResolveIntentRangeRequest @@ -4950,7 +5049,7 @@ func TestResolveIntentPushTxnReplyTxn(t *testing.T) { // Should not get a transaction back from PushTxn. It used to erroneously // return args.PusherTxn. var reply roachpb.PushTxnResponse - if _, err := batcheval.PushTxn(ctx, b, batcheval.CommandArgs{Stats: &ms, Args: &pa}, &reply); err != nil { + if _, err := batcheval.PushTxn(ctx, b, batcheval.CommandArgs{EvalCtx: tc.repl, Stats: &ms, Args: &pa}, &reply); err != nil { t.Fatal(err) } else if reply.Txn != nil { t.Fatalf("expected nil response txn, but got %s", reply.Txn) @@ -5036,10 +5135,9 @@ func TestPushTxnPriorities(t *testing.T) { } } -// TestPushTxnPushTimestamp verifies that with args.Abort is -// false (i.e. for read/write conflict), the pushed txn keeps status -// PENDING, but has its txn Timestamp moved forward to the pusher's -// txn Timestamp + 1. +// TestPushTxnPushTimestamp verifies that with PUSH_TIMESTAMP pushes (i.e. for +// read/write conflict), the pushed txn keeps status PENDING, but has its txn +// Timestamp moved forward to the pusher's txn Timestamp + 1. func TestPushTxnPushTimestamp(t *testing.T) { defer leaktest.AfterTest(t)() tc := testContext{} @@ -5064,7 +5162,7 @@ func TestPushTxnPushTimestamp(t *testing.T) { } pushee.Writing = true - // Now, push the transaction with args.Abort=false. + // Now, push the transaction using a PUSH_TIMESTAMP push request. args := pushTxnArgs(pusher, pushee, roachpb.PUSH_TIMESTAMP) resp, pErr := tc.SendWrapped(&args) @@ -5107,7 +5205,7 @@ func TestPushTxnPushTimestampAlreadyPushed(t *testing.T) { t.Fatal(pErr) } - // Now, push the transaction with args.Abort=false. + // Now, push the transaction using a PUSH_TIMESTAMP push request. args := pushTxnArgs(pusher, pushee, roachpb.PUSH_TIMESTAMP) resp, pErr := tc.SendWrapped(&args) @@ -8458,6 +8556,9 @@ func TestNoopRequestsNotProposed(t *testing.T) { deleteReq := &roachpb.DeleteRequest{ RequestHeader: rh, } + beginTxnReq := &roachpb.BeginTransactionRequest{ + RequestHeader: rh, + } pushTxnReq := &roachpb.PushTxnRequest{ RequestHeader: roachpb.RequestHeader{ Key: txn.TxnMeta.Key, @@ -8465,6 +8566,7 @@ func TestNoopRequestsNotProposed(t *testing.T) { PusheeTxn: txn.TxnMeta, Now: cfg.Clock.Now(), PushType: roachpb.PUSH_ABORT, + Force: true, } resolveCommittedIntentReq := &roachpb.ResolveIntentRequest{ RequestHeader: rh, @@ -8537,13 +8639,19 @@ func TestNoopRequestsNotProposed(t *testing.T) { expProposal: true, }, { - name: "push txn req", + name: "push txn req", + setup: func(ctx context.Context, repl *Replica) *roachpb.Error { + return sendReq(ctx, repl, beginTxnReq, txn) + }, req: pushTxnReq, expProposal: true, }, { name: "redundant push txn req", setup: func(ctx context.Context, repl *Replica) *roachpb.Error { + if pErr := sendReq(ctx, repl, beginTxnReq, txn); pErr != nil { + return pErr + } return sendReq(ctx, repl, pushTxnReq, nil /* txn */) }, req: pushTxnReq, @@ -8590,6 +8698,11 @@ func TestNoopRequestsNotProposed(t *testing.T) { tc.StartWithStoreConfig(t, stopper, cfg) repl := tc.repl + // Update the transaction's timestamps so that it + // doesn't run into issues with the new cluster. + txn.Timestamp = tc.Clock().Now() + txn.OrigTimestamp = txn.Timestamp + if c.setup != nil { if pErr := c.setup(ctx, repl); pErr != nil { t.Fatalf("test setup failed: %v", pErr) @@ -10532,13 +10645,9 @@ func TestCreateTxnRecord(t *testing.T) { bt, btH := beginTxnArgs(txn.Key, txn) return sendWrappedWithErr(btH, &bt) }, - expError: "TransactionAbortedError(ABORT_REASON_ABORTED_RECORD_FOUND)", - expTxn: func(txn *roachpb.Transaction, pushTs hlc.Timestamp) roachpb.TransactionRecord { + expTxn: func(txn *roachpb.Transaction, now hlc.Timestamp) roachpb.TransactionRecord { record := txn.AsRecord() - record.Status = roachpb.ABORTED - record.Timestamp = pushTs - record.OrigTimestamp = pushTs - record.LastHeartbeat = hlc.Timestamp{} + record.Timestamp.Forward(now.Add(0, 1)) return record }, }, @@ -10551,19 +10660,13 @@ func TestCreateTxnRecord(t *testing.T) { return sendWrappedWithErr(roachpb.Header{}, &pt) }, run: func(txn *roachpb.Transaction, now hlc.Timestamp) error { - hb, hbH := heartbeatArgs(txn, now) + hb, hbH := heartbeatArgs(txn, now.Add(0, 5)) return sendWrappedWithErr(hbH, &hb) }, - // The heartbeat request won't throw an error, but also won't update the - // transaction record. It will simply return the updated transaction state. - // This is kind of strange, but also doesn't cause any issues. - expError: "", - expTxn: func(txn *roachpb.Transaction, pushTs hlc.Timestamp) roachpb.TransactionRecord { + expTxn: func(txn *roachpb.Transaction, now hlc.Timestamp) roachpb.TransactionRecord { record := txn.AsRecord() - record.Status = roachpb.ABORTED - record.Timestamp = pushTs - record.OrigTimestamp = pushTs - record.LastHeartbeat = hlc.Timestamp{} + record.Timestamp.Forward(now.Add(0, 1)) + record.LastHeartbeat.Forward(now.Add(0, 5)) return record }, }, @@ -10595,15 +10698,10 @@ func TestCreateTxnRecord(t *testing.T) { et, etH := endTxnArgs(txn, true /* commit */) return sendWrappedWithErr(etH, &et) }, - expError: "TransactionAbortedError(ABORT_REASON_ABORTED_RECORD_FOUND)", - expTxn: func(txn *roachpb.Transaction, pushTs hlc.Timestamp) roachpb.TransactionRecord { - record := txn.AsRecord() - record.Status = roachpb.ABORTED - record.Timestamp = pushTs - record.OrigTimestamp = pushTs - record.LastHeartbeat = hlc.Timestamp{} - return record - }, + expError: "TransactionRetryError: retry txn (RETRY_SERIALIZABLE)", + // The end transaction (commit) does not write a transaction record + // if it hits a serializable retry error. + expTxn: noTxnRecord, }, { name: "begin transaction after push abort", @@ -10617,14 +10715,7 @@ func TestCreateTxnRecord(t *testing.T) { return sendWrappedWithErr(btH, &bt) }, expError: "TransactionAbortedError(ABORT_REASON_ABORTED_RECORD_FOUND)", - expTxn: func(txn *roachpb.Transaction, pushTs hlc.Timestamp) roachpb.TransactionRecord { - record := txn.AsRecord() - record.Status = roachpb.ABORTED - record.Timestamp = pushTs - record.OrigTimestamp = pushTs - record.LastHeartbeat = hlc.Timestamp{} - return record - }, + expTxn: noTxnRecord, }, { name: "heartbeat transaction after push abort", @@ -10637,18 +10728,8 @@ func TestCreateTxnRecord(t *testing.T) { hb, hbH := heartbeatArgs(txn, now) return sendWrappedWithErr(hbH, &hb) }, - // The heartbeat request won't throw an error, but also won't update the - // transaction record. It will simply return the updated transaction state. - // This is kind of strange, but also doesn't cause any issues. - expError: "", - expTxn: func(txn *roachpb.Transaction, pushTs hlc.Timestamp) roachpb.TransactionRecord { - record := txn.AsRecord() - record.Status = roachpb.ABORTED - record.Timestamp = pushTs - record.OrigTimestamp = pushTs - record.LastHeartbeat = hlc.Timestamp{} - return record - }, + expError: "TransactionAbortedError(ABORT_REASON_ABORTED_RECORD_FOUND)", + expTxn: noTxnRecord, }, { name: "heartbeat transaction after push abort and restart", @@ -10667,18 +10748,8 @@ func TestCreateTxnRecord(t *testing.T) { hb, hbH := heartbeatArgs(&clone, now) return sendWrappedWithErr(hbH, &hb) }, - // The heartbeat request won't throw an error, but also won't update the - // transaction record. It will simply return the updated transaction state. - // This is kind of strange, but also doesn't cause any issues. - expError: "", - expTxn: func(txn *roachpb.Transaction, pushTs hlc.Timestamp) roachpb.TransactionRecord { - record := txn.AsRecord() - record.Status = roachpb.ABORTED - record.Timestamp = pushTs - record.OrigTimestamp = pushTs - record.LastHeartbeat = hlc.Timestamp{} - return record - }, + expError: "TransactionAbortedError(ABORT_REASON_ABORTED_RECORD_FOUND)", + expTxn: noTxnRecord, }, { name: "end transaction (abort) after push abort", @@ -10707,14 +10778,7 @@ func TestCreateTxnRecord(t *testing.T) { return sendWrappedWithErr(etH, &et) }, expError: "TransactionAbortedError(ABORT_REASON_ABORTED_RECORD_FOUND)", - expTxn: func(txn *roachpb.Transaction, pushTs hlc.Timestamp) roachpb.TransactionRecord { - record := txn.AsRecord() - record.Status = roachpb.ABORTED - record.Timestamp = pushTs - record.OrigTimestamp = pushTs - record.LastHeartbeat = hlc.Timestamp{} - return record - }, + expTxn: noTxnRecord, }, { name: "begin transaction after gc", diff --git a/pkg/storage/replica_tscache.go b/pkg/storage/replica_tscache.go index ea2d5b0a7997..616b85268758 100644 --- a/pkg/storage/replica_tscache.go +++ b/pkg/storage/replica_tscache.go @@ -20,7 +20,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/util/hlc" - "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/uuid" ) @@ -68,9 +67,32 @@ func (r *Replica) updateTimestampCache( // which is consulted in CanCreateTxnRecord. key := keys.TransactionKey(start, txnID) tc.Add(key, nil, ts, txnID, false /* readCache */) + case *roachpb.PushTxnRequest: + // A successful PushTxn request bumps the timestamp cache for + // the pushee's transaction key. The pushee will consult the + // timestamp cache when creating its record. If the push left + // the transaction in a PENDING state (PUSH_TIMESTAMP) then we + // update the read timestamp cache. This will cause the creator + // of the transaction record to forward its provisional commit + // timestamp to honor the result of this push. If the push left + // the transaction in an ABORTED state (PUSH_ABORT) then we + // update the write timestamp cache. This will prevent the + // creation of the transaction record entirely. + pushee := br.Responses[i].GetInner().(*roachpb.PushTxnResponse).PusheeTxn + + // Update the local clock to the pushee's new timestamp. This + // ensures that we can safely update the timestamp cache based + // on this value. The pushee's timestamp is not reflected in + // the batch request's or response's timestamp. + r.store.Clock().Update(pushee.Timestamp) + + key := keys.TransactionKey(start, pushee.ID) + readCache := pushee.Status != roachpb.ABORTED + tc.Add(key, nil, pushee.Timestamp, t.PusherTxn.ID, readCache) case *roachpb.ConditionalPutRequest: if pErr != nil { // ConditionalPut still updates on ConditionFailedErrors. + // TODO(nvanbenschoten): do we need similar logic for InitPutRequest? if _, ok := pErr.GetDetail().(*roachpb.ConditionFailedError); !ok { continue } @@ -194,44 +216,138 @@ func (r *Replica) applyTimestampCache( } // CanCreateTxnRecord determines whether a transaction record can be created -// for the provided transaction. If not, it returns the reason that transaction -// record was rejected. If the method ever determines that a transaction record -// must be rejected, it will continue to reject that transaction going forwards. +// for the provided transaction information. Callers should provide an upper +// bound on the transaction's minimum timestamp across all epochs (typically +// the original timestamp of its first epoch). If this is not exact then the +// method may return false positives (i.e. it determines that the record could +// be created when it actually couldn't) but will never return false negatives +// (i.e. it determines that the record could not be created when it actually +// could). +// +// Because of this, callers who intend to write the transaction record should +// always provide an exact minimum timestamp. They can't provide their +// provisional commit timestamp because it may have moved forward over the +// course of a single epoch and they can't provide their (current epoch's) +// OrigTimestamp because it may have moved forward over a series of prior +// epochs. Either of these timestamps might be above the timestamp that a +// successful aborter might have used when aborting the transaction. Instead, +// they should provide their epoch-zero original timestamp. +// +// If the method return true, it also returns the minimum provisional commit +// timestamp that the record can be created with. If the method returns false, +// it returns the reason that transaction record was rejected. If the method +// ever determines that a transaction record must be rejected, it will continue +// to reject that transaction going forwards. +// +// The method performs two critical roles: +// 1. It protects against replays or other requests that could otherwise cause +// a transaction record to be created after the transaction has already been +// finalized and its record cleaned up. +// 2. It serves as the mechanism by which successful push requests convey +// information to transactions who have not yet written their transaction +// record. In doing so, it protects against transaction records from being +// created with too low of a timestamp. +// +// This is detailed in the transaction record state machine below: +// +// +-----------------------------------+ +// |vars | +// |-----------------------------------| +// |v1 = rTSCache[txn.id] = timestamp| +// |v2 = wTSCache[txn.id] = timestamp| +// |v3 = txnSpanGCThreshold = timestamp| +// +-----------------------------------+ +// +// PushTxn(TIMESTAMP) HeartbeatTxn +// then: v1 = push.ts then: update record +// +------+ +------+ +// PushTxn(ABORT) | | BeginTxn or HeartbeatTxn | | PushTxn(TIMESTAMP) +// then: v2 = txn.ts | v if: v2 < txn.orig | v then: update record +// +---------------+ & v3 < txn.orig +--------------------+ +// +----| | then: txn.ts.forward(v1) | |----+ +// | | | else: fail | | | +// | | no txn record |------------------------->| txn record written | | +// +--->| | | [pending] |<---+ +// | |-+ | | +// +---------------+ \ +--------------------+ +// ^ ^ \ / / +// | \ \ EndTxn / EndTxn / +// | \ \ if: same conditions / then: v2 = txn.ts +// | \ \ as above / / +// | Eager GC \ \ then: v2 = txn.ts / / +// | if: EndTxn \ \ else: fail / / PushTxn(ABORT) +// | transition \ v v / then: v2 = txn.ts +// | taken \ +--------------------+ / +// | \ | | / +// \ +--| | / +// \ | txn record written |<----+ +// +-------------------| [finalized] | +// GC queue | | +// then: v3 = now() +--------------------+ +// ^ | +// | | PushTxn(*) +// +------+ then: no-op +// +// +// In the diagram, CanCreateTxnRecord is consulted in both of the state +// transitions that move away from the "no txn record" state. Updating +// v1 and v2 is performed in updateTimestampCache. func (r *Replica) CanCreateTxnRecord( - txn *roachpb.Transaction, -) (bool, roachpb.TransactionAbortedReason) { - // We make comparisons below against the epoch zero timestamp because the - // transaction's provisional commit timestamp may be moved forward over the - // course of a single epoch and its original timestamp may have moved - // forward over the course of a series of epochs. - minTxnTS, _ := txn.InclusiveTimeBounds() + txnID uuid.UUID, txnKey []byte, txnMinTSUpperBound hlc.Timestamp, +) (ok bool, minCommitTS hlc.Timestamp, reason roachpb.TransactionAbortedReason) { + // Consult the timestamp cache with the transaction's key. The timestamp + // cache is used in two ways for transactions without transaction records. + // The read timestamp cache is used to push the timestamp of transactions + // that don't have transaction records. The write timestamp cache is used + // to abort transactions entirely that don't have transaction records. + // + // Using this strategy, we enforce the invariant that only requests sent + // from a transaction's own coordinator can create its transaction record. + // However, once a transaction record is written, other concurrent actors + // can modify it. This is reflected in the diagram above. + key := keys.TransactionKey(txnKey, txnID) + + // Look in the read timestamp cache to see if there is an entry for this + // transaction, which indicates the minimum timestamp that the transaction + // can commit at. This is used by pushers to push the timestamp of a + // transaction that hasn't yet written its transaction record. + minCommitTS, _ = r.store.tsCache.GetMaxRead(key, nil /* end */) - // We look in the timestamp cache to see if there is an entry for this - // transaction, which would indicate this transaction has already been - // finalized. If there is an entry, then we return a retriable error: if - // this is a re-evaluation, then the error will be transformed into an - // ambiguous one higher up. Otherwise, if the client is still waiting for a - // result, then this cannot be a "replay" of any sort. - key := keys.TransactionKey(txn.Key, txn.ID) + // Also look in the write timestamp cache to see if there is an entry for + // this transaction, which would indicate this transaction has already been + // finalized or was already aborted by a concurrent transaction. If there is + // an entry, then we return a retriable error: if this is a re-evaluation, + // then the error will be transformed into an ambiguous one higher up. + // Otherwise, if the client is still waiting for a result, then this cannot + // be a "replay" of any sort. wTS, wTxnID := r.store.tsCache.GetMaxWrite(key, nil /* end */) - // GetMaxWrite will only find a timestamp interval with an associated txnID - // on the TransactionKey if an EndTxnReq has been processed. All other - // timestamp intervals will have no associated txnID and will be due to the - // low-water mark. - switch wTxnID { - case txn.ID: - return false, roachpb.ABORT_REASON_ALREADY_COMMITTED_OR_ROLLED_BACK_POSSIBLE_REPLAY - case uuid.UUID{} /* noTxnID */ : - if !wTS.Less(minTxnTS) { + // Compare against the minimum timestamp that the transaction could have + // written intents at. + if !wTS.Less(txnMinTSUpperBound) { + switch wTxnID { + case txnID: + // If we find our own transaction ID then an EndTransaction request + // sent by our coordinator has already been processed. We might be a + // replay, or we raced with an asynchronous abort. Either way, return + // an error. + return false, minCommitTS, roachpb.ABORT_REASON_ALREADY_COMMITTED_OR_ROLLED_BACK_POSSIBLE_REPLAY + case uuid.Nil: // On lease transfers the timestamp cache is reset with the transfer - // time as the low-water mark, so if this replica recently obtained - // the lease, this case will be true for new txns, even if they're - // not a replay. We force these txns to retry. - return false, roachpb.ABORT_REASON_TIMESTAMP_CACHE_REJECTED_POSSIBLE_REPLAY + // time as the low water mark. The timestamp cache may also lose + // information when bumping its low water mark due to memory + // constraints. If this replica recently obtained the lease or if + // the timestamp cache recently bumped its low water mark, this case + // will be true for new txns, even if they're not a replay. We force + // these txns to retry. + return false, minCommitTS, roachpb.ABORT_REASON_TIMESTAMP_CACHE_REJECTED_POSSIBLE_REPLAY + default: + // If we find another transaction's ID then that transaction has + // aborted us before our transaction record was written. It obeyed + // the restriction that it couldn't create a transaction record for + // us, so it bumped the write timestamp cache instead to prevent us + // from ever creating a transaction record. + return false, minCommitTS, roachpb.ABORT_REASON_ABORTED_RECORD_FOUND } - default: - log.Fatalf(context.Background(), "unexpected tscache interval (%s,%s) for txn %s", - wTS, wTxnID, txn) } // Disallow creation or modification of a transaction record if its original @@ -240,10 +356,14 @@ func (r *Replica) CanCreateTxnRecord( // of our intents (which may have been written before our transaction record). // // See #9265. - threshold := r.GetTxnSpanGCThreshold() - if minTxnTS.Less(threshold) { - return false, roachpb.ABORT_REASON_NEW_TXN_RECORD_TOO_OLD + // + // TODO(nvanbenschoten): If we forwarded the Range's entire local txn span in + // the write timestamp cache when we performed a GC then we wouldn't need this + // check at all. Another alternative is that we could rely on the PushTxn(ABORT) + // that the GC queue sends to bump the write timestamp cache on aborted txns. + if !r.GetTxnSpanGCThreshold().Less(txnMinTSUpperBound) { + return false, minCommitTS, roachpb.ABORT_REASON_NEW_TXN_RECORD_TOO_OLD } - return true, 0 + return true, minCommitTS, 0 } diff --git a/pkg/storage/txnwait/txnqueue.go b/pkg/storage/txnwait/txnqueue.go index dfd98ba137ba..a7a58c0a4fcd 100644 --- a/pkg/storage/txnwait/txnqueue.go +++ b/pkg/storage/txnwait/txnqueue.go @@ -717,6 +717,11 @@ func (q *Queue) startQueryPusherTxn( } else if updatedPusher == nil { // No pusher to query; the BeginTransaction hasn't yet created the // pusher's record. Continue in order to backoff and retry. + // TODO(nvanbenschoten): we shouldn't hit this case in a 2.2 + // cluster now that QueryTxn requests synthesize + // transactions from their provided TxnMeta. However, we + // need to keep the logic while we want to support + // compatibility with 2.1 nodes. Remove this in 2.3. log.Event(ctx, "no pusher found; backing off") continue } @@ -806,7 +811,9 @@ func (q *Queue) queryTxnStatus( } br := b.RawResponse() resp := br.Responses[0].GetInner().(*roachpb.QueryTxnResponse) - // ID can be nil if no BeginTransaction has been sent yet. + // ID can be nil if no BeginTransaction has been sent yet and we're talking + // to a 2.1 node. + // TODO(nvanbenschoten): Remove this in 2.3. if updatedTxn := &resp.QueriedTxn; updatedTxn.ID != (uuid.UUID{}) { return updatedTxn, resp.WaitingTxns, nil }