From 16fb58be26c0fbea898dbb8f399e42e0a30fe077 Mon Sep 17 00:00:00 2001 From: Andrew Werner Date: Fri, 15 Mar 2019 12:55:05 -0400 Subject: [PATCH 1/4] storage: fix TestRangeInfo flake and re-enable follower reads by default This PR addresses a test flake introduced by enabling follower reads in conjunction with #35130 which makes follower reads more generally possible in the face of lease transfer. Fixes #35758. Release note: None --- docs/generated/settings/settings.html | 2 +- pkg/storage/client_replica_test.go | 7 +++++-- pkg/storage/replica_follower_read.go | 2 +- 3 files changed, 7 insertions(+), 4 deletions(-) diff --git a/docs/generated/settings/settings.html b/docs/generated/settings/settings.html index a7fc2552b0f6..63f8c48759e9 100644 --- a/docs/generated/settings/settings.html +++ b/docs/generated/settings/settings.html @@ -33,7 +33,7 @@ kv.bulk_io_write.max_ratebyte size8.0 EiBthe rate limit (bytes/sec) to use for writes to disk on behalf of bulk io ops kv.bulk_sst.sync_sizebyte size2.0 MiBthreshold after which non-Rocks SST writes must fsync (0 disables) kv.closed_timestamp.close_fractionfloat0.2fraction of closed timestamp target duration specifying how frequently the closed timestamp is advanced -kv.closed_timestamp.follower_reads_enabledbooleanfalseallow (all) replicas to serve consistent historical reads based on closed timestamp information +kv.closed_timestamp.follower_reads_enabledbooleantrueallow (all) replicas to serve consistent historical reads based on closed timestamp information kv.closed_timestamp.target_durationduration30sif nonzero, attempt to provide closed timestamp notifications for timestamps trailing cluster time by approximately this duration kv.follower_read.target_multiplefloat3if above 1, encourages the distsender to perform a read against the closest replica if a request is older than kv.closed_timestamp.target_duration * (1 + kv.closed_timestamp.close_fraction * this) less a clock uncertainty interval. This value also is used to create follower_timestamp(). (WARNING: may compromise cluster stability or correctness; do not edit without supervision) kv.import.batch_sizebyte size32 MiBthe maximum size of the payload in an AddSSTable request (WARNING: may compromise cluster stability or correctness; do not edit without supervision) diff --git a/pkg/storage/client_replica_test.go b/pkg/storage/client_replica_test.go index e1cbf26d3b8f..a57373a2ba6f 100644 --- a/pkg/storage/client_replica_test.go +++ b/pkg/storage/client_replica_test.go @@ -1603,8 +1603,11 @@ func TestRangeInfo(t *testing.T) { if pErr != nil { t.Fatal(pErr) } - lhsLease, _ = lhsReplica1.GetLease() - rhsLease, _ = rhsReplica1.GetLease() + // Read the expected lease from replica0 rather than replica1 as it may serve + // a follower read which will contain the new lease information before + // replica1 has applied the lease transfer. + lhsLease, _ = lhsReplica0.GetLease() + rhsLease, _ = rhsReplica0.GetLease() expRangeInfos = []roachpb.RangeInfo{ { Desc: *lhsReplica1.Desc(), diff --git a/pkg/storage/replica_follower_read.go b/pkg/storage/replica_follower_read.go index 59bdcb747741..7ec03d95f927 100644 --- a/pkg/storage/replica_follower_read.go +++ b/pkg/storage/replica_follower_read.go @@ -32,7 +32,7 @@ import ( var FollowerReadsEnabled = settings.RegisterBoolSetting( "kv.closed_timestamp.follower_reads_enabled", "allow (all) replicas to serve consistent historical reads based on closed timestamp information", - false, + true, ) // canServeFollowerRead tests, when a range lease could not be From 44d9dab4526c02b4a64718dca95d6101c2ee66cc Mon Sep 17 00:00:00 2001 From: Andrei Matei Date: Thu, 28 Feb 2019 14:27:50 -0500 Subject: [PATCH 2/4] kv: remove unused arg Release note: None --- pkg/kv/txn_coord_sender.go | 13 ++----------- 1 file changed, 2 insertions(+), 11 deletions(-) diff --git a/pkg/kv/txn_coord_sender.go b/pkg/kv/txn_coord_sender.go index c43761781252..8ff43900e428 100644 --- a/pkg/kv/txn_coord_sender.go +++ b/pkg/kv/txn_coord_sender.go @@ -761,7 +761,7 @@ func (tc *TxnCoordSender) Send( // Send the command through the txnInterceptor stack. br, pErr := tc.interceptorStack[0].SendLocked(ctx, ba) - pErr = tc.updateStateLocked(ctx, startNs, ba, br, pErr) + pErr = tc.updateStateLocked(ctx, ba, br, pErr) // If we succeeded to commit, or we attempted to rollback, we move to // txnFinalized. @@ -986,17 +986,8 @@ func (tc *TxnCoordSender) handleRetryableErrLocked( // updateStateLocked updates the transaction state in both the success and error // cases. It also updates retryable errors with the updated transaction for use // by client restarts. -// -// startNS is the time when the request that's updating the state has been sent. -// This is not used if the request is known to not be the one in charge of -// starting tracking the transaction - i.e. this is the case for DistSQL, which -// just does reads and passes 0. func (tc *TxnCoordSender) updateStateLocked( - ctx context.Context, - startNS int64, - ba roachpb.BatchRequest, - br *roachpb.BatchResponse, - pErr *roachpb.Error, + ctx context.Context, ba roachpb.BatchRequest, br *roachpb.BatchResponse, pErr *roachpb.Error, ) *roachpb.Error { // We handle a couple of different cases: From eaeb4b845b09c161c75cb15ff1b4eec364121526 Mon Sep 17 00:00:00 2001 From: Ben Darnell Date: Mon, 18 Mar 2019 11:27:44 -0400 Subject: [PATCH 3/4] roachtest: Skip flaky jepsen nemesis See #35599 Release note: None --- pkg/cmd/roachtest/jepsen.go | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/pkg/cmd/roachtest/jepsen.go b/pkg/cmd/roachtest/jepsen.go index c8329042bb5d..5467e2fb249c 100644 --- a/pkg/cmd/roachtest/jepsen.go +++ b/pkg/cmd/roachtest/jepsen.go @@ -34,9 +34,11 @@ var jepsenNemeses = []struct { {"start-kill-2", "--nemesis start-kill-2"}, {"start-stop-2", "--nemesis start-stop-2"}, {"strobe-skews", "--nemesis strobe-skews"}, - {"subcritical-skews", "--nemesis subcritical-skews"}, - {"majority-ring-subcritical-skews", "--nemesis majority-ring --nemesis2 subcritical-skews"}, - {"subcritical-skews-start-kill-2", "--nemesis subcritical-skews --nemesis2 start-kill-2"}, + // TODO(bdarnell): subcritical-skews nemesis is currently flaky due to ntp rate limiting. + // https://github.com/cockroachdb/cockroach/issues/35599 + //{"subcritical-skews", "--nemesis subcritical-skews"}, + //{"majority-ring-subcritical-skews", "--nemesis majority-ring --nemesis2 subcritical-skews"}, + //{"subcritical-skews-start-kill-2", "--nemesis subcritical-skews --nemesis2 start-kill-2"}, {"majority-ring-start-kill-2", "--nemesis majority-ring --nemesis2 start-kill-2"}, {"parts-start-kill-2", "--nemesis parts --nemesis2 start-kill-2"}, } From 38e796199562090d2df956ad643e71b25ee25915 Mon Sep 17 00:00:00 2001 From: Andrei Matei Date: Wed, 27 Feb 2019 18:26:27 -0500 Subject: [PATCH 4/4] storage,kv: make transaction deadline exceeded errors retriable Before this patch, they were opaque TransactionStatusErrors. The belief is that we should only be seeing such errors when a transaction is pushed by minutes. Shockingly, this seems to hapen enough in our tests, for example as described here: https://github.com/cockroachdb/cockroach/issues/18684#issuecomment-428433335 This patch marks the error as retriable, since it technically is. This patch also changes the semantics of the EndTransactionRequest.Deadline field to make it exclusive so that it matches the nature of SQL leases. No migration needed. Touches #18684 Release note (sql change): "transaction deadline exceeded" errors are now returned to the client with a retriable code. --- pkg/kv/dist_sender_server_test.go | 2 +- pkg/kv/txn_coord_sender.go | 28 +- pkg/kv/txn_coord_sender_test.go | 33 +- pkg/kv/txn_interceptor_committer.go | 6 +- pkg/kv/txn_interceptor_pipeliner.go | 4 +- pkg/kv/txn_interceptor_pipeliner_test.go | 2 +- pkg/kv/txn_interceptor_span_refresher.go | 4 +- pkg/roachpb/api.pb.go | 252 +++++---- pkg/roachpb/api.proto | 16 +- pkg/roachpb/errors.go | 23 +- pkg/roachpb/errors.pb.go | 541 +++++++++++-------- pkg/roachpb/errors.proto | 5 + pkg/sql/as_of_test.go | 3 +- pkg/sql/distsql_running_test.go | 2 +- pkg/sql/lease_test.go | 20 +- pkg/sql/txn_restart_test.go | 2 +- pkg/storage/batcheval/cmd_end_transaction.go | 45 +- pkg/storage/batcheval/cmd_query_intent.go | 4 +- pkg/storage/client_merge_test.go | 9 +- pkg/storage/replica_test.go | 19 +- pkg/storage/replica_write.go | 4 +- pkg/storage/store.go | 3 +- 22 files changed, 564 insertions(+), 463 deletions(-) diff --git a/pkg/kv/dist_sender_server_test.go b/pkg/kv/dist_sender_server_test.go index 0d51723f663e..592a7a7c643d 100644 --- a/pkg/kv/dist_sender_server_test.go +++ b/pkg/kv/dist_sender_server_test.go @@ -1919,7 +1919,7 @@ func TestTxnCoordSenderRetries(t *testing.T) { if atomic.AddInt32(&count, 1) > 1 { return nil } - err := roachpb.NewTransactionRetryError(reason) + err := roachpb.NewTransactionRetryError(reason, "filter err") return roachpb.NewErrorWithTxn(err, fArgs.Hdr.Txn) } return nil diff --git a/pkg/kv/txn_coord_sender.go b/pkg/kv/txn_coord_sender.go index 8ff43900e428..53a5068d0bb3 100644 --- a/pkg/kv/txn_coord_sender.go +++ b/pkg/kv/txn_coord_sender.go @@ -664,6 +664,20 @@ func (tc *TxnCoordSender) DisablePipelining() error { return nil } +func generateTxnDeadlineExceededErr( + txn *roachpb.Transaction, deadline hlc.Timestamp, +) *roachpb.Error { + exceededBy := txn.Timestamp.GoTime().Sub(deadline.GoTime()) + fromStart := txn.Timestamp.GoTime().Sub(txn.OrigTimestamp.GoTime()) + extraMsg := fmt.Sprintf( + "txn timestamp pushed too much; deadline exceeded by %s (%s > %s), "+ + "original timestamp %s ago (%s)", + exceededBy, txn.Timestamp, deadline, fromStart, txn.OrigTimestamp) + txnCpy := txn.Clone() + return roachpb.NewErrorWithTxn( + roachpb.NewTransactionRetryError(roachpb.RETRY_COMMIT_DEADLINE_EXCEEDED, extraMsg), &txnCpy) +} + // commitReadOnlyTxnLocked "commits" a read-only txn. It is equivalent, but // cheaper than, sending an EndTransactionRequest. A read-only txn doesn't have // a transaction record, so there's no need to send any request to the server. @@ -674,11 +688,15 @@ func (tc *TxnCoordSender) DisablePipelining() error { // sendLockedWithElidedEndTransaction method, but we would want to confirm // that doing so doesn't cut into the speed-up we see from this fast-path. func (tc *TxnCoordSender) commitReadOnlyTxnLocked( - ctx context.Context, deadline *hlc.Timestamp, + ctx context.Context, ba roachpb.BatchRequest, ) *roachpb.Error { - if deadline != nil && deadline.Less(tc.mu.txn.Timestamp) { - return roachpb.NewError( - roachpb.NewTransactionStatusError("deadline exceeded before transaction finalization")) + deadline := ba.Requests[0].GetEndTransaction().Deadline + txn := tc.mu.txn + if deadline != nil && !txn.Timestamp.Less(*deadline) { + pErr := generateTxnDeadlineExceededErr(&txn, *deadline) + // We need to bump the epoch and transform this retriable error. + ba.Txn = &txn + return tc.updateStateLocked(ctx, ba, nil /* br */, pErr) } tc.mu.txnState = txnFinalized // Mark the transaction as committed so that, in case this commit is done by @@ -706,7 +724,7 @@ func (tc *TxnCoordSender) Send( } if ba.IsSingleEndTransactionRequest() && !tc.interceptorAlloc.txnIntentCollector.haveIntents() { - return nil, tc.commitReadOnlyTxnLocked(ctx, ba.Requests[0].GetEndTransaction().Deadline) + return nil, tc.commitReadOnlyTxnLocked(ctx, ba) } startNs := tc.clock.PhysicalNow() diff --git a/pkg/kv/txn_coord_sender_test.go b/pkg/kv/txn_coord_sender_test.go index 7407aba9229d..f679b0bda3a4 100644 --- a/pkg/kv/txn_coord_sender_test.go +++ b/pkg/kv/txn_coord_sender_test.go @@ -505,16 +505,13 @@ func TestTxnCoordSenderEndTxn(t *testing.T) { case 1: // Past deadline. - if err := roachpb.CheckTxnDeadlineExceededErr(err); err != nil { - t.Fatal(err) - } - + fallthrough case 2: // Equal deadline. - if err != nil { - t.Fatal(err) + assertTransactionRetryError(t, err) + if !testutils.IsError(err, "RETRY_COMMIT_DEADLINE_EXCEEDED") { + t.Fatalf("expected deadline exceeded, got: %s", err) } - case 3: // Future deadline. if err != nil { @@ -1876,7 +1873,9 @@ func TestEndWriteRestartReadOnlyTransaction(t *testing.T) { sender.match(func(ba roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { calls = append(calls, ba.Methods()...) if _, ok := ba.GetArg(roachpb.Put); ok { - return nil, roachpb.NewErrorWithTxn(roachpb.NewTransactionRetryError(roachpb.RETRY_SERIALIZABLE), ba.Txn) + return nil, roachpb.NewErrorWithTxn( + roachpb.NewTransactionRetryError(roachpb.RETRY_SERIALIZABLE, "test err"), + ba.Txn) } return nil, nil }) @@ -1955,7 +1954,9 @@ func TestTransactionKeyNotChangedInRestart(t *testing.T) { } if attempt == 0 { - return nil, roachpb.NewErrorWithTxn(roachpb.NewTransactionRetryError(roachpb.RETRY_SERIALIZABLE), ba.Txn) + return nil, roachpb.NewErrorWithTxn( + roachpb.NewTransactionRetryError(roachpb.RETRY_SERIALIZABLE, "test err"), + ba.Txn) } return nil, nil }) @@ -2199,9 +2200,10 @@ func TestReadOnlyTxnObeysDeadline(t *testing.T) { if _, err := txn.Get(ctx, "k"); err != nil { t.Fatal(err) } - if err := txn.Commit(ctx); !testutils.IsError( - err, "deadline exceeded before transaction finalization") { - t.Fatal(err) + err := txn.Commit(ctx) + assertTransactionRetryError(t, err) + if !testutils.IsError(err, "RETRY_COMMIT_DEADLINE_EXCEEDED") { + t.Fatalf("expected deadline exceeded, got: %s", err) } }) @@ -2211,9 +2213,10 @@ func TestReadOnlyTxnObeysDeadline(t *testing.T) { txn.UpdateDeadlineMaybe(ctx, clock.Now()) b := txn.NewBatch() b.Get("k") - if err := txn.CommitInBatch(ctx, b); !testutils.IsError( - err, "deadline exceeded before transaction finalization") { - t.Fatal(err) + err := txn.CommitInBatch(ctx, b) + assertTransactionRetryError(t, err) + if !testutils.IsError(err, "RETRY_COMMIT_DEADLINE_EXCEEDED") { + t.Fatalf("expected deadline exceeded, got: %s", err) } }) } diff --git a/pkg/kv/txn_interceptor_committer.go b/pkg/kv/txn_interceptor_committer.go index ad4c69caef2b..a67ec5001b56 100644 --- a/pkg/kv/txn_interceptor_committer.go +++ b/pkg/kv/txn_interceptor_committer.go @@ -91,9 +91,9 @@ func (tc *txnCommitter) sendLockedWithElidedEndTransaction( } // Check if the (read-only) txn was pushed above its deadline. - if et.Deadline != nil && et.Deadline.Less(br.Txn.Timestamp) { - return nil, roachpb.NewErrorWithTxn(roachpb.NewTransactionStatusError( - "deadline exceeded before transaction finalization"), br.Txn) + deadline := et.Deadline + if deadline != nil && !br.Txn.Timestamp.Less(*deadline) { + return nil, generateTxnDeadlineExceededErr(ba.Txn, *deadline) } // Update the response's transaction proto. This normally happens on the diff --git a/pkg/kv/txn_interceptor_pipeliner.go b/pkg/kv/txn_interceptor_pipeliner.go index cc63d6989ff7..fbd04096496b 100644 --- a/pkg/kv/txn_interceptor_pipeliner.go +++ b/pkg/kv/txn_interceptor_pipeliner.go @@ -16,6 +16,7 @@ package kv import ( "context" + "fmt" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings" @@ -426,7 +427,8 @@ func (tp *txnPipeliner) adjustError( // Turn an IntentMissingError into a transactional retry error. if ime, ok := pErr.GetDetail().(*roachpb.IntentMissingError); ok { log.VEventf(ctx, 2, "transforming intent missing error into retry: %v", ime) - err := roachpb.NewTransactionRetryError(roachpb.RETRY_ASYNC_WRITE_FAILURE) + err := roachpb.NewTransactionRetryError( + roachpb.RETRY_ASYNC_WRITE_FAILURE, fmt.Sprintf("missing intent on: %s", ime.Key)) retryErr := roachpb.NewErrorWithTxn(err, pErr.GetTxn()) retryErr.Index = pErr.Index return retryErr diff --git a/pkg/kv/txn_interceptor_pipeliner_test.go b/pkg/kv/txn_interceptor_pipeliner_test.go index c2f183f3720a..0c98b5362bc4 100644 --- a/pkg/kv/txn_interceptor_pipeliner_test.go +++ b/pkg/kv/txn_interceptor_pipeliner_test.go @@ -718,7 +718,7 @@ func TestTxnPipelinerIntentMissingError(t *testing.T) { require.IsType(t, &roachpb.QueryIntentRequest{}, ba.Requests[5].GetInner()) require.IsType(t, &roachpb.PutRequest{}, ba.Requests[6].GetInner()) - err := roachpb.NewIntentMissingError(nil) + err := roachpb.NewIntentMissingError(nil /* key */, nil /* intent */) pErr := roachpb.NewErrorWithTxn(err, &txn) pErr.SetErrorIndex(errIdx) return nil, pErr diff --git a/pkg/kv/txn_interceptor_span_refresher.go b/pkg/kv/txn_interceptor_span_refresher.go index 42c4d7d0dc5c..9044016a7afc 100644 --- a/pkg/kv/txn_interceptor_span_refresher.go +++ b/pkg/kv/txn_interceptor_span_refresher.go @@ -117,7 +117,9 @@ func (sr *txnSpanRefresher) SendLocked( if !sr.appendRefreshSpans(ctx, ba, br) { // The refresh spans are out of date, return a generic client-side retry error. return nil, roachpb.NewErrorWithTxn( - roachpb.NewTransactionRetryError(roachpb.RETRY_SERIALIZABLE), br.Txn, + roachpb.NewTransactionRetryError( + roachpb.RETRY_SERIALIZABLE, "refresh spans are out of date", + ), br.Txn, ) } } diff --git a/pkg/roachpb/api.pb.go b/pkg/roachpb/api.pb.go index 82414b914f4b..b616efadc070 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_e2607f120b04f650, []int{0} + return fileDescriptor_api_a3d9ae2b746aa069, []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_e2607f120b04f650, []int{1} + return fileDescriptor_api_a3d9ae2b746aa069, []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_e2607f120b04f650, []int{2} + return fileDescriptor_api_a3d9ae2b746aa069, []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_e2607f120b04f650, []int{3} + return fileDescriptor_api_a3d9ae2b746aa069, []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_e2607f120b04f650, []int{4} + return fileDescriptor_api_a3d9ae2b746aa069, []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_e2607f120b04f650, []int{2, 0} + return fileDescriptor_api_a3d9ae2b746aa069, []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_e2607f120b04f650, []int{50, 0} + return fileDescriptor_api_a3d9ae2b746aa069, []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_e2607f120b04f650, []int{0} + return fileDescriptor_api_a3d9ae2b746aa069, []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_e2607f120b04f650, []int{1} + return fileDescriptor_api_a3d9ae2b746aa069, []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_e2607f120b04f650, []int{2} + return fileDescriptor_api_a3d9ae2b746aa069, []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_e2607f120b04f650, []int{3} + return fileDescriptor_api_a3d9ae2b746aa069, []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_e2607f120b04f650, []int{4} + return fileDescriptor_api_a3d9ae2b746aa069, []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_e2607f120b04f650, []int{5} + return fileDescriptor_api_a3d9ae2b746aa069, []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_e2607f120b04f650, []int{6} + return fileDescriptor_api_a3d9ae2b746aa069, []int{6} } func (m *PutResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -591,7 +591,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_e2607f120b04f650, []int{7} + return fileDescriptor_api_a3d9ae2b746aa069, []int{7} } func (m *ConditionalPutRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -628,7 +628,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_e2607f120b04f650, []int{8} + return fileDescriptor_api_a3d9ae2b746aa069, []int{8} } func (m *ConditionalPutResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -676,7 +676,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_e2607f120b04f650, []int{9} + return fileDescriptor_api_a3d9ae2b746aa069, []int{9} } func (m *InitPutRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -712,7 +712,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_e2607f120b04f650, []int{10} + return fileDescriptor_api_a3d9ae2b746aa069, []int{10} } func (m *InitPutResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -754,7 +754,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_e2607f120b04f650, []int{11} + return fileDescriptor_api_a3d9ae2b746aa069, []int{11} } func (m *IncrementRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -793,7 +793,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_e2607f120b04f650, []int{12} + return fileDescriptor_api_a3d9ae2b746aa069, []int{12} } func (m *IncrementResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -829,7 +829,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_e2607f120b04f650, []int{13} + return fileDescriptor_api_a3d9ae2b746aa069, []int{13} } func (m *DeleteRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -865,7 +865,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_e2607f120b04f650, []int{14} + return fileDescriptor_api_a3d9ae2b746aa069, []int{14} } func (m *DeleteResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -916,7 +916,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_e2607f120b04f650, []int{15} + return fileDescriptor_api_a3d9ae2b746aa069, []int{15} } func (m *DeleteRangeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -955,7 +955,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_e2607f120b04f650, []int{16} + return fileDescriptor_api_a3d9ae2b746aa069, []int{16} } func (m *DeleteRangeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1004,7 +1004,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_e2607f120b04f650, []int{17} + return fileDescriptor_api_a3d9ae2b746aa069, []int{17} } func (m *ClearRangeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1040,7 +1040,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_e2607f120b04f650, []int{18} + return fileDescriptor_api_a3d9ae2b746aa069, []int{18} } func (m *ClearRangeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1111,7 +1111,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_e2607f120b04f650, []int{19} + return fileDescriptor_api_a3d9ae2b746aa069, []int{19} } func (m *ScanOptions) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1153,7 +1153,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_e2607f120b04f650, []int{20} + return fileDescriptor_api_a3d9ae2b746aa069, []int{20} } func (m *ScanRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1202,7 +1202,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_e2607f120b04f650, []int{21} + return fileDescriptor_api_a3d9ae2b746aa069, []int{21} } func (m *ScanResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1244,7 +1244,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_e2607f120b04f650, []int{22} + return fileDescriptor_api_a3d9ae2b746aa069, []int{22} } func (m *ReverseScanRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1293,7 +1293,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_e2607f120b04f650, []int{23} + return fileDescriptor_api_a3d9ae2b746aa069, []int{23} } func (m *ReverseScanResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1334,7 +1334,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_e2607f120b04f650, []int{24} + return fileDescriptor_api_a3d9ae2b746aa069, []int{24} } func (m *CheckConsistencyRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1371,7 +1371,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_e2607f120b04f650, []int{25} + return fileDescriptor_api_a3d9ae2b746aa069, []int{25} } func (m *CheckConsistencyResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1421,7 +1421,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_e2607f120b04f650, []int{26} + return fileDescriptor_api_a3d9ae2b746aa069, []int{26} } func (m *RecomputeStatsRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1459,7 +1459,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_e2607f120b04f650, []int{27} + return fileDescriptor_api_a3d9ae2b746aa069, []int{27} } func (m *RecomputeStatsResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1495,7 +1495,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_e2607f120b04f650, []int{28} + return fileDescriptor_api_a3d9ae2b746aa069, []int{28} } func (m *BeginTransactionRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1531,7 +1531,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_e2607f120b04f650, []int{29} + return fileDescriptor_api_a3d9ae2b746aa069, []int{29} } func (m *BeginTransactionResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1562,16 +1562,12 @@ type EndTransactionRequest struct { RequestHeader `protobuf:"bytes,1,opt,name=header,proto3,embedded=header" json:"header"` // False to abort and rollback. Commit bool `protobuf:"varint,2,opt,name=commit,proto3" json:"commit,omitempty"` - // If set, deadline represents the maximum timestamp at which the transaction - // can commit (i.e. the maximum timestamp for the txn's writes). If - // EndTransaction(Commit=true) finds that the txn's timestamp has been pushed - // above this deadline, an error will be returned and the client is supposed - // to rollback the txn. - // N.B. Assuming that the deadline was valid to begin with (i.e. it was higher - // than the txn's OrigTimestamp), only Snapshot transactions can get in - // trouble with the deadline check. A Serializable txn that has had its - // timestamp pushed has already lost before the deadline check: it will be - // forced to restart. + // If set, deadline represents the maximum (exclusive) timestamp at which the + // transaction can commit (i.e. the maximum timestamp for the txn's reads and + // writes). + // If EndTransaction(Commit=true) finds that the txn's timestamp has been + // pushed above this deadline, an error will be returned and the client is + // supposed to rollback the txn. Deadline *hlc.Timestamp `protobuf:"bytes,3,opt,name=deadline,proto3" json:"deadline,omitempty"` // commit triggers. Note that commit triggers are for // internal use only and will cause an error if requested through the @@ -1605,7 +1601,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_e2607f120b04f650, []int{30} + return fileDescriptor_api_a3d9ae2b746aa069, []int{30} } func (m *EndTransactionRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1651,7 +1647,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_e2607f120b04f650, []int{31} + return fileDescriptor_api_a3d9ae2b746aa069, []int{31} } func (m *EndTransactionResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1707,7 +1703,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_e2607f120b04f650, []int{32} + return fileDescriptor_api_a3d9ae2b746aa069, []int{32} } func (m *AdminSplitRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1744,7 +1740,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_e2607f120b04f650, []int{33} + return fileDescriptor_api_a3d9ae2b746aa069, []int{33} } func (m *AdminSplitResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1788,7 +1784,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_e2607f120b04f650, []int{34} + return fileDescriptor_api_a3d9ae2b746aa069, []int{34} } func (m *AdminMergeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1825,7 +1821,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_e2607f120b04f650, []int{35} + return fileDescriptor_api_a3d9ae2b746aa069, []int{35} } func (m *AdminMergeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1865,7 +1861,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_e2607f120b04f650, []int{36} + return fileDescriptor_api_a3d9ae2b746aa069, []int{36} } func (m *AdminTransferLeaseRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1900,7 +1896,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_e2607f120b04f650, []int{37} + return fileDescriptor_api_a3d9ae2b746aa069, []int{37} } func (m *AdminTransferLeaseResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1940,7 +1936,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_e2607f120b04f650, []int{38} + return fileDescriptor_api_a3d9ae2b746aa069, []int{38} } func (m *AdminChangeReplicasRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1975,7 +1971,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_e2607f120b04f650, []int{39} + return fileDescriptor_api_a3d9ae2b746aa069, []int{39} } func (m *AdminChangeReplicasResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2014,7 +2010,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_e2607f120b04f650, []int{40} + return fileDescriptor_api_a3d9ae2b746aa069, []int{40} } func (m *AdminRelocateRangeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2049,7 +2045,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_e2607f120b04f650, []int{41} + return fileDescriptor_api_a3d9ae2b746aa069, []int{41} } func (m *AdminRelocateRangeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2090,7 +2086,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_e2607f120b04f650, []int{42} + return fileDescriptor_api_a3d9ae2b746aa069, []int{42} } func (m *HeartbeatTxnRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2129,7 +2125,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_e2607f120b04f650, []int{43} + return fileDescriptor_api_a3d9ae2b746aa069, []int{43} } func (m *HeartbeatTxnResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2174,7 +2170,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_e2607f120b04f650, []int{44} + return fileDescriptor_api_a3d9ae2b746aa069, []int{44} } func (m *GCRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2210,7 +2206,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_e2607f120b04f650, []int{44, 0} + return fileDescriptor_api_a3d9ae2b746aa069, []int{44, 0} } func (m *GCRequest_GCKey) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2246,7 +2242,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_e2607f120b04f650, []int{45} + return fileDescriptor_api_a3d9ae2b746aa069, []int{45} } func (m *GCResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2330,7 +2326,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_e2607f120b04f650, []int{46} + return fileDescriptor_api_a3d9ae2b746aa069, []int{46} } func (m *PushTxnRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2375,7 +2371,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_e2607f120b04f650, []int{47} + return fileDescriptor_api_a3d9ae2b746aa069, []int{47} } func (m *PushTxnResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2421,7 +2417,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_e2607f120b04f650, []int{48} + return fileDescriptor_api_a3d9ae2b746aa069, []int{48} } func (m *QueryTxnRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2462,7 +2458,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_e2607f120b04f650, []int{49} + return fileDescriptor_api_a3d9ae2b746aa069, []int{49} } func (m *QueryTxnResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2524,7 +2520,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_e2607f120b04f650, []int{50} + return fileDescriptor_api_a3d9ae2b746aa069, []int{50} } func (m *QueryIntentRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2562,7 +2558,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_e2607f120b04f650, []int{51} + return fileDescriptor_api_a3d9ae2b746aa069, []int{51} } func (m *QueryIntentResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2608,7 +2604,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_e2607f120b04f650, []int{52} + return fileDescriptor_api_a3d9ae2b746aa069, []int{52} } func (m *ResolveIntentRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2645,7 +2641,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_e2607f120b04f650, []int{53} + return fileDescriptor_api_a3d9ae2b746aa069, []int{53} } func (m *ResolveIntentResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2695,7 +2691,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_e2607f120b04f650, []int{54} + return fileDescriptor_api_a3d9ae2b746aa069, []int{54} } func (m *ResolveIntentRangeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2732,7 +2728,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_e2607f120b04f650, []int{55} + return fileDescriptor_api_a3d9ae2b746aa069, []int{55} } func (m *ResolveIntentRangeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2771,7 +2767,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_e2607f120b04f650, []int{56} + return fileDescriptor_api_a3d9ae2b746aa069, []int{56} } func (m *MergeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2807,7 +2803,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_e2607f120b04f650, []int{57} + return fileDescriptor_api_a3d9ae2b746aa069, []int{57} } func (m *MergeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2854,7 +2850,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_e2607f120b04f650, []int{58} + return fileDescriptor_api_a3d9ae2b746aa069, []int{58} } func (m *TruncateLogRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2890,7 +2886,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_e2607f120b04f650, []int{59} + return fileDescriptor_api_a3d9ae2b746aa069, []int{59} } func (m *TruncateLogResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2936,7 +2932,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_e2607f120b04f650, []int{60} + return fileDescriptor_api_a3d9ae2b746aa069, []int{60} } func (m *RequestLeaseRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2987,7 +2983,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_e2607f120b04f650, []int{61} + return fileDescriptor_api_a3d9ae2b746aa069, []int{61} } func (m *TransferLeaseRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3026,7 +3022,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_e2607f120b04f650, []int{62} + return fileDescriptor_api_a3d9ae2b746aa069, []int{62} } func (m *LeaseInfoRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3065,7 +3061,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_e2607f120b04f650, []int{63} + return fileDescriptor_api_a3d9ae2b746aa069, []int{63} } func (m *LeaseInfoResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3102,7 +3098,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_e2607f120b04f650, []int{64} + return fileDescriptor_api_a3d9ae2b746aa069, []int{64} } func (m *RequestLeaseResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3147,7 +3143,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_e2607f120b04f650, []int{65} + return fileDescriptor_api_a3d9ae2b746aa069, []int{65} } func (m *ComputeChecksumRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3186,7 +3182,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_e2607f120b04f650, []int{66} + return fileDescriptor_api_a3d9ae2b746aa069, []int{66} } func (m *ComputeChecksumResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3227,7 +3223,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_e2607f120b04f650, []int{67} + return fileDescriptor_api_a3d9ae2b746aa069, []int{67} } func (m *ExportStorage) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3263,7 +3259,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_e2607f120b04f650, []int{67, 0} + return fileDescriptor_api_a3d9ae2b746aa069, []int{67, 0} } func (m *ExportStorage_LocalFilePath) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3298,7 +3294,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_e2607f120b04f650, []int{67, 1} + return fileDescriptor_api_a3d9ae2b746aa069, []int{67, 1} } func (m *ExportStorage_Http) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3339,7 +3335,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_e2607f120b04f650, []int{67, 2} + return fileDescriptor_api_a3d9ae2b746aa069, []int{67, 2} } func (m *ExportStorage_S3) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3380,7 +3376,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_e2607f120b04f650, []int{67, 3} + return fileDescriptor_api_a3d9ae2b746aa069, []int{67, 3} } func (m *ExportStorage_GCS) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3418,7 +3414,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_e2607f120b04f650, []int{67, 4} + return fileDescriptor_api_a3d9ae2b746aa069, []int{67, 4} } func (m *ExportStorage_Azure) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3459,7 +3455,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_e2607f120b04f650, []int{67, 5} + return fileDescriptor_api_a3d9ae2b746aa069, []int{67, 5} } func (m *ExportStorage_Workload) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3501,7 +3497,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_e2607f120b04f650, []int{68} + return fileDescriptor_api_a3d9ae2b746aa069, []int{68} } func (m *WriteBatchRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3537,7 +3533,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_e2607f120b04f650, []int{69} + return fileDescriptor_api_a3d9ae2b746aa069, []int{69} } func (m *WriteBatchResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3584,7 +3580,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_e2607f120b04f650, []int{70} + return fileDescriptor_api_a3d9ae2b746aa069, []int{70} } func (m *ExportRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3622,7 +3618,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_e2607f120b04f650, []int{71} + return fileDescriptor_api_a3d9ae2b746aa069, []int{71} } func (m *BulkOpSummary) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3660,7 +3656,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_e2607f120b04f650, []int{72} + return fileDescriptor_api_a3d9ae2b746aa069, []int{72} } func (m *ExportResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3701,7 +3697,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_e2607f120b04f650, []int{72, 0} + return fileDescriptor_api_a3d9ae2b746aa069, []int{72, 0} } func (m *ExportResponse_File) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3753,7 +3749,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_e2607f120b04f650, []int{73} + return fileDescriptor_api_a3d9ae2b746aa069, []int{73} } func (m *ImportRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3790,7 +3786,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_e2607f120b04f650, []int{73, 0} + return fileDescriptor_api_a3d9ae2b746aa069, []int{73, 0} } func (m *ImportRequest_File) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3828,7 +3824,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_e2607f120b04f650, []int{73, 1} + return fileDescriptor_api_a3d9ae2b746aa069, []int{73, 1} } func (m *ImportRequest_TableRekey) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3865,7 +3861,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_e2607f120b04f650, []int{74} + return fileDescriptor_api_a3d9ae2b746aa069, []int{74} } func (m *ImportResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3905,7 +3901,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_e2607f120b04f650, []int{75} + return fileDescriptor_api_a3d9ae2b746aa069, []int{75} } func (m *AdminScatterRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3942,7 +3938,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_e2607f120b04f650, []int{76} + return fileDescriptor_api_a3d9ae2b746aa069, []int{76} } func (m *AdminScatterResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3977,7 +3973,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_e2607f120b04f650, []int{76, 0} + return fileDescriptor_api_a3d9ae2b746aa069, []int{76, 0} } func (m *AdminScatterResponse_Range) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4015,7 +4011,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_e2607f120b04f650, []int{77} + return fileDescriptor_api_a3d9ae2b746aa069, []int{77} } func (m *AddSSTableRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4051,7 +4047,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_e2607f120b04f650, []int{78} + return fileDescriptor_api_a3d9ae2b746aa069, []int{78} } func (m *AddSSTableResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4097,7 +4093,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_e2607f120b04f650, []int{79} + return fileDescriptor_api_a3d9ae2b746aa069, []int{79} } func (m *RefreshRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4133,7 +4129,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_e2607f120b04f650, []int{80} + return fileDescriptor_api_a3d9ae2b746aa069, []int{80} } func (m *RefreshResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4175,7 +4171,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_e2607f120b04f650, []int{81} + return fileDescriptor_api_a3d9ae2b746aa069, []int{81} } func (m *RefreshRangeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4211,7 +4207,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_e2607f120b04f650, []int{82} + return fileDescriptor_api_a3d9ae2b746aa069, []int{82} } func (m *RefreshRangeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4259,7 +4255,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_e2607f120b04f650, []int{83} + return fileDescriptor_api_a3d9ae2b746aa069, []int{83} } func (m *SubsumeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4306,7 +4302,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_e2607f120b04f650, []int{84} + return fileDescriptor_api_a3d9ae2b746aa069, []int{84} } func (m *SubsumeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4343,7 +4339,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_e2607f120b04f650, []int{85} + return fileDescriptor_api_a3d9ae2b746aa069, []int{85} } func (m *RangeStatsRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4384,7 +4380,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_e2607f120b04f650, []int{86} + return fileDescriptor_api_a3d9ae2b746aa069, []int{86} } func (m *RangeStatsResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4466,7 +4462,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_e2607f120b04f650, []int{87} + return fileDescriptor_api_a3d9ae2b746aa069, []int{87} } func (m *RequestUnion) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -5825,7 +5821,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_e2607f120b04f650, []int{88} + return fileDescriptor_api_a3d9ae2b746aa069, []int{88} } func (m *ResponseUnion) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -7174,7 +7170,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_e2607f120b04f650, []int{89} + return fileDescriptor_api_a3d9ae2b746aa069, []int{89} } func (m *Header) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -7212,7 +7208,7 @@ type BatchRequest struct { func (m *BatchRequest) Reset() { *m = BatchRequest{} } func (*BatchRequest) ProtoMessage() {} func (*BatchRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e2607f120b04f650, []int{90} + return fileDescriptor_api_a3d9ae2b746aa069, []int{90} } func (m *BatchRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -7251,7 +7247,7 @@ type BatchResponse struct { func (m *BatchResponse) Reset() { *m = BatchResponse{} } func (*BatchResponse) ProtoMessage() {} func (*BatchResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e2607f120b04f650, []int{91} + return fileDescriptor_api_a3d9ae2b746aa069, []int{91} } func (m *BatchResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -7312,7 +7308,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_e2607f120b04f650, []int{91, 0} + return fileDescriptor_api_a3d9ae2b746aa069, []int{91, 0} } func (m *BatchResponse_Header) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -7350,7 +7346,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_e2607f120b04f650, []int{92} + return fileDescriptor_api_a3d9ae2b746aa069, []int{92} } func (m *RangeFeedRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -7388,7 +7384,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_e2607f120b04f650, []int{93} + return fileDescriptor_api_a3d9ae2b746aa069, []int{93} } func (m *RangeFeedValue) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -7428,7 +7424,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_e2607f120b04f650, []int{94} + return fileDescriptor_api_a3d9ae2b746aa069, []int{94} } func (m *RangeFeedCheckpoint) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -7467,7 +7463,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_e2607f120b04f650, []int{95} + return fileDescriptor_api_a3d9ae2b746aa069, []int{95} } func (m *RangeFeedError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -7506,7 +7502,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_e2607f120b04f650, []int{96} + return fileDescriptor_api_a3d9ae2b746aa069, []int{96} } func (m *RangeFeedEvent) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -33588,9 +33584,9 @@ var ( ErrIntOverflowApi = fmt.Errorf("proto: integer overflow") ) -func init() { proto.RegisterFile("roachpb/api.proto", fileDescriptor_api_e2607f120b04f650) } +func init() { proto.RegisterFile("roachpb/api.proto", fileDescriptor_api_a3d9ae2b746aa069) } -var fileDescriptor_api_e2607f120b04f650 = []byte{ +var fileDescriptor_api_a3d9ae2b746aa069 = []byte{ // 6252 bytes of a gzipped FileDescriptorProto 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xe4, 0x3d, 0x4b, 0x6c, 0x23, 0xc9, 0x75, 0x6a, 0x92, 0x92, 0xc8, 0x47, 0x91, 0x6a, 0x95, 0xe6, 0xc3, 0xd1, 0xcc, 0x8e, 0x34, 0x9c, diff --git a/pkg/roachpb/api.proto b/pkg/roachpb/api.proto index 14bc77196de6..d055ebfe304a 100644 --- a/pkg/roachpb/api.proto +++ b/pkg/roachpb/api.proto @@ -489,16 +489,12 @@ message EndTransactionRequest { RequestHeader header = 1 [(gogoproto.nullable) = false, (gogoproto.embed) = true]; // False to abort and rollback. bool commit = 2; - // If set, deadline represents the maximum timestamp at which the transaction - // can commit (i.e. the maximum timestamp for the txn's writes). If - // EndTransaction(Commit=true) finds that the txn's timestamp has been pushed - // above this deadline, an error will be returned and the client is supposed - // to rollback the txn. - // N.B. Assuming that the deadline was valid to begin with (i.e. it was higher - // than the txn's OrigTimestamp), only Snapshot transactions can get in - // trouble with the deadline check. A Serializable txn that has had its - // timestamp pushed has already lost before the deadline check: it will be - // forced to restart. + // If set, deadline represents the maximum (exclusive) timestamp at which the + // transaction can commit (i.e. the maximum timestamp for the txn's reads and + // writes). + // If EndTransaction(Commit=true) finds that the txn's timestamp has been + // pushed above this deadline, an error will be returned and the client is + // supposed to rollback the txn. util.hlc.Timestamp deadline = 3; // commit triggers. Note that commit triggers are for // internal use only and will cause an error if requested through the diff --git a/pkg/roachpb/errors.go b/pkg/roachpb/errors.go index 0299b605b6e7..30db996bdea6 100644 --- a/pkg/roachpb/errors.go +++ b/pkg/roachpb/errors.go @@ -22,7 +22,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/caller" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/uuid" - "github.com/pkg/errors" ) func (e *UnhandledRetryableError) Error() string { @@ -390,9 +389,12 @@ func (*TransactionPushError) canRestartTransaction() TransactionRestart { } // NewTransactionRetryError initializes a new TransactionRetryError. -func NewTransactionRetryError(reason TransactionRetryReason) *TransactionRetryError { +func NewTransactionRetryError( + reason TransactionRetryReason, extraMsg string, +) *TransactionRetryError { return &TransactionRetryError{ - Reason: reason, + Reason: reason, + ExtraMsg: extraMsg, } } @@ -437,18 +439,6 @@ func (e *TransactionStatusError) message(pErr *Error) string { return fmt.Sprintf("%s: %s", e.Error(), pErr.GetTxn()) } -// CheckTxnDeadlineExceededErr returns an error if deadlineErr is not a -// transaction deadline exceeded roachpb.TransactionStatusError. -func CheckTxnDeadlineExceededErr(deadlineErr error) error { - if statusError, ok := deadlineErr.(*TransactionStatusError); !ok { - return errors.Errorf("expected TransactionStatusError but got %T: %s", - deadlineErr, deadlineErr) - } else if e := "transaction deadline exceeded"; !strings.Contains(statusError.Msg, e) { - return errors.Errorf("expected %s, got %s", e, statusError.Msg) - } - return nil -} - var _ ErrorDetailInterface = &TransactionStatusError{} func (e *WriteIntentError) Error() string { @@ -692,8 +682,9 @@ func (e *BatchTimestampBeforeGCError) message(_ *Error) string { var _ ErrorDetailInterface = &BatchTimestampBeforeGCError{} // NewIntentMissingError creates a new IntentMissingError. -func NewIntentMissingError(wrongIntent *Intent) *IntentMissingError { +func NewIntentMissingError(key Key, wrongIntent *Intent) *IntentMissingError { return &IntentMissingError{ + Key: key, WrongIntent: wrongIntent, } } diff --git a/pkg/roachpb/errors.pb.go b/pkg/roachpb/errors.pb.go index 20137309abc9..9dc082f10c90 100644 --- a/pkg/roachpb/errors.pb.go +++ b/pkg/roachpb/errors.pb.go @@ -114,7 +114,7 @@ func (x *TransactionAbortedReason) UnmarshalJSON(data []byte) error { return nil } func (TransactionAbortedReason) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_errors_2f91914b0d06d64b, []int{0} + return fileDescriptor_errors_ee43f5f6d0e525bf, []int{0} } // TransactionRetryReason specifies what caused a transaction retry. @@ -133,6 +133,8 @@ const ( RETRY_POSSIBLE_REPLAY TransactionRetryReason = 4 // An asynchronous write was observed to have failed. RETRY_ASYNC_WRITE_FAILURE TransactionRetryReason = 5 + // The transaction exceeded its deadline. + RETRY_COMMIT_DEADLINE_EXCEEDED TransactionRetryReason = 6 ) var TransactionRetryReason_name = map[int32]string{ @@ -141,13 +143,15 @@ var TransactionRetryReason_name = map[int32]string{ 3: "RETRY_SERIALIZABLE", 4: "RETRY_POSSIBLE_REPLAY", 5: "RETRY_ASYNC_WRITE_FAILURE", + 6: "RETRY_COMMIT_DEADLINE_EXCEEDED", } var TransactionRetryReason_value = map[string]int32{ - "RETRY_REASON_UNKNOWN": 0, - "RETRY_WRITE_TOO_OLD": 1, - "RETRY_SERIALIZABLE": 3, - "RETRY_POSSIBLE_REPLAY": 4, - "RETRY_ASYNC_WRITE_FAILURE": 5, + "RETRY_REASON_UNKNOWN": 0, + "RETRY_WRITE_TOO_OLD": 1, + "RETRY_SERIALIZABLE": 3, + "RETRY_POSSIBLE_REPLAY": 4, + "RETRY_ASYNC_WRITE_FAILURE": 5, + "RETRY_COMMIT_DEADLINE_EXCEEDED": 6, } func (x TransactionRetryReason) Enum() *TransactionRetryReason { @@ -167,7 +171,7 @@ func (x *TransactionRetryReason) UnmarshalJSON(data []byte) error { return nil } func (TransactionRetryReason) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_errors_2f91914b0d06d64b, []int{1} + return fileDescriptor_errors_ee43f5f6d0e525bf, []int{1} } // TransactionRestart indicates how an error should be handled in a @@ -218,7 +222,7 @@ func (x *TransactionRestart) UnmarshalJSON(data []byte) error { return nil } func (TransactionRestart) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_errors_2f91914b0d06d64b, []int{2} + return fileDescriptor_errors_ee43f5f6d0e525bf, []int{2} } // Reason specifies what caused the error. @@ -261,7 +265,7 @@ func (x *TransactionStatusError_Reason) UnmarshalJSON(data []byte) error { return nil } func (TransactionStatusError_Reason) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_errors_2f91914b0d06d64b, []int{9, 0} + return fileDescriptor_errors_ee43f5f6d0e525bf, []int{9, 0} } // Reason specifies what caused the error. @@ -317,7 +321,7 @@ func (x *RangeFeedRetryError_Reason) UnmarshalJSON(data []byte) error { return nil } func (RangeFeedRetryError_Reason) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_errors_2f91914b0d06d64b, []int{29, 0} + return fileDescriptor_errors_ee43f5f6d0e525bf, []int{29, 0} } // A NotLeaseHolderError indicates that the current range is not the lease @@ -346,7 +350,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_2f91914b0d06d64b, []int{0} + return fileDescriptor_errors_ee43f5f6d0e525bf, []int{0} } func (m *NotLeaseHolderError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -383,7 +387,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_2f91914b0d06d64b, []int{1} + return fileDescriptor_errors_ee43f5f6d0e525bf, []int{1} } func (m *NodeUnavailableError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -419,7 +423,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_2f91914b0d06d64b, []int{2} + return fileDescriptor_errors_ee43f5f6d0e525bf, []int{2} } func (m *UnsupportedRequestError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -458,7 +462,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_2f91914b0d06d64b, []int{3} + return fileDescriptor_errors_ee43f5f6d0e525bf, []int{3} } func (m *RangeNotFoundError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -504,7 +508,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_2f91914b0d06d64b, []int{4} + return fileDescriptor_errors_ee43f5f6d0e525bf, []int{4} } func (m *RangeKeyMismatchError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -552,7 +556,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_2f91914b0d06d64b, []int{5} + return fileDescriptor_errors_ee43f5f6d0e525bf, []int{5} } func (m *ReadWithinUncertaintyIntervalError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -597,7 +601,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_2f91914b0d06d64b, []int{6} + return fileDescriptor_errors_ee43f5f6d0e525bf, []int{6} } func (m *TransactionAbortedError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -635,7 +639,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_2f91914b0d06d64b, []int{7} + return fileDescriptor_errors_ee43f5f6d0e525bf, []int{7} } func (m *TransactionPushError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -664,6 +668,7 @@ var xxx_messageInfo_TransactionPushError proto.InternalMessageInfo // retried, usually with an increased transaction timestamp. type TransactionRetryError struct { Reason TransactionRetryReason `protobuf:"varint,1,opt,name=reason,enum=cockroach.roachpb.TransactionRetryReason" json:"reason"` + ExtraMsg string `protobuf:"bytes,2,opt,name=extra_msg,json=extraMsg" json:"extra_msg"` XXX_NoUnkeyedLiteral struct{} `json:"-"` XXX_sizecache int32 `json:"-"` } @@ -672,7 +677,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_2f91914b0d06d64b, []int{8} + return fileDescriptor_errors_ee43f5f6d0e525bf, []int{8} } func (m *TransactionRetryError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -714,7 +719,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_2f91914b0d06d64b, []int{9} + return fileDescriptor_errors_ee43f5f6d0e525bf, []int{9} } func (m *TransactionStatusError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -754,7 +759,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_2f91914b0d06d64b, []int{10} + return fileDescriptor_errors_ee43f5f6d0e525bf, []int{10} } func (m *WriteIntentError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -794,7 +799,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_2f91914b0d06d64b, []int{11} + return fileDescriptor_errors_ee43f5f6d0e525bf, []int{11} } func (m *WriteTooOldError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -832,7 +837,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_2f91914b0d06d64b, []int{12} + return fileDescriptor_errors_ee43f5f6d0e525bf, []int{12} } func (m *OpRequiresTxnError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -871,7 +876,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_2f91914b0d06d64b, []int{13} + return fileDescriptor_errors_ee43f5f6d0e525bf, []int{13} } func (m *ConditionFailedError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -910,7 +915,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_2f91914b0d06d64b, []int{14} + return fileDescriptor_errors_ee43f5f6d0e525bf, []int{14} } func (m *LeaseRejectedError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -947,7 +952,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_2f91914b0d06d64b, []int{15} + return fileDescriptor_errors_ee43f5f6d0e525bf, []int{15} } func (m *SendError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -987,7 +992,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_2f91914b0d06d64b, []int{16} + return fileDescriptor_errors_ee43f5f6d0e525bf, []int{16} } func (m *AmbiguousResultError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1023,7 +1028,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_2f91914b0d06d64b, []int{17} + return fileDescriptor_errors_ee43f5f6d0e525bf, []int{17} } func (m *RaftGroupDeletedError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1063,7 +1068,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_2f91914b0d06d64b, []int{18} + return fileDescriptor_errors_ee43f5f6d0e525bf, []int{18} } func (m *ReplicaCorruptionError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1102,7 +1107,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_2f91914b0d06d64b, []int{19} + return fileDescriptor_errors_ee43f5f6d0e525bf, []int{19} } func (m *ReplicaTooOldError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1139,7 +1144,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_2f91914b0d06d64b, []int{20} + return fileDescriptor_errors_ee43f5f6d0e525bf, []int{20} } func (m *StoreNotFoundError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1187,7 +1192,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_2f91914b0d06d64b, []int{21} + return fileDescriptor_errors_ee43f5f6d0e525bf, []int{21} } func (m *UnhandledRetryableError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1241,7 +1246,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_2f91914b0d06d64b, []int{22} + return fileDescriptor_errors_ee43f5f6d0e525bf, []int{22} } func (m *TransactionRetryWithProtoRefreshError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1281,7 +1286,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_2f91914b0d06d64b, []int{23} + return fileDescriptor_errors_ee43f5f6d0e525bf, []int{23} } func (m *TxnAlreadyEncounteredErrorError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1320,7 +1325,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_2f91914b0d06d64b, []int{24} + return fileDescriptor_errors_ee43f5f6d0e525bf, []int{24} } func (m *IntegerOverflowError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1358,7 +1363,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_2f91914b0d06d64b, []int{25} + return fileDescriptor_errors_ee43f5f6d0e525bf, []int{25} } func (m *MixedSuccessError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1396,7 +1401,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_2f91914b0d06d64b, []int{26} + return fileDescriptor_errors_ee43f5f6d0e525bf, []int{26} } func (m *BatchTimestampBeforeGCError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1426,7 +1431,9 @@ var xxx_messageInfo_BatchTimestampBeforeGCError proto.InternalMessageInfo // not there. type IntentMissingError struct { // The non-matching intent that was found at that key, if any. - WrongIntent *Intent `protobuf:"bytes,1,opt,name=wrong_intent,json=wrongIntent" json:"wrong_intent,omitempty"` + WrongIntent *Intent `protobuf:"bytes,1,opt,name=wrong_intent,json=wrongIntent" json:"wrong_intent,omitempty"` + // The key where the intent was expected. + Key Key `protobuf:"bytes,2,opt,name=key,casttype=Key" json:"key,omitempty"` XXX_NoUnkeyedLiteral struct{} `json:"-"` XXX_sizecache int32 `json:"-"` } @@ -1435,7 +1442,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_2f91914b0d06d64b, []int{27} + return fileDescriptor_errors_ee43f5f6d0e525bf, []int{27} } func (m *IntentMissingError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1474,7 +1481,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_2f91914b0d06d64b, []int{28} + return fileDescriptor_errors_ee43f5f6d0e525bf, []int{28} } func (m *MergeInProgressError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1511,7 +1518,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_2f91914b0d06d64b, []int{29} + return fileDescriptor_errors_ee43f5f6d0e525bf, []int{29} } func (m *RangeFeedRetryError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1577,7 +1584,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_2f91914b0d06d64b, []int{30} + return fileDescriptor_errors_ee43f5f6d0e525bf, []int{30} } func (m *ErrorDetail) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2537,7 +2544,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_2f91914b0d06d64b, []int{31} + return fileDescriptor_errors_ee43f5f6d0e525bf, []int{31} } func (m *ErrPosition) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2593,7 +2600,7 @@ type Error struct { func (m *Error) Reset() { *m = Error{} } func (*Error) ProtoMessage() {} func (*Error) Descriptor() ([]byte, []int) { - return fileDescriptor_errors_2f91914b0d06d64b, []int{32} + return fileDescriptor_errors_ee43f5f6d0e525bf, []int{32} } func (m *Error) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2904,6 +2911,9 @@ func (this *TransactionRetryError) Equal(that interface{}) bool { if this.Reason != that1.Reason { return false } + if this.ExtraMsg != that1.ExtraMsg { + return false + } return true } func (this *TransactionStatusError) Equal(that interface{}) bool { @@ -3368,6 +3378,9 @@ func (this *IntentMissingError) Equal(that interface{}) bool { if !this.WrongIntent.Equal(that1.WrongIntent) { return false } + if !bytes.Equal(this.Key, that1.Key) { + return false + } return true } func (this *MergeInProgressError) Equal(that interface{}) bool { @@ -4491,6 +4504,10 @@ func (m *TransactionRetryError) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x8 i++ i = encodeVarintErrors(dAtA, i, uint64(m.Reason)) + dAtA[i] = 0x12 + i++ + i = encodeVarintErrors(dAtA, i, uint64(len(m.ExtraMsg))) + i += copy(dAtA[i:], m.ExtraMsg) return i, nil } @@ -5014,6 +5031,12 @@ func (m *IntentMissingError) MarshalTo(dAtA []byte) (int, error) { } i += n22 } + if m.Key != nil { + dAtA[i] = 0x12 + i++ + i = encodeVarintErrors(dAtA, i, uint64(len(m.Key))) + i += copy(dAtA[i:], m.Key) + } return i, nil } @@ -5736,6 +5759,8 @@ func (m *TransactionRetryError) Size() (n int) { var l int _ = l n += 1 + sovErrors(uint64(m.Reason)) + l = len(m.ExtraMsg) + n += 1 + l + sovErrors(uint64(l)) return n } @@ -5971,6 +5996,10 @@ func (m *IntentMissingError) Size() (n int) { l = m.WrongIntent.Size() n += 1 + l + sovErrors(uint64(l)) } + if m.Key != nil { + l = len(m.Key) + n += 1 + l + sovErrors(uint64(l)) + } return n } @@ -7332,6 +7361,35 @@ func (m *TransactionRetryError) Unmarshal(dAtA []byte) error { break } } + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ExtraMsg", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowErrors + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthErrors + } + postIndex := iNdEx + intStringLen + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.ExtraMsg = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipErrors(dAtA[iNdEx:]) @@ -9064,6 +9122,37 @@ func (m *IntentMissingError) Unmarshal(dAtA []byte) error { return err } iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Key", wireType) + } + var byteLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowErrors + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + byteLen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if byteLen < 0 { + return ErrInvalidLengthErrors + } + postIndex := iNdEx + byteLen + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Key = append(m.Key[:0], dAtA[iNdEx:postIndex]...) + if m.Key == nil { + m.Key = []byte{} + } + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipErrors(dAtA[iNdEx:]) @@ -10599,185 +10688,187 @@ var ( ErrIntOverflowErrors = fmt.Errorf("proto: integer overflow") ) -func init() { proto.RegisterFile("roachpb/errors.proto", fileDescriptor_errors_2f91914b0d06d64b) } - -var fileDescriptor_errors_2f91914b0d06d64b = []byte{ - // 2819 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x9c, 0x58, 0xcb, 0x6f, 0xe3, 0xd6, - 0xd5, 0x17, 0x65, 0xd9, 0xb2, 0x8f, 0x5f, 0xf4, 0x1d, 0xc7, 0xc3, 0xf1, 0x64, 0x64, 0x8f, 0x27, - 0x93, 0x4c, 0xe6, 0x43, 0xec, 0x0f, 0x93, 0x6f, 0xf0, 0x35, 0xd3, 0x64, 0xa1, 0x07, 0x6d, 0xc9, - 0xd6, 0xc3, 0xa1, 0xe4, 0x38, 0x93, 0xb4, 0x20, 0x68, 0xf1, 0x5a, 0x66, 0x46, 0x22, 0xd5, 0x4b, - 0xd2, 0x8f, 0x5d, 0x97, 0x41, 0x57, 0x2d, 0xd0, 0x45, 0x77, 0x0d, 0xd0, 0x55, 0xd0, 0x6d, 0x91, - 0x7f, 0xa0, 0x9b, 0x2c, 0xbb, 0x0c, 0xba, 0x18, 0xb4, 0x53, 0xa0, 0x28, 0xda, 0xff, 0x20, 0xab, - 0xe2, 0x3e, 0x48, 0x91, 0x16, 0xe9, 0x51, 0xb2, 0x92, 0x78, 0x5e, 0xf7, 0xdc, 0x73, 0xcf, 0x3d, - 0xe7, 0x77, 0x2e, 0xac, 0x12, 0xc7, 0xe8, 0x9e, 0x0d, 0x4f, 0x76, 0x30, 0x21, 0x0e, 0x71, 0xb7, - 0x87, 0xc4, 0xf1, 0x1c, 0xb4, 0xd2, 0x75, 0xba, 0x2f, 0x18, 0x67, 0x5b, 0xf0, 0xd7, 0x51, 0x20, - 0x68, 0x1a, 0x9e, 0xc1, 0xc5, 0xd6, 0xd7, 0x02, 0xda, 0x00, 0x7b, 0x46, 0x84, 0xfe, 0xc0, 0xf5, - 0x1c, 0x62, 0xf4, 0xf0, 0x0e, 0xb6, 0x7b, 0x96, 0x1d, 0xfc, 0x50, 0xb9, 0xf3, 0x6e, 0xf7, 0x7d, - 0x21, 0xa4, 0xf8, 0x9e, 0xd5, 0xdf, 0x39, 0xeb, 0x77, 0x77, 0x3c, 0x6b, 0x80, 0x5d, 0xcf, 0x18, - 0x0c, 0x05, 0x67, 0xb5, 0xe7, 0xf4, 0x1c, 0xf6, 0x77, 0x87, 0xfe, 0xe3, 0xd4, 0xad, 0x6f, 0xb2, - 0x70, 0xab, 0xe9, 0x78, 0x75, 0x6c, 0xb8, 0xb8, 0xea, 0xf4, 0x4d, 0x4c, 0x54, 0xea, 0x32, 0xaa, - 0x40, 0x9e, 0xe0, 0x61, 0xdf, 0xea, 0x1a, 0x8a, 0xb4, 0x29, 0x3d, 0x9a, 0x7f, 0xf2, 0xd6, 0xf6, - 0x98, 0xf7, 0xdb, 0x1a, 0x97, 0xa8, 0x60, 0xb7, 0x4b, 0xac, 0xa1, 0xe7, 0x90, 0x52, 0xee, 0xdb, - 0x97, 0x1b, 0x19, 0x2d, 0x50, 0x45, 0x7b, 0xb0, 0xd0, 0xa7, 0x96, 0xf5, 0x33, 0x66, 0x5a, 0xc9, - 0x4e, 0x6e, 0x4a, 0x9b, 0xef, 0x8f, 0x7c, 0x42, 0x4f, 0x61, 0x96, 0x18, 0x76, 0x0f, 0xeb, 0x96, - 0xa9, 0x4c, 0x6d, 0x4a, 0x8f, 0xa6, 0x4a, 0xeb, 0x74, 0xa5, 0x57, 0x2f, 0x37, 0xf2, 0x1a, 0xa5, - 0xd7, 0x2a, 0xdf, 0x8f, 0xfe, 0x6a, 0x79, 0x26, 0x5b, 0x33, 0xd1, 0x36, 0x4c, 0x33, 0x2b, 0x4a, - 0x8e, 0x2d, 0xac, 0x24, 0x2c, 0xcc, 0x76, 0xae, 0x71, 0x31, 0xf4, 0x00, 0xa0, 0xeb, 0xbb, 0x9e, - 0x33, 0xd0, 0x07, 0x6e, 0x4f, 0x99, 0xde, 0x94, 0x1e, 0xcd, 0x89, 0x2d, 0xcd, 0x71, 0x7a, 0xc3, - 0xed, 0x3d, 0xcb, 0xfd, 0xeb, 0xab, 0x0d, 0x69, 0xeb, 0x4d, 0x58, 0x6d, 0x3a, 0x26, 0x3e, 0xb2, - 0x8d, 0x73, 0xc3, 0xea, 0x1b, 0x27, 0x7d, 0xcc, 0x02, 0x27, 0xb8, 0x1b, 0x70, 0xfb, 0xc8, 0x76, - 0xfd, 0xe1, 0xd0, 0x21, 0x1e, 0x36, 0x35, 0xfc, 0x0b, 0x1f, 0xbb, 0x5e, 0x54, 0xe0, 0x4b, 0x09, - 0x10, 0x73, 0xb7, 0xe9, 0x78, 0xbb, 0x8e, 0x6f, 0x9b, 0x3c, 0xec, 0xd1, 0x7d, 0x4a, 0x93, 0xef, - 0xf3, 0x29, 0xcc, 0xd2, 0xe4, 0x60, 0x6a, 0xd9, 0xb8, 0x5a, 0x9b, 0xd2, 0xb9, 0x9a, 0xf8, 0xab, - 0xe5, 0x99, 0x6c, 0xcd, 0x14, 0xae, 0xfc, 0x3e, 0x0b, 0x6f, 0x30, 0x8b, 0x07, 0xf8, 0xaa, 0x61, - 0xb9, 0x03, 0xc3, 0xeb, 0x9e, 0x71, 0x6f, 0xde, 0x87, 0x15, 0xc2, 0x5d, 0xd7, 0x5d, 0xcf, 0x20, - 0x9e, 0xfe, 0x02, 0x5f, 0x31, 0xb7, 0x16, 0x4a, 0xf9, 0xef, 0x5f, 0x6e, 0x4c, 0x1d, 0xe0, 0x2b, - 0x6d, 0x59, 0x48, 0xb4, 0xa9, 0xc0, 0x01, 0xbe, 0x42, 0x3b, 0x10, 0x90, 0x74, 0x6c, 0x9b, 0x4c, - 0x25, 0x1b, 0x57, 0x59, 0x14, 0x7c, 0xd5, 0x36, 0xa9, 0x42, 0x03, 0xe4, 0x81, 0x58, 0x16, 0x9b, - 0x3a, 0xdb, 0x12, 0x3b, 0xe3, 0xf9, 0x27, 0x5b, 0x49, 0x89, 0x42, 0xf9, 0x91, 0x34, 0x59, 0x1e, - 0xe9, 0x32, 0x16, 0x3a, 0x80, 0x65, 0xd7, 0xef, 0xf5, 0xb0, 0xeb, 0x85, 0xd6, 0x72, 0x13, 0x5b, - 0x5b, 0x0a, 0x55, 0x19, 0x47, 0x44, 0xe8, 0x3f, 0x59, 0xd8, 0xd2, 0xb0, 0x61, 0x1e, 0x5b, 0xde, - 0x99, 0x65, 0x1f, 0xd9, 0x5d, 0x4c, 0x3c, 0xc3, 0xb2, 0xbd, 0xab, 0x9a, 0xed, 0x61, 0x72, 0x6e, - 0xf4, 0x79, 0xb8, 0xf6, 0x61, 0x89, 0x60, 0xc3, 0xd4, 0xc3, 0x9b, 0x27, 0xae, 0xce, 0xbd, 0xc8, - 0xc2, 0xf4, 0x7a, 0x6e, 0x9f, 0xf5, 0xbb, 0xdb, 0x9d, 0x40, 0x48, 0x24, 0xd8, 0x22, 0x55, 0x0d, - 0x89, 0x48, 0x03, 0x84, 0x2f, 0x2d, 0xd7, 0xb3, 0xec, 0x5e, 0xc4, 0x5e, 0x76, 0x72, 0x7b, 0x2b, - 0x81, 0xfa, 0xc8, 0x66, 0x09, 0x16, 0x07, 0xc6, 0x65, 0xc4, 0xdc, 0xd4, 0x04, 0xe6, 0xb4, 0x85, - 0x81, 0x71, 0x39, 0xb2, 0xf1, 0x39, 0xdc, 0x72, 0x4e, 0x5c, 0x4c, 0xce, 0x71, 0x64, 0x9f, 0xae, - 0x92, 0xdb, 0x9c, 0x4a, 0xb9, 0xd8, 0x2d, 0x21, 0x7d, 0xdd, 0x3f, 0xe4, 0x5c, 0x67, 0xb8, 0x22, - 0xda, 0x5f, 0xc0, 0xed, 0x0e, 0x31, 0x6c, 0xd7, 0xe8, 0x7a, 0x96, 0x63, 0x17, 0x4f, 0xd8, 0x15, - 0xe2, 0x11, 0xae, 0xc1, 0x0c, 0xc1, 0x86, 0xeb, 0xd8, 0x2c, 0xb2, 0x4b, 0x4f, 0xfe, 0x27, 0x61, - 0xc1, 0x71, 0x5d, 0x8d, 0xa9, 0x88, 0x75, 0x85, 0x01, 0xb1, 0x96, 0x01, 0xab, 0x11, 0xf9, 0x43, - 0xdf, 0x15, 0x99, 0x5f, 0x06, 0x18, 0xfa, 0xee, 0x19, 0xc6, 0xba, 0x77, 0x69, 0x8b, 0x63, 0x2c, - 0xdc, 0xbc, 0x58, 0x50, 0x28, 0xb8, 0x5e, 0xe7, 0x32, 0x58, 0xe2, 0x14, 0xde, 0x88, 0x48, 0x69, - 0xd8, 0x23, 0x57, 0x7c, 0x8d, 0xbd, 0x6b, 0x9b, 0x79, 0xf7, 0x66, 0xfb, 0x4c, 0xf3, 0x86, 0xad, - 0x7c, 0x27, 0xc1, 0x5a, 0x44, 0xbc, 0xed, 0x19, 0x9e, 0xef, 0xf2, 0x95, 0xd6, 0x60, 0x8a, 0xd6, - 0x33, 0x29, 0x52, 0xcf, 0x28, 0x01, 0x35, 0x43, 0x0f, 0xb2, 0xcc, 0x83, 0xff, 0xbd, 0xd9, 0x83, - 0x88, 0xc9, 0xed, 0x24, 0x47, 0xb6, 0x0e, 0x61, 0x86, 0xd3, 0x11, 0x82, 0x25, 0x4d, 0x2d, 0xb6, - 0x5b, 0x4d, 0xfd, 0xa8, 0x79, 0xd0, 0x6c, 0x1d, 0x37, 0xe5, 0x0c, 0x52, 0x60, 0x55, 0xd0, 0x3a, - 0x9f, 0x36, 0xf5, 0x66, 0xab, 0xa3, 0xef, 0xb6, 0x8e, 0x9a, 0x15, 0x59, 0xba, 0xc6, 0x29, 0xb7, - 0x1a, 0x8d, 0x5a, 0xa7, 0xa3, 0x56, 0xe4, 0xac, 0xd8, 0xda, 0x73, 0x90, 0x8f, 0x89, 0xe5, 0x61, - 0x7a, 0xdd, 0x6c, 0x5e, 0x46, 0xd1, 0x07, 0x90, 0xb7, 0xd8, 0xa7, 0xab, 0x48, 0x2c, 0xf9, 0xee, - 0x24, 0x38, 0xcf, 0x15, 0x82, 0xae, 0x24, 0xe4, 0xb9, 0xd1, 0xfd, 0xdc, 0x6c, 0x56, 0x9e, 0xda, - 0xfa, 0xa3, 0x24, 0x6c, 0x77, 0x1c, 0xa7, 0xd5, 0x17, 0x69, 0x56, 0x84, 0xb9, 0x1f, 0x75, 0x87, - 0x47, 0x5a, 0xa8, 0x09, 0xb2, 0xd1, 0xf5, 0x7c, 0xa3, 0xff, 0xe3, 0x6e, 0xef, 0x32, 0x57, 0x0e, - 0xc9, 0x22, 0x10, 0xeb, 0x80, 0x5a, 0x43, 0xda, 0x4d, 0x2c, 0x82, 0xdd, 0xce, 0xa5, 0x1d, 0xed, - 0x28, 0xcf, 0x61, 0xb5, 0xec, 0xd8, 0xa6, 0x45, 0x4f, 0x6a, 0xd7, 0xb0, 0xfa, 0xc1, 0x9d, 0xf9, - 0x29, 0x2c, 0x08, 0x4f, 0xce, 0x8d, 0xbe, 0x8f, 0xc5, 0x7e, 0x92, 0x5a, 0xe1, 0x27, 0x94, 0xaf, - 0xcd, 0x73, 0x69, 0xf6, 0x21, 0x4c, 0xff, 0x49, 0x02, 0xc4, 0xfb, 0x24, 0xfe, 0x02, 0x77, 0xc3, - 0xdb, 0x58, 0x80, 0xfc, 0x00, 0xbb, 0xae, 0xd1, 0xc3, 0xb1, 0xd4, 0x0a, 0x88, 0xe8, 0x43, 0x98, - 0x13, 0x95, 0x1e, 0x9b, 0x62, 0xf3, 0xa9, 0x1d, 0x38, 0x88, 0x60, 0xa8, 0x80, 0x9e, 0xc1, 0x6c, - 0x50, 0xc2, 0x44, 0xa1, 0x7a, 0x9d, 0x72, 0x28, 0x2f, 0xdc, 0xfe, 0x7f, 0x98, 0x6b, 0x63, 0x7b, - 0x32, 0x67, 0x63, 0x49, 0x71, 0x01, 0xab, 0xc5, 0xc1, 0x89, 0xd5, 0xf3, 0x1d, 0xdf, 0xd5, 0xb0, - 0xeb, 0xf7, 0xbd, 0xc9, 0x36, 0xfc, 0x01, 0xcc, 0x5f, 0x10, 0x63, 0x38, 0xc4, 0xa6, 0x8e, 0x09, - 0xb9, 0x61, 0xcb, 0xcc, 0x9c, 0x06, 0x42, 0x58, 0x25, 0xc1, 0x19, 0xde, 0xa3, 0x9d, 0xf8, 0xd4, - 0xdb, 0x23, 0x8e, 0x3f, 0xac, 0xe0, 0x3e, 0x0e, 0x42, 0x2d, 0xd8, 0x18, 0xd6, 0x04, 0x4e, 0x2a, - 0x3b, 0x84, 0xf8, 0x43, 0x7a, 0xd4, 0xdc, 0xb3, 0xfb, 0x30, 0xc7, 0xa0, 0xa6, 0x7e, 0xfd, 0x9e, - 0xcf, 0x32, 0x72, 0xc3, 0xed, 0xa1, 0x2d, 0x98, 0x1b, 0x12, 0xa7, 0x8b, 0x5d, 0x57, 0x9c, 0xc6, - 0x6c, 0x58, 0xb1, 0x02, 0x72, 0x98, 0x49, 0x48, 0x2c, 0x13, 0xbd, 0x14, 0x1f, 0x01, 0x08, 0x58, - 0x17, 0x80, 0x93, 0xe9, 0x52, 0x41, 0xa0, 0x8c, 0x39, 0x21, 0xcf, 0x70, 0xc6, 0xe8, 0x83, 0x1e, - 0x27, 0xff, 0x1b, 0x98, 0xfe, 0x18, 0x10, 0x43, 0x21, 0x63, 0xa8, 0x27, 0x84, 0x2f, 0xd2, 0x0f, - 0x85, 0x2f, 0x0d, 0x0a, 0xb5, 0xce, 0x0c, 0xdb, 0xec, 0xd3, 0x4a, 0xef, 0x91, 0xab, 0x10, 0x8b, - 0xa1, 0x27, 0x90, 0x1b, 0xaa, 0x84, 0xdc, 0x90, 0xf2, 0x4c, 0x4e, 0xc4, 0x81, 0xc9, 0x6e, 0xfd, - 0x5b, 0x82, 0x87, 0xd7, 0xab, 0x2e, 0xed, 0xfb, 0x87, 0x14, 0x2e, 0x6b, 0xf8, 0x94, 0xe0, 0xa0, - 0x47, 0xa4, 0x55, 0xd5, 0xcf, 0x61, 0xc6, 0xbb, 0xb4, 0x03, 0x28, 0xb6, 0x50, 0xaa, 0x50, 0xd6, - 0x5f, 0x5f, 0x6e, 0xbc, 0xdf, 0xb3, 0xbc, 0x33, 0xff, 0x64, 0xbb, 0xeb, 0x0c, 0x76, 0x42, 0x4f, - 0xcc, 0x93, 0xd1, 0xff, 0x9d, 0xe1, 0x8b, 0xde, 0x0e, 0xc3, 0xef, 0xbe, 0x6f, 0x99, 0xdb, 0x47, - 0x47, 0xb5, 0xca, 0xab, 0x97, 0x1b, 0xd3, 0x9d, 0x4b, 0xbb, 0x56, 0xd1, 0xa6, 0xbd, 0x4b, 0xbb, - 0x66, 0xa2, 0x5d, 0x98, 0xf7, 0x46, 0xde, 0x89, 0x8b, 0x31, 0x59, 0x67, 0x8a, 0x2a, 0x8a, 0xd8, - 0xd5, 0x61, 0xa3, 0x73, 0x69, 0x17, 0xfb, 0x14, 0x7b, 0x5c, 0xa9, 0x76, 0xd7, 0xf1, 0x29, 0xa0, - 0x11, 0x49, 0xc7, 0x77, 0xf9, 0x00, 0x60, 0x48, 0xf0, 0xb9, 0xce, 0xf2, 0x28, 0xb6, 0xd9, 0x39, - 0x4a, 0x8f, 0xa6, 0xe7, 0x6f, 0x24, 0x58, 0xa5, 0x15, 0xb7, 0x87, 0x49, 0xeb, 0x1c, 0x93, 0xd3, - 0xbe, 0x73, 0xc1, 0x6d, 0xdc, 0x81, 0xa9, 0x04, 0xe4, 0x48, 0x69, 0xe8, 0x5d, 0x58, 0xec, 0xfa, - 0x84, 0x60, 0xdb, 0x13, 0xe5, 0x89, 0xc3, 0x57, 0xbe, 0xc2, 0x82, 0x60, 0xb1, 0x5a, 0x84, 0xde, - 0x83, 0x65, 0xcb, 0xee, 0x12, 0x3c, 0x18, 0x09, 0x4f, 0x45, 0x84, 0x97, 0x42, 0x66, 0xb4, 0x74, - 0x35, 0x60, 0xa5, 0x61, 0x5d, 0x62, 0xb3, 0xed, 0x77, 0x69, 0x8e, 0x07, 0x79, 0x91, 0x17, 0x57, - 0xef, 0x75, 0xa9, 0xa1, 0x05, 0x82, 0xc2, 0xdc, 0xd7, 0x12, 0xdc, 0x2d, 0x51, 0xb4, 0x39, 0x2a, - 0xd8, 0xf8, 0xd4, 0x21, 0x78, 0xaf, 0x1c, 0x76, 0x8e, 0xce, 0x8f, 0xea, 0x1c, 0x23, 0x84, 0x45, - 0x4d, 0x9c, 0xd1, 0x2c, 0x73, 0xfa, 0xe6, 0x0f, 0x69, 0x19, 0x23, 0x2d, 0xe1, 0xeb, 0xa7, 0x80, - 0x78, 0xff, 0x6b, 0x58, 0xae, 0x6b, 0xd9, 0x3d, 0xee, 0xe1, 0x87, 0xb0, 0x70, 0x41, 0x1c, 0xbb, - 0xa7, 0xf3, 0x6e, 0x28, 0x9c, 0x4c, 0x6f, 0x9e, 0xda, 0x3c, 0x13, 0xe7, 0x1f, 0xa3, 0xd9, 0xa7, - 0x81, 0x49, 0x0f, 0xd7, 0xec, 0x43, 0xe2, 0xf4, 0x48, 0x10, 0x57, 0xc1, 0xfd, 0x6d, 0x16, 0x6e, - 0x31, 0xf4, 0xbc, 0x8b, 0xc5, 0x8d, 0xe4, 0x2b, 0x1f, 0x5c, 0xc3, 0x3b, 0xef, 0xa5, 0xe1, 0xf1, - 0xb8, 0x5e, 0x32, 0xd4, 0xf8, 0x5a, 0x0a, 0xb1, 0xc6, 0x3a, 0xac, 0x09, 0xf4, 0xa0, 0xa9, 0x87, - 0xf5, 0x5a, 0xb9, 0xa8, 0x6b, 0x6a, 0xa3, 0xf5, 0x89, 0x5a, 0x91, 0x33, 0x68, 0x0d, 0x50, 0xc0, - 0x2b, 0x36, 0xf7, 0x54, 0xbd, 0x7d, 0x58, 0xaf, 0x75, 0x64, 0x09, 0xdd, 0x86, 0x5b, 0x31, 0x7a, - 0x43, 0xd5, 0xf6, 0x28, 0xe0, 0x88, 0x40, 0x11, 0xad, 0xb8, 0xdb, 0xd1, 0xdb, 0xcd, 0xe2, 0x61, - 0xbb, 0xda, 0xea, 0xc8, 0x53, 0xa8, 0x00, 0xeb, 0x82, 0x53, 0x6f, 0xed, 0xd5, 0xca, 0xc5, 0xba, - 0xde, 0x3a, 0x6c, 0xeb, 0x8d, 0x5a, 0xbb, 0x5d, 0x6b, 0xee, 0xc9, 0xb9, 0x88, 0x66, 0xbb, 0xde, - 0x3a, 0xd6, 0xcb, 0xad, 0x66, 0xfb, 0xa8, 0xa1, 0x6a, 0xf2, 0xb4, 0x08, 0xcb, 0x9f, 0x57, 0x61, - 0x9e, 0x6d, 0xa8, 0x82, 0x3d, 0xc3, 0xea, 0x23, 0x0d, 0x64, 0xdb, 0xf1, 0xf4, 0xd8, 0x7c, 0xcc, - 0x0f, 0xe3, 0xed, 0x84, 0xc0, 0x24, 0xcc, 0xe8, 0xd5, 0x8c, 0xb6, 0x64, 0xc7, 0xc8, 0xa8, 0x05, - 0xcb, 0x7c, 0x7c, 0xa4, 0x96, 0x4f, 0x69, 0x81, 0x15, 0x19, 0xf4, 0x30, 0x2d, 0xd6, 0xb1, 0x42, - 0x5c, 0xa5, 0x63, 0x48, 0x94, 0x8a, 0x3e, 0x05, 0xc4, 0x0d, 0xbe, 0xc0, 0x57, 0x7a, 0x30, 0x69, - 0x89, 0xaa, 0xf3, 0x28, 0xcd, 0xe6, 0xf5, 0x39, 0xb2, 0x9a, 0xd1, 0x64, 0x72, 0x8d, 0x81, 0x7e, - 0x29, 0xc1, 0x26, 0x9b, 0x96, 0x2e, 0xd8, 0x50, 0xa5, 0xfb, 0xa3, 0xa9, 0x8a, 0xa5, 0x26, 0x1d, - 0xab, 0xc4, 0xe0, 0xf6, 0x34, 0xf1, 0xbd, 0xe0, 0x75, 0xe3, 0x58, 0x35, 0xa3, 0xdd, 0x23, 0x37, - 0x49, 0xa1, 0x9f, 0xc3, 0xad, 0x48, 0x49, 0xd4, 0x0d, 0x3e, 0x2d, 0xb0, 0xb1, 0x7f, 0xfe, 0xc9, - 0xe3, 0x89, 0x46, 0x8b, 0x60, 0x25, 0xe4, 0x8d, 0xb1, 0x50, 0x07, 0xe4, 0xa8, 0x79, 0x3a, 0x17, - 0x28, 0x33, 0xcc, 0xf6, 0x3b, 0x37, 0xdb, 0x0e, 0xc7, 0x90, 0x6a, 0x46, 0x5b, 0xf6, 0xe2, 0x74, - 0x74, 0x0c, 0x2b, 0x51, 0xab, 0x84, 0xde, 0x13, 0x25, 0x9f, 0x7a, 0x20, 0x89, 0xa3, 0x07, 0x3d, - 0x10, 0xef, 0x1a, 0x03, 0x7d, 0x06, 0xd1, 0x4d, 0xd0, 0x81, 0xdf, 0xf3, 0x5d, 0x65, 0x96, 0x59, - 0x7e, 0x77, 0xe2, 0xc1, 0xa0, 0x9a, 0xd1, 0xa2, 0xfe, 0x71, 0x0e, 0xaa, 0xd2, 0xa2, 0x63, 0x79, - 0x38, 0x28, 0x3a, 0x73, 0xcc, 0xea, 0x83, 0x04, 0xab, 0xd7, 0x71, 0x7e, 0x35, 0x43, 0x0b, 0x50, - 0x48, 0x43, 0x35, 0x58, 0xe4, 0x96, 0x3c, 0xc7, 0xd1, 0x69, 0x85, 0x84, 0x9b, 0x4d, 0x45, 0x10, - 0x4c, 0x68, 0x8a, 0xd3, 0xe8, 0x65, 0x71, 0x86, 0x3a, 0x11, 0x68, 0x9a, 0x0d, 0x7a, 0xf3, 0xa9, - 0x97, 0x65, 0x1c, 0x76, 0xd3, 0xcb, 0xe2, 0x44, 0xa9, 0xf4, 0xc0, 0xbb, 0x01, 0x02, 0xd7, 0x4f, - 0x19, 0x04, 0x57, 0x16, 0x52, 0x0f, 0x3c, 0x09, 0xac, 0xd3, 0x03, 0xef, 0xc6, 0xe9, 0xa8, 0x09, - 0x4b, 0xbc, 0x46, 0x10, 0x01, 0xbe, 0x95, 0xc5, 0x54, 0x2f, 0xc7, 0x41, 0x3a, 0xf5, 0xb2, 0x1f, - 0xa5, 0x52, 0x2f, 0x6d, 0xc7, 0xc4, 0xba, 0x3f, 0x7a, 0xb9, 0x52, 0x96, 0x52, 0xbd, 0x4c, 0x7a, - 0xe3, 0xa2, 0x5e, 0xda, 0x71, 0x3a, 0x85, 0x5a, 0x2e, 0xb6, 0x4d, 0x65, 0x99, 0x59, 0x7a, 0x33, - 0xc1, 0x52, 0x08, 0xc5, 0xab, 0x19, 0x8d, 0xc9, 0xf2, 0xe2, 0x72, 0xea, 0xe9, 0x3d, 0x0a, 0x77, - 0x75, 0x93, 0xe3, 0x5d, 0x45, 0xbe, 0xa1, 0xb8, 0x24, 0x40, 0x63, 0x5e, 0x5c, 0xe2, 0x0c, 0x9a, - 0xcb, 0x01, 0x56, 0xed, 0x86, 0x48, 0x59, 0x59, 0x49, 0xcd, 0xe5, 0x64, 0x54, 0x4d, 0x73, 0x99, - 0x5c, 0xe7, 0xb0, 0x1a, 0x2b, 0x6c, 0x07, 0x39, 0x88, 0xd2, 0x6b, 0xec, 0x18, 0x8e, 0x66, 0x35, - 0x36, 0x4a, 0xa5, 0x07, 0x62, 0x04, 0xd3, 0x86, 0x4e, 0xd8, 0xb8, 0xa1, 0xac, 0xa7, 0x1e, 0x48, - 0xd2, 0x60, 0x42, 0x0f, 0xc4, 0x88, 0xd3, 0xa9, 0x9b, 0x1c, 0x53, 0x8f, 0x5a, 0xc1, 0xdd, 0x54, - 0x37, 0xc7, 0x31, 0x39, 0x75, 0xd3, 0x8d, 0x52, 0xd1, 0xaf, 0x24, 0x78, 0x6b, 0xac, 0xf2, 0xb0, - 0xea, 0xad, 0xb3, 0xa7, 0x64, 0x9d, 0x70, 0x70, 0xac, 0xbc, 0xc9, 0x96, 0xf9, 0xc9, 0x04, 0xc5, - 0x28, 0x11, 0x57, 0x57, 0x33, 0xda, 0xa6, 0xf7, 0x1a, 0x41, 0x1a, 0x33, 0x8b, 0x23, 0x4d, 0xdd, - 0x11, 0x50, 0x53, 0xd9, 0x48, 0x8d, 0x59, 0x12, 0x28, 0xa5, 0x31, 0xb3, 0xe2, 0x74, 0xda, 0x10, - 0xfc, 0xd1, 0xab, 0xad, 0x2e, 0x66, 0x51, 0x65, 0x33, 0xb5, 0x21, 0xa4, 0xbc, 0xf1, 0xd2, 0x86, - 0xe0, 0x8f, 0xb1, 0xd0, 0x01, 0x2c, 0x0e, 0x28, 0x16, 0xd5, 0x5d, 0x0e, 0x46, 0x95, 0xfb, 0xa9, - 0xcf, 0xe1, 0x63, 0x98, 0xb5, 0x9a, 0xd1, 0x16, 0x06, 0x11, 0x22, 0xfa, 0x1c, 0xe4, 0xf0, 0x65, - 0x41, 0x3f, 0x61, 0x20, 0x54, 0xd9, 0x62, 0xf6, 0xb6, 0x13, 0xec, 0xdd, 0x80, 0x59, 0x59, 0x93, - 0x89, 0x73, 0xd0, 0x05, 0xdc, 0xa3, 0x23, 0x8c, 0xc1, 0x07, 0x03, 0x1d, 0x8f, 0x26, 0x03, 0x31, - 0x07, 0x3c, 0x60, 0x2b, 0x3d, 0x49, 0x3a, 0xe3, 0x9b, 0xe7, 0x89, 0x6a, 0x46, 0x5b, 0xf7, 0x52, - 0x45, 0x68, 0xb1, 0xe3, 0x2d, 0x82, 0x82, 0x0d, 0x0a, 0x5a, 0x95, 0xb7, 0x52, 0x93, 0x76, 0x1c, - 0xdc, 0xd2, 0xa4, 0xb5, 0xa2, 0x54, 0x74, 0x04, 0x2b, 0x03, 0x8a, 0x54, 0x75, 0xcb, 0xa6, 0x59, - 0xca, 0xb0, 0xaa, 0xf2, 0x30, 0x35, 0x51, 0x92, 0x50, 0x2d, 0x8d, 0xcf, 0x20, 0x4e, 0x47, 0x1f, - 0x0b, 0x9c, 0x75, 0x8a, 0x59, 0x9a, 0xd0, 0x16, 0xfc, 0x76, 0x2a, 0x74, 0x4b, 0xc0, 0xb4, 0x14, - 0xba, 0x85, 0x06, 0x18, 0x99, 0x83, 0xc4, 0x52, 0x1e, 0xa6, 0xd9, 0x64, 0xb3, 0x9f, 0x9b, 0x5d, - 0x93, 0x6f, 0xef, 0xe7, 0x66, 0xef, 0xc8, 0xeb, 0xfb, 0xb9, 0xd9, 0x7b, 0x72, 0x61, 0x3f, 0x37, - 0x5b, 0x90, 0x37, 0xb6, 0x76, 0x18, 0x88, 0x3c, 0x74, 0x5c, 0xd6, 0x22, 0xd0, 0x3a, 0x4c, 0x5b, - 0xb6, 0x89, 0x2f, 0xc5, 0x3c, 0xce, 0x31, 0x32, 0x27, 0x09, 0xd8, 0xf9, 0xcd, 0x14, 0x4c, 0x4f, - 0xf6, 0x7a, 0xf1, 0xb3, 0x38, 0x1c, 0x22, 0x98, 0xbd, 0xf9, 0x33, 0xb0, 0xb7, 0x94, 0x78, 0x00, - 0xb1, 0xeb, 0xcc, 0x84, 0x83, 0xb7, 0x5d, 0x6f, 0x8c, 0x83, 0xca, 0xb0, 0xe8, 0xdb, 0xf8, 0x72, - 0xe8, 0xb8, 0xd8, 0x64, 0xbd, 0x36, 0x37, 0xc9, 0xe8, 0xaa, 0x2d, 0x84, 0x4a, 0xb4, 0xc3, 0xee, - 0xc0, 0xbc, 0x43, 0xac, 0x9e, 0x65, 0xeb, 0xb4, 0xff, 0x30, 0xa4, 0x36, 0x5d, 0x5a, 0xa2, 0x6b, - 0x7e, 0xff, 0x72, 0x63, 0x86, 0xf6, 0xaa, 0x5a, 0x45, 0x03, 0x2e, 0x42, 0xbf, 0xd0, 0x87, 0x30, - 0x63, 0x32, 0xb8, 0x2d, 0x90, 0x57, 0x21, 0x6d, 0xd0, 0xe3, 0xa0, 0x3c, 0x18, 0x32, 0xb8, 0x0e, - 0xfa, 0xbf, 0x20, 0xba, 0xf9, 0x9b, 0x94, 0x83, 0xc3, 0x10, 0x71, 0x47, 0x4f, 0x61, 0xca, 0x76, - 0x2e, 0x04, 0x72, 0x9a, 0x68, 0x74, 0xa3, 0xf2, 0xcf, 0x66, 0x7f, 0xf7, 0xd5, 0x46, 0x66, 0xf4, - 0x08, 0xf5, 0xf8, 0x9f, 0x59, 0x50, 0xd2, 0xde, 0xb2, 0xe9, 0xb0, 0x51, 0x2c, 0xb5, 0xb4, 0x8e, - 0x3e, 0xf6, 0xca, 0xfa, 0x10, 0xee, 0xc7, 0x38, 0xec, 0x43, 0xad, 0xe8, 0x9a, 0x5a, 0x6e, 0x69, - 0x95, 0xf0, 0xc9, 0xf5, 0x1d, 0x78, 0x10, 0x13, 0x6b, 0xaa, 0xc7, 0xec, 0xf1, 0x55, 0x88, 0x75, - 0x5a, 0x2d, 0xbd, 0x55, 0xa7, 0x03, 0x51, 0x01, 0xd6, 0x63, 0x82, 0xe5, 0x7a, 0x4d, 0x6d, 0xd2, - 0xaf, 0x7d, 0xb5, 0x4c, 0xc7, 0xa2, 0x0d, 0xb8, 0x1b, 0xe3, 0x1f, 0x1e, 0xb5, 0xab, 0xaa, 0x16, - 0x2c, 0x2b, 0xe7, 0xd0, 0x5d, 0xb8, 0x3d, 0xee, 0x90, 0xde, 0x3e, 0x2c, 0x36, 0xe5, 0x69, 0x54, - 0x84, 0x8f, 0xe2, 0xcc, 0xba, 0xa6, 0x16, 0x2b, 0xcf, 0x47, 0x6f, 0xc0, 0x7a, 0x4b, 0xd3, 0xb5, - 0x56, 0xbd, 0xae, 0x56, 0xf4, 0x52, 0xb1, 0x7c, 0xa0, 0x1f, 0xb6, 0xda, 0xed, 0x5a, 0xa9, 0xae, - 0xb2, 0x59, 0xaf, 0xf8, 0x5c, 0x9e, 0x41, 0x1f, 0xc0, 0xd3, 0x98, 0x89, 0x4e, 0xad, 0xa1, 0xb6, - 0x3b, 0xc5, 0xc6, 0xa1, 0x5e, 0x2e, 0x96, 0xab, 0xaa, 0xf0, 0x54, 0xad, 0x8c, 0xa9, 0xe6, 0xd7, - 0x73, 0x5f, 0xfe, 0xa1, 0x90, 0x79, 0xfc, 0x75, 0xfc, 0xe1, 0x3c, 0xf2, 0xce, 0xce, 0x67, 0xba, - 0x8e, 0xf6, 0x7c, 0x3c, 0xcc, 0x6c, 0x80, 0xa4, 0x9c, 0x63, 0xad, 0xd6, 0x51, 0xc3, 0x78, 0x49, - 0x7c, 0xe2, 0xa4, 0x8c, 0xb6, 0xaa, 0xd5, 0x8a, 0xf5, 0xda, 0x67, 0xc5, 0x52, 0x5d, 0x95, 0xa7, - 0xd0, 0x1d, 0x78, 0x83, 0xd3, 0xaf, 0xbb, 0x91, 0x43, 0xf7, 0xe0, 0x0e, 0x67, 0x15, 0xdb, 0xcf, - 0x9b, 0x65, 0x61, 0x71, 0xb7, 0x58, 0xab, 0x1f, 0x69, 0xaa, 0x3c, 0xcd, 0xbd, 0xdc, 0xa2, 0x49, - 0x91, 0x7d, 0xfc, 0x0c, 0xd0, 0xf8, 0xad, 0x43, 0xb3, 0x90, 0x6b, 0xb6, 0x9a, 0xaa, 0x9c, 0x41, - 0xf3, 0x90, 0xa7, 0x61, 0x6a, 0xed, 0xee, 0xca, 0x12, 0x5a, 0x84, 0xb9, 0x5a, 0xa3, 0xa1, 0x56, - 0x6a, 0xc5, 0x8e, 0x2a, 0x67, 0x4b, 0xf7, 0xbf, 0xfd, 0x7b, 0x21, 0xf3, 0xed, 0xab, 0x82, 0xf4, - 0x97, 0x57, 0x05, 0xe9, 0xbb, 0x57, 0x05, 0xe9, 0x6f, 0xaf, 0x0a, 0xd2, 0xaf, 0xff, 0x51, 0xc8, - 0x7c, 0x96, 0x17, 0xd9, 0xfc, 0xdf, 0x00, 0x00, 0x00, 0xff, 0xff, 0x3a, 0x02, 0x56, 0x4f, 0xb8, - 0x1e, 0x00, 0x00, +func init() { proto.RegisterFile("roachpb/errors.proto", fileDescriptor_errors_ee43f5f6d0e525bf) } + +var fileDescriptor_errors_ee43f5f6d0e525bf = []byte{ + // 2864 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x9c, 0x59, 0xcd, 0x6f, 0x1b, 0xc7, + 0x15, 0xe7, 0x52, 0x94, 0x48, 0x3d, 0x7d, 0xad, 0xc7, 0x8a, 0xbc, 0x96, 0x63, 0x4a, 0x96, 0xe3, + 0xc4, 0x71, 0x11, 0xa9, 0x70, 0x6a, 0xb4, 0x71, 0x93, 0x03, 0x3f, 0x56, 0x22, 0x25, 0x7e, 0x28, + 0x4b, 0x2a, 0x8a, 0x93, 0x16, 0x83, 0x15, 0x77, 0x44, 0x6d, 0x4c, 0xee, 0xb2, 0xb3, 0x4b, 0x89, + 0xba, 0xf5, 0x98, 0xf6, 0xd4, 0x02, 0x3d, 0xf4, 0xd6, 0x00, 0x3d, 0x05, 0xbd, 0x16, 0xf9, 0x07, + 0x7a, 0xc9, 0xa5, 0x40, 0x8f, 0x41, 0x0f, 0x46, 0xeb, 0x02, 0x45, 0xd1, 0xfe, 0x07, 0x39, 0x15, + 0xf3, 0xb1, 0xe4, 0xae, 0xb8, 0x94, 0x99, 0x9c, 0xc4, 0x7d, 0x5f, 0xf3, 0xe6, 0xcd, 0x9b, 0xf7, + 0x7e, 0x6f, 0x04, 0xab, 0xd4, 0x35, 0x5b, 0x67, 0xbd, 0x93, 0x1d, 0x42, 0xa9, 0x4b, 0xbd, 0xed, + 0x1e, 0x75, 0x7d, 0x17, 0xdd, 0x68, 0xb9, 0xad, 0xe7, 0x9c, 0xb3, 0x2d, 0xf9, 0xeb, 0x28, 0x10, + 0xb4, 0x4c, 0xdf, 0x14, 0x62, 0xeb, 0x6b, 0x01, 0xad, 0x4b, 0x7c, 0x33, 0x44, 0xbf, 0xef, 0xf9, + 0x2e, 0x35, 0xdb, 0x64, 0x87, 0x38, 0x6d, 0xdb, 0x09, 0xfe, 0x30, 0xb9, 0xf3, 0x56, 0xeb, 0x5d, + 0x29, 0xa4, 0xf5, 0x7d, 0xbb, 0xb3, 0x73, 0xd6, 0x69, 0xed, 0xf8, 0x76, 0x97, 0x78, 0xbe, 0xd9, + 0xed, 0x49, 0xce, 0x6a, 0xdb, 0x6d, 0xbb, 0xfc, 0xe7, 0x0e, 0xfb, 0x25, 0xa8, 0x5b, 0x5f, 0x25, + 0xe1, 0x66, 0xcd, 0xf5, 0x2b, 0xc4, 0xf4, 0x48, 0xc9, 0xed, 0x58, 0x84, 0xea, 0xcc, 0x65, 0x54, + 0x84, 0x34, 0x25, 0xbd, 0x8e, 0xdd, 0x32, 0x35, 0x65, 0x53, 0x79, 0xb8, 0xf0, 0xf8, 0x8d, 0xed, + 0x31, 0xef, 0xb7, 0x0d, 0x21, 0x51, 0x24, 0x5e, 0x8b, 0xda, 0x3d, 0xdf, 0xa5, 0xf9, 0xd4, 0xd7, + 0x2f, 0x36, 0x12, 0x46, 0xa0, 0x8a, 0xf6, 0x60, 0xb1, 0xc3, 0x2c, 0xe3, 0x33, 0x6e, 0x5a, 0x4b, + 0x4e, 0x6f, 0xca, 0x58, 0xe8, 0x8c, 0x7c, 0x42, 0x4f, 0x20, 0x43, 0x4d, 0xa7, 0x4d, 0xb0, 0x6d, + 0x69, 0x33, 0x9b, 0xca, 0xc3, 0x99, 0xfc, 0x3a, 0x5b, 0xe9, 0xe5, 0x8b, 0x8d, 0xb4, 0xc1, 0xe8, + 0xe5, 0xe2, 0xb7, 0xa3, 0x9f, 0x46, 0x9a, 0xcb, 0x96, 0x2d, 0xb4, 0x0d, 0xb3, 0xdc, 0x8a, 0x96, + 0xe2, 0x0b, 0x6b, 0x31, 0x0b, 0xf3, 0x9d, 0x1b, 0x42, 0x0c, 0xdd, 0x07, 0x68, 0xf5, 0x3d, 0xdf, + 0xed, 0xe2, 0xae, 0xd7, 0xd6, 0x66, 0x37, 0x95, 0x87, 0xf3, 0x72, 0x4b, 0xf3, 0x82, 0x5e, 0xf5, + 0xda, 0x4f, 0x53, 0xff, 0xf9, 0x62, 0x43, 0xd9, 0x7a, 0x1d, 0x56, 0x6b, 0xae, 0x45, 0x8e, 0x1c, + 0xf3, 0xdc, 0xb4, 0x3b, 0xe6, 0x49, 0x87, 0xf0, 0xc0, 0x49, 0xee, 0x06, 0xdc, 0x3a, 0x72, 0xbc, + 0x7e, 0xaf, 0xe7, 0x52, 0x9f, 0x58, 0x06, 0xf9, 0x45, 0x9f, 0x78, 0x7e, 0x58, 0xe0, 0x73, 0x05, + 0x10, 0x77, 0xb7, 0xe6, 0xfa, 0xbb, 0x6e, 0xdf, 0xb1, 0x44, 0xd8, 0xc3, 0xfb, 0x54, 0xa6, 0xdf, + 0xe7, 0x13, 0xc8, 0xb0, 0xe4, 0xe0, 0x6a, 0xc9, 0xa8, 0x5a, 0x83, 0xd1, 0x85, 0x9a, 0xfc, 0x69, + 0xa4, 0xb9, 0x6c, 0xd9, 0x92, 0xae, 0xfc, 0x21, 0x09, 0xaf, 0x71, 0x8b, 0x07, 0xe4, 0xb2, 0x6a, + 0x7b, 0x5d, 0xd3, 0x6f, 0x9d, 0x09, 0x6f, 0xde, 0x85, 0x1b, 0x54, 0xb8, 0x8e, 0x3d, 0xdf, 0xa4, + 0x3e, 0x7e, 0x4e, 0x2e, 0xb9, 0x5b, 0x8b, 0xf9, 0xf4, 0xb7, 0x2f, 0x36, 0x66, 0x0e, 0xc8, 0xa5, + 0xb1, 0x22, 0x25, 0x1a, 0x4c, 0xe0, 0x80, 0x5c, 0xa2, 0x1d, 0x08, 0x48, 0x98, 0x38, 0x16, 0x57, + 0x49, 0x46, 0x55, 0x96, 0x24, 0x5f, 0x77, 0x2c, 0xa6, 0x50, 0x05, 0xb5, 0x2b, 0x97, 0x25, 0x16, + 0xe6, 0x5b, 0xe2, 0x67, 0xbc, 0xf0, 0x78, 0x2b, 0x2e, 0x51, 0x18, 0x3f, 0x94, 0x26, 0x2b, 0x23, + 0x5d, 0xce, 0x42, 0x07, 0xb0, 0xe2, 0xf5, 0xdb, 0x6d, 0xe2, 0xf9, 0x43, 0x6b, 0xa9, 0xa9, 0xad, + 0x2d, 0x0f, 0x55, 0x39, 0x47, 0x46, 0xe8, 0x7f, 0x49, 0xd8, 0x32, 0x88, 0x69, 0x1d, 0xdb, 0xfe, + 0x99, 0xed, 0x1c, 0x39, 0x2d, 0x42, 0x7d, 0xd3, 0x76, 0xfc, 0xcb, 0xb2, 0xe3, 0x13, 0x7a, 0x6e, + 0x76, 0x44, 0xb8, 0xf6, 0x61, 0x99, 0x12, 0xd3, 0xc2, 0xc3, 0x9b, 0x27, 0xaf, 0xce, 0xdd, 0xd0, + 0xc2, 0xec, 0x7a, 0x6e, 0x9f, 0x75, 0x5a, 0xdb, 0xcd, 0x40, 0x48, 0x26, 0xd8, 0x12, 0x53, 0x1d, + 0x12, 0x91, 0x01, 0x88, 0x0c, 0x6c, 0xcf, 0xb7, 0x9d, 0x76, 0xc8, 0x5e, 0x72, 0x7a, 0x7b, 0x37, + 0x02, 0xf5, 0x91, 0xcd, 0x3c, 0x2c, 0x75, 0xcd, 0x41, 0xc8, 0xdc, 0xcc, 0x14, 0xe6, 0x8c, 0xc5, + 0xae, 0x39, 0x18, 0xd9, 0xf8, 0x14, 0x6e, 0xba, 0x27, 0x1e, 0xa1, 0xe7, 0x24, 0xb4, 0x4f, 0x4f, + 0x4b, 0x6d, 0xce, 0x4c, 0xb8, 0xd8, 0x75, 0x29, 0x7d, 0xd5, 0x3f, 0xe4, 0x5e, 0x65, 0x78, 0x32, + 0xda, 0x9f, 0xc1, 0xad, 0x26, 0x35, 0x1d, 0xcf, 0x6c, 0xf9, 0xb6, 0xeb, 0xe4, 0x4e, 0xf8, 0x15, + 0x12, 0x11, 0x2e, 0xc3, 0x1c, 0x25, 0xa6, 0xe7, 0x3a, 0x3c, 0xb2, 0xcb, 0x8f, 0x7f, 0x10, 0xb3, + 0xe0, 0xb8, 0xae, 0xc1, 0x55, 0xe4, 0xba, 0xd2, 0x80, 0x5c, 0xcb, 0x84, 0xd5, 0x90, 0xfc, 0x61, + 0xdf, 0x93, 0x99, 0x5f, 0x00, 0xe8, 0xf5, 0xbd, 0x33, 0x42, 0xb0, 0x3f, 0x70, 0xe4, 0x31, 0x66, + 0xaf, 0x5f, 0x2c, 0x28, 0x14, 0x42, 0xaf, 0x39, 0x08, 0x96, 0xf8, 0x95, 0x02, 0xaf, 0x85, 0xc4, + 0x0c, 0xe2, 0xd3, 0x4b, 0xb1, 0xc8, 0xde, 0x95, 0xdd, 0xbc, 0x7d, 0xfd, 0x02, 0x5c, 0x33, 0x6e, + 0x2f, 0xe8, 0x1e, 0xcc, 0x93, 0x81, 0x4f, 0x4d, 0x5e, 0xb5, 0x92, 0xa1, 0xaa, 0x95, 0xe1, 0xe4, + 0x51, 0xd1, 0xfa, 0x46, 0x81, 0xb5, 0x90, 0xc5, 0x86, 0x6f, 0xfa, 0x7d, 0x4f, 0x38, 0xb3, 0x06, + 0x33, 0x4c, 0x5b, 0x09, 0x69, 0x33, 0x02, 0xaa, 0x0d, 0x9d, 0x4c, 0x72, 0x27, 0x7f, 0x78, 0xbd, + 0x93, 0x21, 0x93, 0xdb, 0x71, 0xbe, 0x6e, 0x1d, 0xc2, 0x9c, 0xa0, 0x23, 0x04, 0xcb, 0x86, 0x9e, + 0x6b, 0xd4, 0x6b, 0xf8, 0xa8, 0x76, 0x50, 0xab, 0x1f, 0xd7, 0xd4, 0x04, 0xd2, 0x60, 0x55, 0xd2, + 0x9a, 0x1f, 0xd7, 0x70, 0xad, 0xde, 0xc4, 0xbb, 0xf5, 0xa3, 0x5a, 0x51, 0x55, 0xae, 0x70, 0x0a, + 0xf5, 0x6a, 0xb5, 0xdc, 0x6c, 0xea, 0x45, 0x35, 0x29, 0xb7, 0xf6, 0x0c, 0xd4, 0x63, 0x6a, 0xfb, + 0x84, 0x5d, 0x49, 0x47, 0x94, 0x5a, 0xf4, 0x1e, 0xa4, 0x6d, 0xfe, 0xe9, 0x69, 0x0a, 0x4f, 0xd0, + 0xdb, 0x31, 0xce, 0x0b, 0x85, 0xa0, 0x73, 0x49, 0x79, 0x61, 0x74, 0x3f, 0x95, 0x49, 0xaa, 0x33, + 0x5b, 0x7f, 0x52, 0xa4, 0xed, 0xa6, 0xeb, 0xd6, 0x3b, 0x32, 0x15, 0x73, 0x30, 0xff, 0xbd, 0xee, + 0xf9, 0x48, 0x0b, 0xd5, 0x40, 0x35, 0x5b, 0x7e, 0xdf, 0xec, 0x7c, 0xbf, 0x1b, 0xbe, 0x22, 0x94, + 0x87, 0x64, 0x19, 0x88, 0x75, 0x40, 0xf5, 0x1e, 0xeb, 0x38, 0x36, 0x25, 0x5e, 0x73, 0xe0, 0x84, + 0xbb, 0xce, 0x33, 0x58, 0x2d, 0xb8, 0x8e, 0x65, 0xb3, 0x93, 0xda, 0x35, 0xed, 0x4e, 0x70, 0xaf, + 0x7e, 0x0a, 0x8b, 0xd2, 0x93, 0x73, 0xb3, 0xd3, 0x27, 0x72, 0x3f, 0x71, 0xed, 0xf2, 0x23, 0xc6, + 0x37, 0x16, 0x84, 0x34, 0xff, 0x90, 0xa6, 0xff, 0xac, 0x00, 0x12, 0xbd, 0x94, 0x7c, 0x46, 0x5a, + 0xc3, 0x1b, 0x9b, 0x85, 0x74, 0x97, 0x78, 0x9e, 0xd9, 0x26, 0x91, 0xd4, 0x0a, 0x88, 0xe8, 0x7d, + 0x98, 0x97, 0xdd, 0x80, 0x58, 0x72, 0xf3, 0x13, 0xbb, 0x74, 0x10, 0xc1, 0xa1, 0x02, 0x7a, 0x0a, + 0x99, 0xa0, 0xcc, 0xc9, 0x62, 0xf6, 0x2a, 0xe5, 0xa1, 0xbc, 0x74, 0xfb, 0xc7, 0x30, 0xdf, 0x20, + 0xce, 0x74, 0xce, 0x46, 0x92, 0xe2, 0x02, 0x56, 0x73, 0xdd, 0x13, 0xbb, 0xdd, 0x77, 0xfb, 0x9e, + 0x41, 0xbc, 0x7e, 0xc7, 0x9f, 0x6e, 0xc3, 0xef, 0xc1, 0xc2, 0x05, 0x35, 0x7b, 0x3d, 0x62, 0x61, + 0x42, 0xe9, 0x35, 0x5b, 0xe6, 0xe6, 0x0c, 0x90, 0xc2, 0x3a, 0x0d, 0xce, 0xf0, 0x2e, 0xeb, 0xd6, + 0xa7, 0xfe, 0x1e, 0x75, 0xfb, 0xbd, 0x22, 0xe9, 0x90, 0x20, 0xd4, 0x92, 0x4d, 0x60, 0x4d, 0x62, + 0xa9, 0x82, 0x4b, 0x69, 0xbf, 0xc7, 0x8e, 0x5a, 0x78, 0xc6, 0xaa, 0x04, 0xfb, 0x81, 0xaf, 0xde, + 0xf3, 0x0c, 0x27, 0x57, 0xbd, 0x36, 0xda, 0x82, 0xf9, 0x1e, 0x75, 0x5b, 0xc4, 0xf3, 0xe4, 0x69, + 0x64, 0x86, 0x55, 0x2d, 0x20, 0x0f, 0x33, 0x09, 0xc9, 0x65, 0xc2, 0x97, 0xe2, 0x03, 0x00, 0x09, + 0xfd, 0x02, 0x00, 0x33, 0x9b, 0xcf, 0x4a, 0x24, 0x32, 0x2f, 0xe5, 0x39, 0x16, 0x19, 0x7d, 0xb0, + 0xe3, 0x14, 0x3f, 0x03, 0xd3, 0x1f, 0x02, 0xe2, 0x48, 0x65, 0x0c, 0x19, 0x0d, 0x21, 0x8e, 0xf2, + 0x5d, 0x21, 0x4e, 0x95, 0xc1, 0xb1, 0x33, 0xd3, 0xb1, 0x3a, 0xac, 0x1b, 0xf8, 0xf4, 0x72, 0x88, + 0xd7, 0xd0, 0x63, 0x48, 0xf5, 0x74, 0x4a, 0xaf, 0x49, 0x79, 0x2e, 0x27, 0xe3, 0xc0, 0x65, 0xb7, + 0xfe, 0xab, 0xc0, 0x83, 0xab, 0x85, 0x99, 0x61, 0x83, 0x43, 0x06, 0xa9, 0x0d, 0x72, 0x4a, 0x49, + 0xd0, 0x47, 0x26, 0x55, 0xd5, 0x4f, 0x61, 0xce, 0x1f, 0x38, 0x01, 0x5c, 0x5b, 0xcc, 0x17, 0x19, + 0xeb, 0xef, 0x2f, 0x36, 0xde, 0x6d, 0xdb, 0xfe, 0x59, 0xff, 0x64, 0xbb, 0xe5, 0x76, 0x77, 0x86, + 0x9e, 0x58, 0x27, 0xa3, 0xdf, 0x3b, 0xbd, 0xe7, 0xed, 0x1d, 0x8e, 0xf1, 0xfb, 0x7d, 0xdb, 0xda, + 0x3e, 0x3a, 0x2a, 0x17, 0x5f, 0xbe, 0xd8, 0x98, 0x6d, 0x0e, 0x9c, 0x72, 0xd1, 0x98, 0xf5, 0x07, + 0x4e, 0xd9, 0x42, 0xbb, 0xb0, 0xe0, 0x8f, 0xbc, 0x93, 0x17, 0x63, 0xba, 0xee, 0x15, 0x56, 0x94, + 0xb1, 0xab, 0xc0, 0x46, 0x73, 0xe0, 0xe4, 0x3a, 0x0c, 0x9f, 0x5c, 0xea, 0x4e, 0xcb, 0xed, 0x33, + 0xd0, 0x23, 0x93, 0x4e, 0xec, 0xf2, 0x3e, 0x40, 0x8f, 0x92, 0x73, 0xcc, 0xf3, 0x28, 0xb2, 0xd9, + 0x79, 0x46, 0x0f, 0xa7, 0xe7, 0x6f, 0x15, 0x58, 0x65, 0x15, 0xb7, 0x4d, 0x68, 0xfd, 0x9c, 0xd0, + 0xd3, 0x8e, 0x7b, 0x21, 0x6c, 0xdc, 0x86, 0x99, 0x18, 0x74, 0xc9, 0x68, 0xe8, 0x6d, 0x58, 0x6a, + 0xf5, 0x29, 0x25, 0x8e, 0x2f, 0xcb, 0x93, 0x80, 0xb8, 0x62, 0x85, 0x45, 0xc9, 0xe2, 0xb5, 0x08, + 0xbd, 0x03, 0x2b, 0xb6, 0xd3, 0xa2, 0xa4, 0x3b, 0x12, 0x9e, 0x09, 0x09, 0x2f, 0x0f, 0x99, 0xe1, + 0xd2, 0x55, 0x85, 0x1b, 0x55, 0x7b, 0x40, 0xac, 0x46, 0xbf, 0xc5, 0x72, 0x3c, 0xc8, 0x8b, 0xb4, + 0xbc, 0x7a, 0xaf, 0x4a, 0x0d, 0x23, 0x10, 0x94, 0xe6, 0xbe, 0x54, 0xe0, 0x4e, 0x9e, 0x21, 0xd2, + 0x51, 0xc1, 0x26, 0xa7, 0x2e, 0x25, 0x7b, 0x85, 0x61, 0xe7, 0x68, 0x7e, 0xaf, 0xce, 0x31, 0x42, + 0x61, 0xcc, 0xc4, 0x19, 0xcb, 0x32, 0xb7, 0x63, 0x7d, 0x97, 0x96, 0x31, 0xd2, 0x92, 0xbe, 0x7a, + 0x80, 0x44, 0xff, 0xab, 0xda, 0x9e, 0x67, 0x3b, 0x6d, 0xe1, 0xe1, 0xfb, 0xb0, 0x78, 0x41, 0x5d, + 0xa7, 0x8d, 0x45, 0x37, 0x94, 0x4e, 0x4e, 0x6e, 0x9e, 0xc6, 0x02, 0x17, 0x17, 0x1f, 0xc1, 0x49, + 0x26, 0xc7, 0x4f, 0x72, 0x34, 0x3a, 0x55, 0x09, 0x6d, 0x93, 0xb2, 0x73, 0x48, 0xdd, 0x36, 0x0d, + 0x42, 0x2e, 0xb9, 0xbf, 0x4b, 0xc2, 0x4d, 0x0e, 0xbe, 0x77, 0x89, 0xbc, 0xac, 0xc2, 0xa9, 0x83, + 0x2b, 0x68, 0xe9, 0x9d, 0x49, 0x70, 0x3e, 0xaa, 0x17, 0x8f, 0x42, 0xbe, 0x54, 0x86, 0x30, 0x64, + 0x1d, 0xd6, 0x24, 0xb0, 0x30, 0xf4, 0xc3, 0x4a, 0xb9, 0x90, 0xc3, 0x86, 0x5e, 0xad, 0x7f, 0xa4, + 0x17, 0xd5, 0x04, 0x5a, 0x03, 0x14, 0xf0, 0x72, 0xb5, 0x3d, 0x1d, 0x37, 0x0e, 0x2b, 0xe5, 0xa6, + 0xaa, 0xa0, 0x5b, 0x70, 0x33, 0x42, 0xaf, 0xea, 0xc6, 0x1e, 0xc3, 0x22, 0x21, 0x94, 0x62, 0xe4, + 0x76, 0x9b, 0xb8, 0x51, 0xcb, 0x1d, 0x36, 0x4a, 0xf5, 0xa6, 0x3a, 0x83, 0xb2, 0xb0, 0x2e, 0x39, + 0x95, 0xfa, 0x5e, 0xb9, 0x90, 0xab, 0xe0, 0xfa, 0x61, 0x03, 0x57, 0xcb, 0x8d, 0x46, 0xb9, 0xb6, + 0xa7, 0xa6, 0x42, 0x9a, 0x8d, 0x4a, 0xfd, 0x18, 0x17, 0xea, 0xb5, 0xc6, 0x51, 0x55, 0x37, 0xd4, + 0x59, 0x19, 0x96, 0xbf, 0xac, 0xc2, 0x02, 0xdf, 0x50, 0x91, 0xf8, 0xa6, 0xdd, 0x41, 0x06, 0xa8, + 0x8e, 0xeb, 0xe3, 0xc8, 0x78, 0x2d, 0xce, 0xe9, 0xcd, 0x98, 0xc0, 0xc4, 0x8c, 0xf8, 0xa5, 0x84, + 0xb1, 0xec, 0x44, 0xc8, 0xa8, 0x0e, 0x2b, 0x62, 0xfa, 0x64, 0x96, 0x4f, 0x59, 0xed, 0x95, 0xc9, + 0xf5, 0x60, 0x52, 0xac, 0x23, 0x35, 0xba, 0xc4, 0xa6, 0x98, 0x30, 0x15, 0x7d, 0x0c, 0x48, 0x18, + 0x7c, 0x4e, 0x2e, 0x71, 0x30, 0xa8, 0xc9, 0x82, 0xf4, 0x70, 0x92, 0xcd, 0xab, 0x63, 0x68, 0x29, + 0x61, 0xa8, 0xf4, 0x0a, 0x03, 0xfd, 0x52, 0x81, 0x4d, 0x3e, 0x6c, 0x5d, 0xf0, 0x99, 0x0c, 0xf7, + 0x47, 0x43, 0x19, 0xcf, 0x5a, 0x36, 0x95, 0xc9, 0xb9, 0xef, 0x49, 0xec, 0x73, 0xc3, 0xab, 0xa6, + 0xb9, 0x52, 0xc2, 0xb8, 0x4b, 0xaf, 0x93, 0x42, 0x3f, 0x87, 0x9b, 0xa1, 0x6a, 0x89, 0x4d, 0x31, + 0x6c, 0xf0, 0x57, 0x83, 0x85, 0xc7, 0x8f, 0xa6, 0x9a, 0x4c, 0x82, 0x95, 0x90, 0x3f, 0xc6, 0x42, + 0x4d, 0x50, 0xc3, 0xe6, 0xd9, 0x58, 0xa1, 0xcd, 0x71, 0xdb, 0x6f, 0x5d, 0x6f, 0x7b, 0x38, 0xc5, + 0x94, 0x12, 0xc6, 0x8a, 0x1f, 0xa5, 0xa3, 0x63, 0xb8, 0x11, 0xb6, 0x4a, 0xd9, 0x3d, 0xd1, 0xd2, + 0x13, 0x0f, 0x24, 0x76, 0x70, 0x61, 0x07, 0xe2, 0x5f, 0x61, 0xa0, 0x4f, 0x20, 0xbc, 0x09, 0xec, + 0xf1, 0x39, 0x40, 0xcb, 0x70, 0xcb, 0x6f, 0x4f, 0x3d, 0x33, 0x94, 0x12, 0x46, 0xd8, 0x3f, 0xc1, + 0x41, 0x25, 0x56, 0x8f, 0x6c, 0x9f, 0x04, 0xf5, 0x68, 0x9e, 0x5b, 0xbd, 0x1f, 0x63, 0xf5, 0xea, + 0x08, 0x50, 0x4a, 0xb0, 0xda, 0x34, 0xa4, 0xa1, 0x32, 0x2c, 0x09, 0x4b, 0xbe, 0xeb, 0x62, 0x56, + 0x3c, 0xe1, 0x7a, 0x53, 0x21, 0x70, 0x33, 0x34, 0x25, 0x68, 0xec, 0xb2, 0xb8, 0x3d, 0x4c, 0x25, + 0xd0, 0xe6, 0x73, 0xe2, 0xc2, 0xc4, 0xcb, 0x32, 0x8e, 0xc8, 0xd9, 0x65, 0x71, 0xc3, 0x54, 0x76, + 0xe0, 0xad, 0x00, 0x9c, 0xe3, 0x53, 0x8e, 0xce, 0xb5, 0xc5, 0x89, 0x07, 0x1e, 0x87, 0xe3, 0xd9, + 0x81, 0xb7, 0xa2, 0x74, 0x54, 0x83, 0x65, 0x51, 0x23, 0xa8, 0xc4, 0xe5, 0xda, 0xd2, 0x44, 0x2f, + 0xc7, 0xf1, 0x3b, 0xf3, 0xb2, 0x13, 0xa6, 0x32, 0x2f, 0x1d, 0xd7, 0x22, 0xb8, 0x3f, 0x7a, 0xf8, + 0xd2, 0x96, 0x27, 0x7a, 0x19, 0xf7, 0x44, 0xc6, 0xbc, 0x74, 0xa2, 0x74, 0x86, 0xc2, 0x3c, 0xe2, + 0x58, 0xda, 0x0a, 0xb7, 0xf4, 0x7a, 0x8c, 0xa5, 0x21, 0x4a, 0x2f, 0x25, 0x0c, 0x2e, 0x2b, 0x8a, + 0xcb, 0xa9, 0x8f, 0xdb, 0x0c, 0x09, 0x63, 0x4b, 0x40, 0x61, 0x4d, 0xbd, 0xa6, 0xb8, 0xc4, 0xa0, + 0x66, 0x51, 0x5c, 0xa2, 0x0c, 0x96, 0xcb, 0x01, 0x8c, 0x6d, 0x0d, 0x41, 0xb4, 0x76, 0x63, 0x62, + 0x2e, 0xc7, 0x03, 0x6e, 0x96, 0xcb, 0xf4, 0x2a, 0x87, 0xd7, 0x58, 0x69, 0x3b, 0xc8, 0x41, 0x34, + 0xb9, 0xc6, 0x8e, 0x41, 0x6c, 0x5e, 0x63, 0xc3, 0x54, 0x76, 0x20, 0x66, 0x30, 0x88, 0x60, 0xca, + 0x27, 0x11, 0x6d, 0x7d, 0xe2, 0x81, 0xc4, 0xcd, 0x2c, 0xec, 0x40, 0xcc, 0x28, 0x9d, 0xb9, 0x29, + 0xe0, 0xf6, 0xa8, 0x15, 0xdc, 0x99, 0xe8, 0xe6, 0x38, 0x5c, 0x67, 0x6e, 0x7a, 0x61, 0x2a, 0xfa, + 0xb5, 0x02, 0x6f, 0x8c, 0x55, 0x1e, 0x5e, 0xbd, 0x31, 0x7f, 0x89, 0xc6, 0x54, 0xe0, 0x66, 0xed, + 0x75, 0xbe, 0xcc, 0x4f, 0xa6, 0x28, 0x46, 0xb1, 0x90, 0xbb, 0x94, 0x30, 0x36, 0xfd, 0x57, 0x08, + 0xb2, 0x98, 0xd9, 0x02, 0x84, 0x62, 0x57, 0xa2, 0x50, 0x6d, 0x63, 0x62, 0xcc, 0xe2, 0xf0, 0x2a, + 0x8b, 0x99, 0x1d, 0xa5, 0xb3, 0x86, 0xd0, 0x1f, 0x3d, 0xfa, 0x62, 0x39, 0xa6, 0x6a, 0x9b, 0x13, + 0x1b, 0xc2, 0x84, 0x27, 0x62, 0xd6, 0x10, 0xfa, 0x63, 0x2c, 0x74, 0x00, 0x4b, 0x5d, 0x06, 0x53, + 0xb1, 0x27, 0x70, 0xaa, 0x76, 0x6f, 0xe2, 0x6b, 0xfa, 0x18, 0x9c, 0x2d, 0x25, 0x8c, 0xc5, 0x6e, + 0x88, 0x88, 0x3e, 0x05, 0x75, 0xf8, 0xe8, 0x80, 0x4f, 0x38, 0x3e, 0xd5, 0xb6, 0xb8, 0xbd, 0xed, + 0x18, 0x7b, 0xd7, 0xc0, 0x59, 0xde, 0x64, 0xa2, 0x1c, 0x74, 0x01, 0x77, 0xd9, 0x74, 0x63, 0x8a, + 0x99, 0x01, 0x93, 0xd1, 0xd0, 0x20, 0x47, 0x84, 0xfb, 0x7c, 0xa5, 0xc7, 0x71, 0x67, 0x7c, 0xfd, + 0xa8, 0x51, 0x4a, 0x18, 0xeb, 0xfe, 0x44, 0x11, 0x56, 0xec, 0x44, 0x8b, 0x60, 0x60, 0x83, 0xe1, + 0x59, 0xed, 0x8d, 0x89, 0x49, 0x3b, 0x8e, 0x7b, 0x59, 0xd2, 0xda, 0x61, 0x2a, 0x3a, 0x82, 0x1b, + 0x5d, 0x86, 0x54, 0xb1, 0xed, 0xb0, 0x2c, 0xe5, 0x58, 0x55, 0x7b, 0x30, 0x31, 0x51, 0xe2, 0x50, + 0x2d, 0x8b, 0x4f, 0x37, 0x4a, 0x47, 0x1f, 0x4a, 0x9c, 0x75, 0x4a, 0x78, 0x9a, 0xb0, 0x16, 0xfc, + 0xe6, 0x44, 0xe8, 0x16, 0x83, 0x69, 0x19, 0x74, 0x1b, 0x1a, 0xe0, 0x64, 0x01, 0x12, 0xf3, 0x69, + 0x98, 0xe5, 0x43, 0xcf, 0x7e, 0x2a, 0xb3, 0xa6, 0xde, 0xda, 0x4f, 0x65, 0x6e, 0xab, 0xeb, 0xfb, + 0xa9, 0xcc, 0x5d, 0x35, 0xbb, 0x9f, 0xca, 0x64, 0xd5, 0x8d, 0xad, 0x1d, 0x0e, 0x22, 0x0f, 0x5d, + 0x8f, 0xb7, 0x08, 0xb4, 0x0e, 0xb3, 0xb6, 0x63, 0x91, 0x81, 0x1c, 0xd5, 0x05, 0x46, 0x16, 0x24, + 0x09, 0x3b, 0xbf, 0x9a, 0x81, 0xd9, 0xe9, 0x1e, 0x36, 0x7e, 0x16, 0x85, 0x43, 0x94, 0xf0, 0x7f, + 0x19, 0x70, 0xb0, 0xb7, 0x1c, 0x7b, 0x00, 0x91, 0xeb, 0xcc, 0x85, 0x83, 0xa7, 0x61, 0x7f, 0x8c, + 0x83, 0x0a, 0xb0, 0xd4, 0x77, 0xc8, 0xa0, 0xe7, 0x7a, 0xc4, 0xe2, 0xbd, 0x36, 0x35, 0xcd, 0x54, + 0x6b, 0x2c, 0x0e, 0x95, 0x58, 0x87, 0xdd, 0x81, 0x05, 0x97, 0xda, 0x6d, 0xdb, 0xc1, 0xac, 0xff, + 0x70, 0xa4, 0x36, 0x9b, 0x5f, 0x66, 0x6b, 0x7e, 0xfb, 0x62, 0x63, 0x8e, 0xf5, 0xaa, 0x72, 0xd1, + 0x00, 0x21, 0xc2, 0xbe, 0xd0, 0xfb, 0x30, 0x67, 0x71, 0xb8, 0x2d, 0x91, 0x57, 0x76, 0xd2, 0x0c, + 0x28, 0x40, 0x79, 0x30, 0x64, 0x08, 0x1d, 0xf4, 0xa3, 0x20, 0xba, 0xe9, 0xeb, 0x94, 0x83, 0xc3, + 0x90, 0x71, 0x47, 0x4f, 0x60, 0xc6, 0x71, 0x2f, 0x24, 0x72, 0x9a, 0x6a, 0xaa, 0x63, 0xf2, 0x4f, + 0x33, 0xbf, 0xff, 0x62, 0x23, 0x31, 0x7a, 0x9f, 0x7a, 0xf4, 0xef, 0x24, 0x68, 0x93, 0x9e, 0xc2, + 0xd9, 0xb0, 0x91, 0xcb, 0xd7, 0x8d, 0x26, 0x1e, 0x7b, 0x80, 0x7d, 0x00, 0xf7, 0x22, 0x1c, 0xfe, + 0xa1, 0x17, 0xb1, 0xa1, 0x17, 0xea, 0x46, 0x71, 0xf8, 0x1a, 0xfb, 0x16, 0xdc, 0x8f, 0x88, 0xd5, + 0xf4, 0x63, 0xfe, 0x2e, 0x2b, 0xc5, 0x9a, 0xf5, 0x3a, 0xae, 0x57, 0xd8, 0x40, 0x94, 0x85, 0xf5, + 0x88, 0x60, 0xa1, 0x52, 0xd6, 0x6b, 0xec, 0x6b, 0x5f, 0x2f, 0xb0, 0xb1, 0x68, 0x03, 0xee, 0x44, + 0xf8, 0x87, 0x47, 0x8d, 0x92, 0x6e, 0x04, 0xcb, 0xaa, 0x29, 0x74, 0x07, 0x6e, 0x8d, 0x3b, 0x84, + 0x1b, 0x87, 0xb9, 0x9a, 0x3a, 0x8b, 0x72, 0xf0, 0x41, 0x94, 0x59, 0x31, 0xf4, 0x5c, 0xf1, 0xd9, + 0xe8, 0x79, 0x18, 0xd7, 0x0d, 0x6c, 0xd4, 0x2b, 0x15, 0xbd, 0x88, 0xf3, 0xb9, 0xc2, 0x01, 0x3e, + 0xac, 0x37, 0x1a, 0xe5, 0x7c, 0x45, 0xe7, 0xb3, 0x5e, 0xee, 0x99, 0x3a, 0x87, 0xde, 0x83, 0x27, + 0x11, 0x13, 0xcd, 0x72, 0x55, 0x6f, 0x34, 0x73, 0xd5, 0x43, 0x5c, 0xc8, 0x15, 0x4a, 0xba, 0xf4, + 0x54, 0x2f, 0x8e, 0xa9, 0xa6, 0xd7, 0x53, 0x9f, 0xff, 0x31, 0x9b, 0x78, 0xf4, 0xd7, 0xe8, 0x9b, + 0x7a, 0xe8, 0x95, 0x5e, 0xcc, 0x74, 0x4d, 0xe3, 0xd9, 0x78, 0x98, 0xf9, 0x00, 0xc9, 0x38, 0xc7, + 0x46, 0xb9, 0xa9, 0x0f, 0xe3, 0xa5, 0x88, 0x89, 0x93, 0x31, 0x1a, 0xba, 0x51, 0xce, 0x55, 0xca, + 0x9f, 0xe4, 0xf2, 0x15, 0x5d, 0x9d, 0x41, 0xb7, 0xe1, 0x35, 0x41, 0xbf, 0xea, 0x46, 0x0a, 0xdd, + 0x85, 0xdb, 0x82, 0x95, 0x6b, 0x3c, 0xab, 0x15, 0xa4, 0xc5, 0xdd, 0x5c, 0xb9, 0x72, 0x64, 0xe8, + 0xea, 0x2c, 0xda, 0x82, 0xac, 0x60, 0x8b, 0xa0, 0xe0, 0xa2, 0x9e, 0x2b, 0x56, 0xca, 0x35, 0x1d, + 0xeb, 0x1f, 0x17, 0x74, 0xbd, 0xa8, 0x17, 0xd5, 0x39, 0xb1, 0x93, 0x2d, 0x96, 0x38, 0xc9, 0x47, + 0x4f, 0x01, 0x8d, 0xdf, 0x4c, 0x94, 0x81, 0x54, 0xad, 0x5e, 0xd3, 0xd5, 0x04, 0x5a, 0x80, 0x34, + 0x0b, 0x65, 0x7d, 0x77, 0x57, 0x55, 0xd0, 0x12, 0xcc, 0x97, 0xab, 0x55, 0xbd, 0x58, 0xce, 0x35, + 0x75, 0x35, 0x99, 0xbf, 0xf7, 0xf5, 0x3f, 0xb3, 0x89, 0xaf, 0x5f, 0x66, 0x95, 0xbf, 0xbd, 0xcc, + 0x2a, 0xdf, 0xbc, 0xcc, 0x2a, 0xff, 0x78, 0x99, 0x55, 0x7e, 0xf3, 0xaf, 0x6c, 0xe2, 0x93, 0xb4, + 0xcc, 0xf8, 0xff, 0x07, 0x00, 0x00, 0xff, 0xff, 0x85, 0x13, 0x03, 0xac, 0x1b, 0x1f, 0x00, 0x00, } diff --git a/pkg/roachpb/errors.proto b/pkg/roachpb/errors.proto index 5f6aa6488b37..c6fec78018b4 100644 --- a/pkg/roachpb/errors.proto +++ b/pkg/roachpb/errors.proto @@ -212,6 +212,8 @@ enum TransactionRetryReason { RETRY_POSSIBLE_REPLAY = 4; // An asynchronous write was observed to have failed. RETRY_ASYNC_WRITE_FAILURE = 5; + // The transaction exceeded its deadline. + RETRY_COMMIT_DEADLINE_EXCEEDED = 6; } // A TransactionRetryError indicates that the transaction must be @@ -220,6 +222,7 @@ message TransactionRetryError { option (gogoproto.equal) = true; optional TransactionRetryReason reason = 1 [(gogoproto.nullable) = false]; + optional string extra_msg = 2 [(gogoproto.nullable) = false]; } // A TransactionStatusError indicates that the transaction status is @@ -447,6 +450,8 @@ message IntentMissingError { // The non-matching intent that was found at that key, if any. optional Intent wrong_intent = 1; + // The key where the intent was expected. + optional bytes key = 2 [(gogoproto.casttype) = "Key"]; } // A MergeInProgressError indicates that the request could not be completed diff --git a/pkg/sql/as_of_test.go b/pkg/sql/as_of_test.go index ac2957665888..366ac792a5c8 100644 --- a/pkg/sql/as_of_test.go +++ b/pkg/sql/as_of_test.go @@ -301,7 +301,8 @@ func TestAsOfRetry(t *testing.T) { } if count > 0 && bytes.Contains(req.Key, []byte(key)) { magicVals.restartCounts[key]-- - err := roachpb.NewTransactionRetryError(roachpb.RETRY_REASON_UNKNOWN) + err := roachpb.NewTransactionRetryError( + roachpb.RETRY_REASON_UNKNOWN, "filter err") magicVals.failedValues[string(req.Key)] = failureRecord{err, args.Hdr.Txn} txn := args.Hdr.Txn.Clone() diff --git a/pkg/sql/distsql_running_test.go b/pkg/sql/distsql_running_test.go index 0ca03e450fb1..68c350b6dae4 100644 --- a/pkg/sql/distsql_running_test.go +++ b/pkg/sql/distsql_running_test.go @@ -213,7 +213,7 @@ func TestDistSQLReceiverErrorRanking(t *testing.T) { retryErr := roachpb.NewErrorWithTxn( roachpb.NewTransactionRetryError( - roachpb.RETRY_SERIALIZABLE), + roachpb.RETRY_SERIALIZABLE, "test err"), txn.Serialize()).GoError() abortErr := roachpb.NewErrorWithTxn( diff --git a/pkg/sql/lease_test.go b/pkg/sql/lease_test.go index 91f8ee7400a6..18eef47558d7 100644 --- a/pkg/sql/lease_test.go +++ b/pkg/sql/lease_test.go @@ -33,6 +33,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/server" "github.com/cockroachdb/cockroach/pkg/sql" + "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" "github.com/cockroachdb/cockroach/pkg/sql/sqlbase" "github.com/cockroachdb/cockroach/pkg/sql/tests" "github.com/cockroachdb/cockroach/pkg/testutils" @@ -43,6 +44,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/syncutil" "github.com/cockroachdb/cockroach/pkg/util/timeutil" "github.com/cockroachdb/cockroach/pkg/util/tracing" + "github.com/lib/pq" "github.com/pkg/errors" ) @@ -1010,10 +1012,14 @@ INSERT INTO t.kv VALUES ('a', 'b'); t.Fatal(err) } - deadlineError := "TransactionStatusError: transaction deadline exceeded" - if err := txWrite.Commit(); !testutils.IsError(err, deadlineError) { - t.Fatalf("err = %v", err) + checkDeadlineErr := func(err error, t *testing.T) { + pqe, ok := err.(*pq.Error) + if !ok || pqe.Code != pgerror.CodeSerializationFailureError || + !testutils.IsError(err, "RETRY_COMMIT_DEADLINE_EXCEEDED") { + t.Fatalf("expected deadline exceeded, got: %v", err) + } } + checkDeadlineErr(txWrite.Commit(), t) // Test the deadline exceeded error with a CREATE/DROP INDEX. txWrite, err = sqlDB.Begin() @@ -1037,17 +1043,13 @@ INSERT INTO t.kv VALUES ('a', 'b'); t.Fatal(err) } - if err := txWrite.Commit(); !testutils.IsError(err, deadlineError) { - t.Fatalf("err = %v", err) - } + checkDeadlineErr(txWrite.Commit(), t) if _, err := txUpdate.Exec(`UPDATE t.kv SET v = 'c' WHERE k = 'a';`); err != nil { t.Fatal(err) } - if err := txUpdate.Commit(); !testutils.IsError(err, deadlineError) { - t.Fatalf("err = %v", err) - } + checkDeadlineErr(txUpdate.Commit(), t) txWrite, err = sqlDB.Begin() if err != nil { diff --git a/pkg/sql/txn_restart_test.go b/pkg/sql/txn_restart_test.go index 45f76fec6746..410df9dec491 100644 --- a/pkg/sql/txn_restart_test.go +++ b/pkg/sql/txn_restart_test.go @@ -130,7 +130,7 @@ func injectErrors( {counts: magicVals.restartCounts, errFn: func() error { // Note we use a retry error that cannot be automatically retried // by the transaction coord sender. - return roachpb.NewTransactionRetryError(roachpb.RETRY_POSSIBLE_REPLAY) + return roachpb.NewTransactionRetryError(roachpb.RETRY_POSSIBLE_REPLAY, "injected err") }}, {counts: magicVals.abortCounts, errFn: func() error { return roachpb.NewTransactionAbortedError(roachpb.ABORT_REASON_ABORTED_RECORD_FOUND) diff --git a/pkg/storage/batcheval/cmd_end_transaction.go b/pkg/storage/batcheval/cmd_end_transaction.go index ab50caed7bb4..00dbb36f2bd6 100644 --- a/pkg/storage/batcheval/cmd_end_transaction.go +++ b/pkg/storage/batcheval/cmd_end_transaction.go @@ -271,28 +271,8 @@ func evalEndTransaction( // Set transaction status to COMMITTED or ABORTED as per the // args.Commit parameter. if args.Commit { - if retry, reason := IsEndTransactionTriggeringRetryError(reply.Txn, *args); retry { - return result.Result{}, roachpb.NewTransactionRetryError(reason) - } - - if IsEndTransactionExceedingDeadline(reply.Txn.Timestamp, *args) { - // If the deadline has lapsed return an error and rely on the client - // issuing a Rollback() that aborts the transaction and cleans up - // intents. Unfortunately, we're returning an error and unable to - // write on error (see #1989): we can't write ABORTED into the master - // transaction record which remains PENDING, and thus rely on the - // client to issue a Rollback() for cleanup. - // - // N.B. This deadline test is expected to be a Noop for Serializable - // transactions; unless the client misconfigured the txn, the deadline can - // only be expired if the txn has been pushed, and pushed Serializable - // transactions are detected above. - exceededBy := reply.Txn.Timestamp.GoTime().Sub(args.Deadline.GoTime()) - fromStart := reply.Txn.Timestamp.GoTime().Sub(reply.Txn.OrigTimestamp.GoTime()) - return result.Result{}, roachpb.NewTransactionStatusError(fmt.Sprintf( - "transaction deadline exceeded by %s (%s > %s), original timestamp %s ago (%s): %+v", - exceededBy, reply.Txn.Timestamp, args.Deadline, fromStart, reply.Txn.OrigTimestamp, - reply.Txn)) + if retry, reason, extraMsg := IsEndTransactionTriggeringRetryError(reply.Txn, *args); retry { + return result.Result{}, roachpb.NewTransactionRetryError(reason, extraMsg) } reply.Txn.Status = roachpb.COMMITTED @@ -375,15 +355,16 @@ func evalEndTransaction( // IsEndTransactionExceedingDeadline returns true if the transaction // exceeded its deadline. func IsEndTransactionExceedingDeadline(t hlc.Timestamp, args roachpb.EndTransactionRequest) bool { - return args.Deadline != nil && args.Deadline.Less(t) + return args.Deadline != nil && !t.Less(*args.Deadline) } // IsEndTransactionTriggeringRetryError returns true if the // EndTransactionRequest cannot be committed and needs to return a -// TransactionRetryError. +// TransactionRetryError. It also returns the reason and possibly an extra +// message to be used for the error. func IsEndTransactionTriggeringRetryError( txn *roachpb.Transaction, args roachpb.EndTransactionRequest, -) (retry bool, reason roachpb.TransactionRetryReason) { +) (retry bool, reason roachpb.TransactionRetryReason, extraMsg string) { // If we saw any WriteTooOldErrors, we must restart to avoid lost // update anomalies. if txn.WriteTooOld { @@ -405,7 +386,19 @@ func IsEndTransactionTriggeringRetryError( retry, reason = false, 0 } - return retry, reason + if !retry { + if IsEndTransactionExceedingDeadline(txn.Timestamp, args) { + exceededBy := txn.Timestamp.GoTime().Sub(args.Deadline.GoTime()) + fromStart := txn.Timestamp.GoTime().Sub(txn.OrigTimestamp.GoTime()) + extraMsg = fmt.Sprintf( + "txn timestamp pushed too much; deadline exceeded by %s (%s > %s), "+ + "original timestamp %s ago (%s)", + exceededBy, txn.Timestamp, args.Deadline, fromStart, txn.OrigTimestamp) + retry, reason = true, roachpb.RETRY_COMMIT_DEADLINE_EXCEEDED + } + } + + return retry, reason, extraMsg } // canForwardSerializableTimestamp returns whether a serializable txn can diff --git a/pkg/storage/batcheval/cmd_query_intent.go b/pkg/storage/batcheval/cmd_query_intent.go index dd487fd71317..58485b566ac9 100644 --- a/pkg/storage/batcheval/cmd_query_intent.go +++ b/pkg/storage/batcheval/cmd_query_intent.go @@ -95,9 +95,9 @@ func QueryIntent( // return a TransactionRetryError immediately with an updated // transaction proto. This is an optimization that can help // the txn use refresh spans more effectively. - return result.Result{}, roachpb.NewTransactionRetryError(roachpb.RETRY_SERIALIZABLE) + return result.Result{}, roachpb.NewTransactionRetryError(roachpb.RETRY_SERIALIZABLE, "" /* extraMsg */) } - return result.Result{}, roachpb.NewIntentMissingError(intent) + return result.Result{}, roachpb.NewIntentMissingError(args.Key, intent) case roachpb.QueryIntentRequest_PREVENT: // The intent will be prevented by bumping the timestamp cache for // the key to the txn timestamp in Replica.updateTimestampCache. diff --git a/pkg/storage/client_merge_test.go b/pkg/storage/client_merge_test.go index 52c18ef666b7..c37c5f24d208 100644 --- a/pkg/storage/client_merge_test.go +++ b/pkg/storage/client_merge_test.go @@ -236,7 +236,8 @@ func mergeWithData(t *testing.T, retries int64) { for _, req := range ba.Requests { if et := req.GetEndTransaction(); et != nil && et.InternalCommitTrigger.GetMergeTrigger() != nil { if atomic.AddInt64(&retries, -1) >= 0 { - return roachpb.NewError(roachpb.NewTransactionRetryError(roachpb.RETRY_SERIALIZABLE)) + return roachpb.NewError( + roachpb.NewTransactionRetryError(roachpb.RETRY_SERIALIZABLE, "filter err")) } } if req.GetSubsume() != nil { @@ -658,7 +659,8 @@ func TestStoreRangeMergeTxnFailure(t *testing.T) { for _, req := range ba.Requests { if et := req.GetEndTransaction(); et != nil && et.InternalCommitTrigger.GetMergeTrigger() != nil { if atomic.AddInt64(&retriesBeforeFailure, -1) >= 0 { - return roachpb.NewError(roachpb.NewTransactionRetryError(roachpb.RETRY_SERIALIZABLE)) + return roachpb.NewError( + roachpb.NewTransactionRetryError(roachpb.RETRY_SERIALIZABLE, "filter err")) } return roachpb.NewError(errors.New("injected permafail")) } @@ -2601,7 +2603,8 @@ func testMergeWatcher(t *testing.T, injectFailures bool) { for _, req := range ba.Requests { if et := req.GetEndTransaction(); et != nil && et.InternalCommitTrigger.GetMergeTrigger() != nil { if atomic.AddInt64(&mergeTxnRetries, -1) >= 0 { - return roachpb.NewError(roachpb.NewTransactionRetryError(roachpb.RETRY_SERIALIZABLE)) + return roachpb.NewError( + roachpb.NewTransactionRetryError(roachpb.RETRY_SERIALIZABLE, "filter err")) } } if pt := req.GetPushTxn(); pt != nil { diff --git a/pkg/storage/replica_test.go b/pkg/storage/replica_test.go index 621e4faba05b..e330249edd42 100644 --- a/pkg/storage/replica_test.go +++ b/pkg/storage/replica_test.go @@ -3185,17 +3185,13 @@ func TestEndTransactionDeadline(t *testing.T) { } case 1: - // Past deadline. - if err := roachpb.CheckTxnDeadlineExceededErr(pErr.GetDetail()); err != nil { - t.Error(err) - } - + fallthrough case 2: - // Equal deadline. - if pErr != nil { - t.Error(pErr) + // Past deadline. + retErr, ok := pErr.GetDetail().(*roachpb.TransactionRetryError) + if !ok || retErr.Reason != roachpb.RETRY_COMMIT_DEADLINE_EXCEEDED { + t.Fatalf("expected deadline exceeded, got: %v", pErr) } - case 3: // Future deadline. if pErr != nil { @@ -3398,8 +3394,9 @@ func TestEndTransactionDeadline_1PC(t *testing.T) { ba.Add(&bt, &put, &et) assignSeqNumsForReqs(txn, &bt, &put, &et) _, pErr := tc.Sender().Send(context.Background(), ba) - if err := roachpb.CheckTxnDeadlineExceededErr(pErr.GetDetail()); err != nil { - t.Error(err) + retErr, ok := pErr.GetDetail().(*roachpb.TransactionRetryError) + if !ok || retErr.Reason != roachpb.RETRY_COMMIT_DEADLINE_EXCEEDED { + t.Fatalf("expected deadline exceeded, got: %v", pErr) } } diff --git a/pkg/storage/replica_write.go b/pkg/storage/replica_write.go index ef4aa58ad787..82c6be7c2d5c 100644 --- a/pkg/storage/replica_write.go +++ b/pkg/storage/replica_write.go @@ -369,7 +369,7 @@ func (r *Replica) evaluateWriteBatch( if pErr != nil { return batch, ms, nil, result.Result{}, pErr } else if ba.Timestamp != br.Timestamp { - err := roachpb.NewTransactionRetryError(roachpb.RETRY_REASON_UNKNOWN) + err := roachpb.NewTransactionRetryError(roachpb.RETRY_REASON_UNKNOWN, "Require1PC batch pushed") return batch, ms, nil, result.Result{}, roachpb.NewError(err) } log.Fatal(ctx, "unreachable") @@ -484,7 +484,7 @@ func isOnePhaseCommit(ba roachpb.BatchRequest, knobs *StoreTestingKnobs) bool { if batcheval.IsEndTransactionExceedingDeadline(ba.Txn.Timestamp, *etArg) { return false } - if retry, _ := batcheval.IsEndTransactionTriggeringRetryError(ba.Txn, *etArg); retry { + if retry, _, _ := batcheval.IsEndTransactionTriggeringRetryError(ba.Txn, *etArg); retry { return false } return !knobs.DisableOptional1PC || etArg.Require1PC diff --git a/pkg/storage/store.go b/pkg/storage/store.go index 10f723c3021d..b999d302390d 100644 --- a/pkg/storage/store.go +++ b/pkg/storage/store.go @@ -2924,7 +2924,8 @@ func (s *Store) Send( // If we're not retrying on push txn failures return a txn retry error // after the first failure to guarantee a retry. if ba.Txn != nil { - err := roachpb.NewTransactionRetryError(roachpb.RETRY_REASON_UNKNOWN) + err := roachpb.NewTransactionRetryError( + roachpb.RETRY_REASON_UNKNOWN, "DontRetryPushTxnFailures testing knob") return nil, roachpb.NewErrorWithTxn(err, ba.Txn) } return nil, pErr