diff --git a/pkg/keys/constants.go b/pkg/keys/constants.go index d4f8c978accf..150372342a4e 100644 --- a/pkg/keys/constants.go +++ b/pkg/keys/constants.go @@ -22,7 +22,7 @@ import ( // These constants are single bytes for performance. They allow single-byte // comparisons which are considerably faster than bytes.HasPrefix. const ( - localPrefixByte = '\x01' + LocalPrefixByte = '\x01' localMaxByte = '\x02' meta1PrefixByte = localMaxByte meta2PrefixByte = '\x03' @@ -42,8 +42,8 @@ var ( // MaxKey is the infinity marker which is larger than any other key. MaxKey = roachpb.KeyMax - // localPrefix is the prefix for all local keys. - localPrefix = roachpb.Key{localPrefixByte} + // LocalPrefix is the prefix for all local keys. + LocalPrefix = roachpb.Key{LocalPrefixByte} // LocalMax is the end of the local key range. It is itself a global // key. LocalMax = roachpb.Key{localMaxByte} @@ -64,7 +64,7 @@ var ( // metadata is identified by one of the suffixes listed below, along // with potentially additional encoded key info, for instance in the // case of AbortSpan entry. - LocalRangeIDPrefix = roachpb.RKey(makeKey(localPrefix, roachpb.Key("i"))) + LocalRangeIDPrefix = roachpb.RKey(makeKey(LocalPrefix, roachpb.Key("i"))) // LocalRangeIDReplicatedInfix is the post-Range ID specifier for all Raft // replicated per-range data. By appending this after the Range ID, these // keys will be sorted directly before the local unreplicated keys for the @@ -134,7 +134,7 @@ var ( // specific sort of per-range metadata is identified by one of the // suffixes listed below, along with potentially additional encoded // key info, such as the txn ID in the case of a transaction record. - LocalRangePrefix = roachpb.Key(makeKey(localPrefix, roachpb.RKey("k"))) + LocalRangePrefix = roachpb.Key(makeKey(LocalPrefix, roachpb.RKey("k"))) LocalRangeMax = LocalRangePrefix.PrefixEnd() // LocalQueueLastProcessedSuffix is the suffix for replica queue state keys. LocalQueueLastProcessedSuffix = roachpb.RKey("qlpt") @@ -148,7 +148,7 @@ var ( // 4. Store local keys // // LocalStorePrefix is the prefix identifying per-store data. - LocalStorePrefix = makeKey(localPrefix, roachpb.Key("s")) + LocalStorePrefix = makeKey(LocalPrefix, roachpb.Key("s")) // localStoreSuggestedCompactionSuffix stores suggested compactions to // be aggregated and processed on the store. localStoreSuggestedCompactionSuffix = []byte("comp") @@ -200,7 +200,7 @@ var ( // double duty as a reference to a provisional MVCC value. // TODO(sumeer): remember to adjust this comment when adding locks of // other strengths, or range locks. - LocalRangeLockTablePrefix = roachpb.Key(makeKey(localPrefix, roachpb.RKey("z"))) + LocalRangeLockTablePrefix = roachpb.Key(makeKey(LocalPrefix, roachpb.RKey("z"))) LockTableSingleKeyInfix = []byte("k") // LockTableSingleKeyStart is the inclusive start key of the key range // containing single key locks. diff --git a/pkg/keys/doc.go b/pkg/keys/doc.go index 3147014c29a3..c2a94ba441d2 100644 --- a/pkg/keys/doc.go +++ b/pkg/keys/doc.go @@ -177,8 +177,8 @@ var _ = [...]interface{}{ // `LocalRangeLockTablePrefix`. // // `LocalRangeIDPrefix`, `localRangePrefix`, `localStorePrefix`, and - // `LocalRangeLockTablePrefix` all in turn share `localPrefix`. - // `localPrefix` was chosen arbitrarily. Local keys would work just as well + // `LocalRangeLockTablePrefix` all in turn share `LocalPrefix`. + // `LocalPrefix` was chosen arbitrarily. Local keys would work just as well // with a different prefix, like 0xff, or even with a suffix. // 1. Replicated range-ID local keys: These store metadata pertaining to a diff --git a/pkg/keys/keys.go b/pkg/keys/keys.go index 00e614bcfb88..8fc8d3b175f4 100644 --- a/pkg/keys/keys.go +++ b/pkg/keys/keys.go @@ -478,7 +478,7 @@ func DecodeLockTableSingleKey(key roachpb.Key) (lockedKey roachpb.Key, err error // opposed to "user") keys, but unfortunately that name has already been // claimed by a related (but not identical) concept. func IsLocal(k roachpb.Key) bool { - return bytes.HasPrefix(k, localPrefix) + return bytes.HasPrefix(k, LocalPrefix) } // Addr returns the address for the key, used to lookup the range containing the diff --git a/pkg/keys/keys_test.go b/pkg/keys/keys_test.go index 0eb436ffab6f..64453a03b628 100644 --- a/pkg/keys/keys_test.go +++ b/pkg/keys/keys_test.go @@ -66,7 +66,7 @@ func TestKeySorting(t *testing.T) { roachpb.RKey("\x01").Less(roachpb.RKey("\x01\x00"))) { t.Fatalf("something is seriously wrong with this machine") } - if bytes.Compare(localPrefix, Meta1Prefix) >= 0 { + if bytes.Compare(LocalPrefix, Meta1Prefix) >= 0 { t.Fatalf("local key spilling into replicated ranges") } if !bytes.Equal(roachpb.Key(""), roachpb.Key(nil)) { @@ -140,7 +140,7 @@ func TestKeyAddressError(t *testing.T) { RangeLastReplicaGCTimestampKey(0), }, "local key .* malformed": { - makeKey(localPrefix, roachpb.Key("z")), + makeKey(LocalPrefix, roachpb.Key("z")), }, } for regexp, keyList := range testCases { diff --git a/pkg/keys/printer.go b/pkg/keys/printer.go index 28fba6fd6d2e..2ac5c3b8bdfb 100644 --- a/pkg/keys/printer.go +++ b/pkg/keys/printer.go @@ -70,7 +70,7 @@ var ( // KeyDict drives the pretty-printing and pretty-scanning of the key space. KeyDict = KeyComprehensionTable{ - {Name: "/Local", start: localPrefix, end: LocalMax, Entries: []DictEntry{ + {Name: "/Local", start: LocalPrefix, end: LocalMax, Entries: []DictEntry{ {Name: "/Store", prefix: roachpb.Key(LocalStorePrefix), ppFunc: localStoreKeyPrint, PSFunc: localStoreKeyParse}, {Name: "/RangeID", prefix: roachpb.Key(LocalRangeIDPrefix), diff --git a/pkg/kv/kvserver/batcheval/BUILD.bazel b/pkg/kv/kvserver/batcheval/BUILD.bazel index 74533bf6b386..8b2bc863b321 100644 --- a/pkg/kv/kvserver/batcheval/BUILD.bazel +++ b/pkg/kv/kvserver/batcheval/BUILD.bazel @@ -92,8 +92,8 @@ go_test( "cmd_resolve_intent_test.go", "cmd_revert_range_test.go", "cmd_scan_test.go", - "cmd_subsume_test.go", "cmd_truncate_log_test.go", + "declare_test.go", "intent_test.go", "main_test.go", "transaction_test.go", diff --git a/pkg/kv/kvserver/batcheval/cmd_lease_test.go b/pkg/kv/kvserver/batcheval/cmd_lease_test.go index 4eafd2448804..fcfa4c18b384 100644 --- a/pkg/kv/kvserver/batcheval/cmd_lease_test.go +++ b/pkg/kv/kvserver/batcheval/cmd_lease_test.go @@ -17,7 +17,10 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/storage" + "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" + "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/stretchr/testify/require" @@ -120,10 +123,13 @@ func TestLeaseCommandLearnerReplica(t *testing.T) { } desc := roachpb.RangeDescriptor{} desc.SetReplicas(roachpb.MakeReplicaSet(replicas)) + manual := hlc.NewManualClock(123) + clock := hlc.NewClock(manual.UnixNano, time.Nanosecond) cArgs := CommandArgs{ EvalCtx: (&MockEvalCtx{ StoreID: voterStoreID, Desc: &desc, + Clock: clock, }).EvalContext(), Args: &roachpb.TransferLeaseRequest{ Lease: roachpb.Lease{ @@ -157,6 +163,64 @@ func TestLeaseCommandLearnerReplica(t *testing.T) { require.EqualError(t, err, expForLearner) } +// TestLeaseTransferForwardsStartTime tests that during a lease transfer, the +// start time of the new lease is determined during evaluation, after latches +// have granted the lease transfer full mutual exclusion over the leaseholder. +func TestLeaseTransferForwardsStartTime(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + testutils.RunTrueAndFalse(t, "epoch", func(t *testing.T, epoch bool) { + ctx := context.Background() + db := storage.NewDefaultInMem() + defer db.Close() + batch := db.NewBatch() + defer batch.Close() + + replicas := []roachpb.ReplicaDescriptor{ + {NodeID: 1, StoreID: 1, Type: roachpb.ReplicaTypeVoterFull(), ReplicaID: 1}, + {NodeID: 2, StoreID: 2, Type: roachpb.ReplicaTypeVoterFull(), ReplicaID: 2}, + } + desc := roachpb.RangeDescriptor{} + desc.SetReplicas(roachpb.MakeReplicaSet(replicas)) + manual := hlc.NewManualClock(123) + clock := hlc.NewClock(manual.UnixNano, time.Nanosecond) + + nextLease := roachpb.Lease{ + Replica: replicas[1], + Start: clock.NowAsClockTimestamp(), + } + if epoch { + nextLease.Epoch = 1 + } else { + exp := nextLease.Start.ToTimestamp().Add(9*time.Second.Nanoseconds(), 0) + nextLease.Expiration = &exp + } + cArgs := CommandArgs{ + EvalCtx: (&MockEvalCtx{ + StoreID: 1, + Desc: &desc, + Clock: clock, + }).EvalContext(), + Args: &roachpb.TransferLeaseRequest{ + Lease: nextLease, + }, + } + + manual.Increment(1000) + beforeEval := clock.NowAsClockTimestamp() + + res, err := TransferLease(ctx, batch, cArgs, nil) + require.NoError(t, err) + + // The proposed lease start time should be assigned at eval time. + propLease := res.Replicated.State.Lease + require.NotNil(t, propLease) + require.True(t, nextLease.Start.Less(propLease.Start)) + require.True(t, beforeEval.Less(propLease.Start)) + }) +} + func TestCheckCanReceiveLease(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) diff --git a/pkg/kv/kvserver/batcheval/cmd_lease_transfer.go b/pkg/kv/kvserver/batcheval/cmd_lease_transfer.go index 3f17dbcb4468..fa561fdf825c 100644 --- a/pkg/kv/kvserver/batcheval/cmd_lease_transfer.go +++ b/pkg/kv/kvserver/batcheval/cmd_lease_transfer.go @@ -13,7 +13,6 @@ package batcheval import ( "context" - "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval/result" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/spanset" "github.com/cockroachdb/cockroach/pkg/roachpb" @@ -28,13 +27,26 @@ func init() { func declareKeysTransferLease( rs ImmutableRangeState, _ roachpb.Header, _ roachpb.Request, latchSpans, _ *spanset.SpanSet, ) { - latchSpans.AddNonMVCC(spanset.SpanReadWrite, roachpb.Span{Key: keys.RangeLeaseKey(rs.GetRangeID())}) - latchSpans.AddNonMVCC(spanset.SpanReadOnly, roachpb.Span{Key: keys.RangeDescriptorKey(rs.GetStartKey())}) - // Cover the entire addressable key space with a latch to prevent any writes - // from overlapping with lease transfers. In principle we could just use the - // current range descriptor (desc) but it could potentially change due to an - // as of yet unapplied merge. - latchSpans.AddNonMVCC(spanset.SpanReadOnly, roachpb.Span{Key: keys.LocalMax, EndKey: keys.MaxKey}) + // TransferLease must not run concurrently with any other request so it uses + // latches to synchronize with all other reads and writes on the outgoing + // leaseholder. Additionally, it observes the state of the timestamp cache + // and so it uses latches to wait for all in-flight requests to complete. + // + // Because of this, it declares a non-MVCC write over every addressable key + // in the range, even through the only key the TransferLease actually writes + // to is the RangeLeaseKey. This guarantees that it conflicts with any other + // request because every request must declare at least one addressable key. + // + // We could, in principle, declare these latches as MVCC writes at the time + // of the new lease. Doing so would block all concurrent writes but would + // allow reads below the new lease timestamp through. However, doing so + // would only be safe if we also accounted for clock uncertainty in all read + // latches so that any read that may need to observe state on the new + // leaseholder gets blocked. We actually already do this for transactional + // reads (see DefaultDeclareIsolatedKeys), but not for non-transactional + // reads. We'd need to be careful here, so we should only pull on this if we + // decide that doing so is important. + declareAllKeys(latchSpans) } // TransferLease sets the lease holder for the range. @@ -55,13 +67,22 @@ func TransferLease( // LeaseRejectedError before going through Raft. prevLease, _ := cArgs.EvalCtx.GetLease() + // Forward the lease's start time to a current clock reading. At this + // point, we're holding latches across the entire range, we know that + // this time is greater than the timestamps at which any request was + // serviced by the leaseholder before it stopped serving requests (i.e. + // before the TransferLease request acquired latches). + newLease := args.Lease + newLease.Start.Forward(cArgs.EvalCtx.Clock().NowAsClockTimestamp()) + args.Lease = roachpb.Lease{} // prevent accidental use below + // If this check is removed at some point, the filtering of learners on the // sending side would have to be removed as well. - if err := roachpb.CheckCanReceiveLease(args.Lease.Replica, cArgs.EvalCtx.Desc()); err != nil { + if err := roachpb.CheckCanReceiveLease(newLease.Replica, cArgs.EvalCtx.Desc()); err != nil { return newFailedLeaseTrigger(true /* isTransfer */), err } - log.VEventf(ctx, 2, "lease transfer: prev lease: %+v, new lease: %+v", prevLease, args.Lease) + log.VEventf(ctx, 2, "lease transfer: prev lease: %+v, new lease: %+v", prevLease, newLease) return evalNewLease(ctx, cArgs.EvalCtx, readWriter, cArgs.Stats, - args.Lease, prevLease, false /* isExtension */, true /* isTransfer */) + newLease, prevLease, false /* isExtension */, true /* isTransfer */) } diff --git a/pkg/kv/kvserver/batcheval/cmd_push_txn.go b/pkg/kv/kvserver/batcheval/cmd_push_txn.go index 01ba61602052..d459f6719258 100644 --- a/pkg/kv/kvserver/batcheval/cmd_push_txn.go +++ b/pkg/kv/kvserver/batcheval/cmd_push_txn.go @@ -124,6 +124,8 @@ func PushTxn( return result.Result{}, errors.Errorf("request timestamp %s less than pushee txn timestamp %s", h.Timestamp, args.PusheeTxn.WriteTimestamp) } now := cArgs.EvalCtx.Clock().Now() + // TODO(nvanbenschoten): remove this limitation. But when doing so, + // keep the h.Timestamp.Less(args.PushTo) check above. if now.Less(h.Timestamp) { // The batch's timestamp should have been used to update the clock. return result.Result{}, errors.Errorf("request timestamp %s less than current clock time %s", h.Timestamp, now) diff --git a/pkg/kv/kvserver/batcheval/cmd_subsume.go b/pkg/kv/kvserver/batcheval/cmd_subsume.go index 80de40328605..cc7b44015ff1 100644 --- a/pkg/kv/kvserver/batcheval/cmd_subsume.go +++ b/pkg/kv/kvserver/batcheval/cmd_subsume.go @@ -32,27 +32,10 @@ func declareKeysSubsume( ) { // Subsume must not run concurrently with any other command. It declares a // non-MVCC write over every addressable key in the range; this guarantees - // that it conflicts with any other command because every command must declare - // at least one addressable key. It does not, in fact, write any keys. - // - // We use the key bounds from the range descriptor in the request instead - // of the current range descriptor. Either would be fine because we verify - // that these match during the evaluation of the Subsume request. - args := req.(*roachpb.SubsumeRequest) - desc := args.RightDesc - latchSpans.AddNonMVCC(spanset.SpanReadWrite, roachpb.Span{ - Key: desc.StartKey.AsRawKey(), - EndKey: desc.EndKey.AsRawKey(), - }) - latchSpans.AddNonMVCC(spanset.SpanReadWrite, roachpb.Span{ - Key: keys.MakeRangeKeyPrefix(desc.StartKey), - EndKey: keys.MakeRangeKeyPrefix(desc.EndKey).PrefixEnd(), - }) - rangeIDPrefix := keys.MakeRangeIDReplicatedPrefix(desc.RangeID) - latchSpans.AddNonMVCC(spanset.SpanReadWrite, roachpb.Span{ - Key: rangeIDPrefix, - EndKey: rangeIDPrefix.PrefixEnd(), - }) + // that it conflicts with any other command because every command must + // declare at least one addressable key. It does not, in fact, write any + // keys. + declareAllKeys(latchSpans) } // Subsume freezes a range for merging with its left-hand neighbor. When called diff --git a/pkg/kv/kvserver/batcheval/declare.go b/pkg/kv/kvserver/batcheval/declare.go index 21341482e468..d24224ab9219 100644 --- a/pkg/kv/kvserver/batcheval/declare.go +++ b/pkg/kv/kvserver/batcheval/declare.go @@ -82,6 +82,20 @@ func DeclareKeysForBatch( } } +// declareAllKeys declares a non-MVCC write over every addressable key. This +// guarantees that the caller conflicts with any other command because every +// command must declare at least one addressable key, which is tested against +// in TestRequestsSerializeWithAllKeys. +func declareAllKeys(latchSpans *spanset.SpanSet) { + // NOTE: we don't actually know what the end key of the Range will + // be at the time of request evaluation (see ImmutableRangeState), + // so we simply declare a latch over the entire keyspace. This may + // extend beyond the Range, but this is ok for the purpose of + // acquiring latches. + latchSpans.AddNonMVCC(spanset.SpanReadWrite, roachpb.Span{Key: keys.LocalPrefix, EndKey: keys.LocalMax}) + latchSpans.AddNonMVCC(spanset.SpanReadWrite, roachpb.Span{Key: keys.LocalMax, EndKey: keys.MaxKey}) +} + // CommandArgs contains all the arguments to a command. // TODO(bdarnell): consider merging with kvserverbase.FilterArgs (which // would probably require removing the EvalCtx field due to import order diff --git a/pkg/kv/kvserver/batcheval/cmd_subsume_test.go b/pkg/kv/kvserver/batcheval/declare_test.go similarity index 54% rename from pkg/kv/kvserver/batcheval/cmd_subsume_test.go rename to pkg/kv/kvserver/batcheval/declare_test.go index 1ce40c608796..409418806ea5 100644 --- a/pkg/kv/kvserver/batcheval/cmd_subsume_test.go +++ b/pkg/kv/kvserver/batcheval/declare_test.go @@ -21,37 +21,36 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/uuid" ) -// TestRequestsSerializeWithSubsume ensures that no request can be evaluated -// concurrently with a Subsume request. For more details, refer to the big -// comment block at the end of Subsume() in cmd_subsume.go. -// -// NB: This test is broader than it really needs to be. A more precise statement -// of the condition necessary to uphold the invariant mentioned in Subsume() is: -// No request that bumps the lease applied index of a range can be evaluated -// concurrently with a Subsume request. -func TestRequestsSerializeWithSubsume(t *testing.T) { +// TestRequestsSerializeWithAllKeys ensures that no request can be evaluated +// concurrently with either a Subsume request or a TransferLease request, both +// of which declare latches using declareAllKeys to guarantee mutual exclusion +// over the leaseholder. +func TestRequestsSerializeWithAllKeys(t *testing.T) { defer leaktest.AfterTest(t)() - var subsumeLatchSpans, subsumeLockSpans, otherLatchSpans, otherLockSpans spanset.SpanSet - startKey := []byte(`a`) - endKey := []byte(`b`) - desc := &roachpb.RangeDescriptor{ - RangeID: 0, - StartKey: startKey, - EndKey: endKey, - } - testTxn := &roachpb.Transaction{ - TxnMeta: enginepb.TxnMeta{ - ID: uuid.FastMakeV4(), - Key: startKey, - WriteTimestamp: hlc.Timestamp{WallTime: 1}, - }, - Name: "test txn", - } - header := roachpb.Header{Txn: testTxn} - subsumeRequest := &roachpb.SubsumeRequest{RightDesc: *desc} - declareKeysSubsume(desc, header, subsumeRequest, &subsumeLatchSpans, &subsumeLockSpans) + + var allLatchSpans spanset.SpanSet + declareAllKeys(&allLatchSpans) + for method, command := range cmds { t.Run(method.String(), func(t *testing.T) { + var otherLatchSpans, otherLockSpans spanset.SpanSet + + startKey := []byte(`a`) + endKey := []byte(`b`) + desc := &roachpb.RangeDescriptor{ + RangeID: 0, + StartKey: startKey, + EndKey: endKey, + } + testTxn := &roachpb.Transaction{ + TxnMeta: enginepb.TxnMeta{ + ID: uuid.FastMakeV4(), + Key: startKey, + WriteTimestamp: hlc.Timestamp{WallTime: 1}, + }, + Name: "test txn", + } + header := roachpb.Header{Txn: testTxn} otherRequest := roachpb.CreateRequest(method) if queryTxnReq, ok := otherRequest.(*roachpb.QueryTxnRequest); ok { // QueryTxnRequest declares read-only access over the txn record of the txn @@ -60,7 +59,6 @@ func TestRequestsSerializeWithSubsume(t *testing.T) { // falling outside our test range's keyspace. queryTxnReq.Txn = testTxn.TxnMeta } - otherRequest.SetHeader(roachpb.RequestHeader{ Key: startKey, EndKey: endKey, @@ -68,8 +66,8 @@ func TestRequestsSerializeWithSubsume(t *testing.T) { }) command.DeclareKeys(desc, header, otherRequest, &otherLatchSpans, &otherLockSpans) - if !subsumeLatchSpans.Intersects(&otherLatchSpans) { - t.Errorf("%s does not serialize with Subsume", method) + if !allLatchSpans.Intersects(&otherLatchSpans) { + t.Errorf("%s does not serialize with declareAllKeys", method) } }) } diff --git a/pkg/kv/kvserver/client_merge_test.go b/pkg/kv/kvserver/client_merge_test.go index d38e9dc0efc8..1150a6cafa6e 100644 --- a/pkg/kv/kvserver/client_merge_test.go +++ b/pkg/kv/kvserver/client_merge_test.go @@ -34,7 +34,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvserver" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/concurrency/lock" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase" - "github.com/cockroachdb/cockroach/pkg/kv/kvserver/liveness" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/rditer" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/stateloader" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/txnwait" @@ -1373,21 +1372,22 @@ func TestStoreRangeMergeRHSLeaseExpiration(t *testing.T) { } // Install a hook to observe when a get or a put request for a special key, - // rhsSentinel, acquires latches and begins evaluating. + // rhsSentinel, hits a MergeInProgressError and begins waiting on the merge. const reqConcurrency = 10 var rhsSentinel roachpb.Key - reqAcquiredLatch := make(chan struct{}, reqConcurrency) - testingLatchFilter := func(_ context.Context, ba roachpb.BatchRequest) *roachpb.Error { - for _, r := range ba.Requests { - req := r.GetInner() - switch req.Method() { - case roachpb.Get, roachpb.Put: - if req.Header().Key.Equal(rhsSentinel) { - reqAcquiredLatch <- struct{}{} + reqWaitingOnMerge := make(chan struct{}, reqConcurrency) + testingConcurrencyRetryFilter := func(_ context.Context, ba roachpb.BatchRequest, pErr *roachpb.Error) { + if _, ok := pErr.GetDetail().(*roachpb.MergeInProgressError); ok { + for _, r := range ba.Requests { + req := r.GetInner() + switch req.Method() { + case roachpb.Get, roachpb.Put: + if req.Header().Key.Equal(rhsSentinel) { + reqWaitingOnMerge <- struct{}{} + } } } } - return nil } manualClock := hlc.NewHybridManualClock() @@ -1401,8 +1401,9 @@ func TestStoreRangeMergeRHSLeaseExpiration(t *testing.T) { ClockSource: manualClock.UnixNano, }, Store: &kvserver.StoreTestingKnobs{ - TestingRequestFilter: testingRequestFilter, - TestingLatchFilter: testingLatchFilter, + TestingRequestFilter: testingRequestFilter, + TestingConcurrencyRetryFilter: testingConcurrencyRetryFilter, + AllowLeaseRequestProposalsWhenNotLeader: true, }, }, }, @@ -1416,6 +1417,7 @@ func TestStoreRangeMergeRHSLeaseExpiration(t *testing.T) { // during the merge. lhsDesc, rhsDesc, err := tc.Servers[0].ScratchRangeWithExpirationLeaseEx() require.NoError(t, err) + rhsSentinel = rhsDesc.StartKey.AsRawKey() tc.AddVotersOrFatal(t, lhsDesc.StartKey.AsRawKey(), tc.Target(1)) tc.AddVotersOrFatal(t, rhsDesc.StartKey.AsRawKey(), tc.Target(1)) @@ -1438,6 +1440,7 @@ func TestStoreRangeMergeRHSLeaseExpiration(t *testing.T) { // is aware of the merge and is refusing all traffic, so we can't just send a // TransferLease request. Instead, we need to expire the second store's lease, // then acquire the lease on the first store. + toAdvance := store.GetStoreConfig().LeaseExpiration() // Before doing so, however, ensure that the merge transaction has written // its transaction record so that it doesn't run into trouble with the low @@ -1445,19 +1448,18 @@ func TestStoreRangeMergeRHSLeaseExpiration(t *testing.T) { // the transaction being inadvertently aborted during its first attempt, // which this test is not designed to handle. If the merge transaction did // abort then the get requests could complete on r2 before the merge retried. - hb, hbH := heartbeatArgs(mergeTxn, tc.Servers[0].Clock().Now()) + // + // We heartbeat the merge's transaction record with a timestamp forwarded by + // the duration we plan to advance the clock by so that the transaction does + // not look expired even after the manual clock update. + afterAdvance := tc.Servers[0].Clock().Now().Add(toAdvance, 0) + hb, hbH := heartbeatArgs(mergeTxn, afterAdvance) if _, pErr := kv.SendWrappedWith(ctx, store.TestSender(), hbH, hb); pErr != nil { t.Fatal(pErr) } - // Turn off liveness heartbeats on the second store, then advance the clock - // past the liveness expiration time. This expires all leases on all stores. - tc.Servers[1].NodeLiveness().(*liveness.NodeLiveness).PauseHeartbeatLoopForTest() - manualClock.Increment(store.GetStoreConfig().LeaseExpiration()) - - // Manually heartbeat the liveness on the first store to ensure it's - // considered live. The automatic heartbeat might not come for a while. - require.NoError(t, tc.HeartbeatLiveness(ctx, 0)) + // Then increment the clock to expire all leases. + manualClock.Increment(toAdvance) // Send several get and put requests to the RHS. The first of these to // arrive will acquire the lease; the remaining requests will wait for that @@ -1513,19 +1515,17 @@ func TestStoreRangeMergeRHSLeaseExpiration(t *testing.T) { time.Sleep(time.Millisecond) } - // Wait for the get and put requests to acquire latches, which is as far as - // they can get while the merge is in progress. Then wait a little bit - // longer. This tests that the requests really do get stuck waiting for the - // merge to complete without depending too heavily on implementation - // details. + // Wait for the get and put requests to begin waiting on the merge to + // complete. Then wait a little bit longer. This tests that the requests + // really do get stuck waiting for the merge to complete without depending + // too heavily on implementation details. for i := 0; i < reqConcurrency; i++ { select { - case <-reqAcquiredLatch: - // Latch acquired. + case <-reqWaitingOnMerge: + // Waiting on merge. case pErr := <-reqErrs: - // Requests may never make it to the latch acquisition if s1 has not - // yet learned s2's lease is expired. Instead, we'll see a - // NotLeaseholderError. + // Requests may never wait on the merge if s1 has not yet learned + // s2's lease is expired. Instead, we'll see a NotLeaseholderError. require.IsType(t, &roachpb.NotLeaseHolderError{}, pErr.GetDetail()) } } diff --git a/pkg/kv/kvserver/client_raft_test.go b/pkg/kv/kvserver/client_raft_test.go index b758412e4905..ac7d63ca32c0 100644 --- a/pkg/kv/kvserver/client_raft_test.go +++ b/pkg/kv/kvserver/client_raft_test.go @@ -1089,7 +1089,7 @@ func TestRequestsOnLaggingReplica(t *testing.T) { tc := testcluster.StartTestCluster(t, 3, clusterArgs) defer tc.Stopper().Stop(ctx) - rngDesc, err := tc.Servers[0].ScratchRangeEx() + _, rngDesc, err := tc.Servers[0].ScratchRangeEx() require.NoError(t, err) key := rngDesc.StartKey.AsRawKey() // Add replicas on all the stores. diff --git a/pkg/kv/kvserver/client_replica_test.go b/pkg/kv/kvserver/client_replica_test.go index fddc0505807c..d8c0d367fdb9 100644 --- a/pkg/kv/kvserver/client_replica_test.go +++ b/pkg/kv/kvserver/client_replica_test.go @@ -3353,12 +3353,15 @@ func TestDiscoverIntentAcrossLeaseTransferAwayAndBack(t *testing.T) { defer leaktest.AfterTest(t)() ctx := context.Background() + // Use a manual clock so we can efficiently force leases to expire. + // Required by TestCluster.MoveRangeLeaseNonCooperatively. + manual := hlc.NewHybridManualClock() + // Detect when txn2 has completed its read of txn1's intent and block. var txn2ID atomic.Value var txn2BBlockOnce sync.Once txn2BlockedC := make(chan chan struct{}) - knobs := &kvserver.StoreTestingKnobs{} - knobs.EvalKnobs.TestingPostEvalFilter = func(args kvserverbase.FilterArgs) *roachpb.Error { + postEvalFilter := func(args kvserverbase.FilterArgs) *roachpb.Error { if txn := args.Hdr.Txn; txn != nil && txn.ID == txn2ID.Load() { txn2BBlockOnce.Do(func() { if !errors.HasType(args.Err, (*roachpb.WriteIntentError)(nil)) { @@ -3376,7 +3379,7 @@ func TestDiscoverIntentAcrossLeaseTransferAwayAndBack(t *testing.T) { // Detect when txn4 discovers txn3's intent and begins to push. var txn4ID atomic.Value txn4PushingC := make(chan struct{}, 1) - knobs.TestingRequestFilter = func(_ context.Context, ba roachpb.BatchRequest) *roachpb.Error { + requestFilter := func(_ context.Context, ba roachpb.BatchRequest) *roachpb.Error { if !ba.IsSinglePushTxnRequest() { return nil } @@ -3390,7 +3393,19 @@ func TestDiscoverIntentAcrossLeaseTransferAwayAndBack(t *testing.T) { } tc := testcluster.StartTestCluster(t, 3, base.TestClusterArgs{ - ServerArgs: base.TestServerArgs{Knobs: base.TestingKnobs{Store: knobs}}, + ServerArgs: base.TestServerArgs{Knobs: base.TestingKnobs{ + Store: &kvserver.StoreTestingKnobs{ + EvalKnobs: kvserverbase.BatchEvalTestingKnobs{ + TestingPostEvalFilter: postEvalFilter, + }, + TestingRequestFilter: requestFilter, + // Required by TestCluster.MoveRangeLeaseNonCooperatively. + AllowLeaseRequestProposalsWhenNotLeader: true, + }, + Server: &server.TestingKnobs{ + ClockSource: manual.UnixNano, + }, + }}, }) defer tc.Stopper().Stop(ctx) kvDB := tc.Servers[0].DB() @@ -3418,8 +3433,16 @@ func TestDiscoverIntentAcrossLeaseTransferAwayAndBack(t *testing.T) { }() txn2UnblockC := <-txn2BlockedC - // Transfer the lease to Server 1. - err = tc.TransferRangeLease(rangeDesc, tc.Target(1)) + // Transfer the lease to Server 1. Do so non-cooperatively instead of using + // a lease transfer, because the cooperative lease transfer would get stuck + // acquiring latches, which are held by txn2. + err = tc.MoveRangeLeaseNonCooperatively(rangeDesc, tc.Target(1), manual) + require.NoError(t, err) + + // Send an arbitrary request to the range to update the range descriptor + // cache with the new lease. This prevents the rollback from getting stuck + // waiting on latches held by txn2's read on the old leaseholder. + _, err = kvDB.Get(ctx, "c") require.NoError(t, err) // Roll back txn1. diff --git a/pkg/kv/kvserver/concurrency/concurrency_manager.go b/pkg/kv/kvserver/concurrency/concurrency_manager.go index 14cd6e20f844..dcb9c5315e57 100644 --- a/pkg/kv/kvserver/concurrency/concurrency_manager.go +++ b/pkg/kv/kvserver/concurrency/concurrency_manager.go @@ -89,6 +89,7 @@ func NewManager(cfg Config) Manager { lt: lt, ltw: &lockTableWaiterImpl{ st: cfg.Settings, + clock: cfg.Clock, stopper: cfg.Stopper, ir: cfg.IntentResolver, lt: lt, @@ -443,9 +444,10 @@ func (g *Guard) HoldingLatches() bool { return g != nil && g.lg != nil } -// AssertLatches asserts that the guard is non-nil and holding latches. +// AssertLatches asserts that the guard is non-nil and holding latches, if the +// request is supposed to hold latches while evaluating in the first place. func (g *Guard) AssertLatches() { - if !g.HoldingLatches() { + if shouldAcquireLatches(g.Req) && !g.HoldingLatches() { panic("expected latches held, found none") } } diff --git a/pkg/kv/kvserver/concurrency/concurrency_manager_test.go b/pkg/kv/kvserver/concurrency/concurrency_manager_test.go index affee03eb6e9..49f18f363cce 100644 --- a/pkg/kv/kvserver/concurrency/concurrency_manager_test.go +++ b/pkg/kv/kvserver/concurrency/concurrency_manager_test.go @@ -73,6 +73,7 @@ import ( // debug-latch-manager // debug-lock-table // debug-disable-txn-pushes +// debug-set-clock ts= // reset // func TestConcurrencyManagerBasic(t *testing.T) { @@ -119,7 +120,6 @@ func TestConcurrencyManagerBasic(t *testing.T) { ReadTimestamp: ts, MaxTimestamp: maxTS, } - txn.UpdateObservedTimestamp(c.nodeDesc.NodeID, ts.UnsafeToClockTimestamp()) c.registerTxn(txnName, txn) return "" @@ -459,6 +459,17 @@ func TestConcurrencyManagerBasic(t *testing.T) { c.disableTxnPushes() return "" + case "debug-set-clock": + var secs int + d.ScanArgs(t, "ts", &secs) + + nanos := int64(secs) * time.Second.Nanoseconds() + if nanos < c.manual.UnixNano() { + d.Fatalf(t, "manual clock must advance") + } + c.manual.Set(nanos) + return "" + case "reset": if n := mon.numMonitored(); n > 0 { d.Fatalf(t, "%d requests still in flight", n) @@ -494,6 +505,8 @@ type cluster struct { nodeDesc *roachpb.NodeDescriptor rangeDesc *roachpb.RangeDescriptor st *clustersettings.Settings + manual *hlc.ManualClock + clock *hlc.Clock m concurrency.Manager // Definitions. @@ -523,10 +536,13 @@ type txnPush struct { } func newCluster() *cluster { + manual := hlc.NewManualClock(123 * time.Second.Nanoseconds()) return &cluster{ - st: clustersettings.MakeTestingClusterSettings(), nodeDesc: &roachpb.NodeDescriptor{NodeID: 1}, rangeDesc: &roachpb.RangeDescriptor{RangeID: 1}, + st: clustersettings.MakeTestingClusterSettings(), + manual: manual, + clock: hlc.NewClock(manual.UnixNano, time.Nanosecond), txnsByName: make(map[string]*roachpb.Transaction), requestsByName: make(map[string]testReq), @@ -541,6 +557,7 @@ func (c *cluster) makeConfig() concurrency.Config { NodeDesc: c.nodeDesc, RangeDesc: c.rangeDesc, Settings: c.st, + Clock: c.clock, IntentResolver: c, OnContentionEvent: func(ev *roachpb.ContentionEvent) { ev.Duration = 1234 * time.Millisecond // for determinism diff --git a/pkg/kv/kvserver/concurrency/lock_table_waiter.go b/pkg/kv/kvserver/concurrency/lock_table_waiter.go index b60b796775a0..16e37d6d1a1f 100644 --- a/pkg/kv/kvserver/concurrency/lock_table_waiter.go +++ b/pkg/kv/kvserver/concurrency/lock_table_waiter.go @@ -23,6 +23,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/settings" "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/storage/enginepb" + "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/stop" "github.com/cockroachdb/cockroach/pkg/util/syncutil" @@ -94,6 +95,7 @@ var LockTableDeadlockDetectionPushDelay = settings.RegisterDurationSetting( // lockTableWaiterImpl is an implementation of lockTableWaiter. type lockTableWaiterImpl struct { st *cluster.Settings + clock *hlc.Clock stopper *stop.Stopper ir IntentResolver lt lockTable @@ -573,12 +575,34 @@ func (w *lockTableWaiterImpl) pushHeader(req Request) roachpb.Header { // could race). Since the subsequent execution of the original request // might mutate the transaction, make a copy here. See #9130. h.Txn = req.Txn.Clone() + // We must push at least to req.Timestamp, but for transactional // requests we actually want to go all the way up to the top of the // transaction's uncertainty interval. This allows us to not have to // restart for uncertainty if the push succeeds and we come back and // read. - h.Timestamp.Forward(req.Txn.MaxTimestamp) + // + // However, because we intend to read on the same node, we can limit + // this to a clock reading from the local clock, relying on the fact + // that an observed timestamp from this node will limit our local + // uncertainty limit when we return to read. + // + // We intentionally do not use an observed timestamp directly to limit + // the push timestamp, because observed timestamps are not applicable in + // some cases (e.g. across lease changes). So to avoid an infinite loop + // where we continue to push to an unusable observed timestamp and + // continue to find the pushee in our uncertainty interval, we instead + // use the present time to limit the push timestamp, which is less + // optimal but is guaranteed to progress. + // + // There is some inherent raciness here, because the lease may move + // between when we push and when we later read. In such cases, we may + // need to push again, but expect to eventually succeed in reading, + // either after lease movement subsides or after the reader's read + // timestamp surpasses its global uncertainty limit. + localMaxTS := req.Txn.MaxTimestamp + localMaxTS.Backward(w.clock.Now()) + h.Timestamp.Forward(localMaxTS) } return h } diff --git a/pkg/kv/kvserver/concurrency/lock_table_waiter_test.go b/pkg/kv/kvserver/concurrency/lock_table_waiter_test.go index 57819aebbcb2..bdd9198bc1d4 100644 --- a/pkg/kv/kvserver/concurrency/lock_table_waiter_test.go +++ b/pkg/kv/kvserver/concurrency/lock_table_waiter_test.go @@ -15,6 +15,7 @@ import ( "fmt" "math/rand" "testing" + "time" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/concurrency/lock" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/intentresolver" @@ -95,11 +96,13 @@ func setupLockTableWaiterTest() (*lockTableWaiterImpl, *mockIntentResolver, *moc st := cluster.MakeTestingClusterSettings() LockTableLivenessPushDelay.Override(&st.SV, 0) LockTableDeadlockDetectionPushDelay.Override(&st.SV, 0) + manual := hlc.NewManualClock(123) guard := &mockLockTableGuard{ signal: make(chan struct{}, 1), } w := &lockTableWaiterImpl{ st: st, + clock: hlc.NewClock(manual.UnixNano, time.Nanosecond), stopper: stop.NewStopper(), ir: ir, lt: &mockLockTable{}, diff --git a/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/uncertainty b/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/uncertainty index d282b510e1b5..c3a5328d48f0 100644 --- a/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/uncertainty +++ b/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/uncertainty @@ -59,6 +59,70 @@ finish req=req1 reset namespace ---- +# ------------------------------------------------------------- +# Same situation as above, only here, the read-only transaction +# has an uncertainty interval that extends past present time. +# The transaction only pushes to present time, not all the way +# to its max timestamp. See lockTableWaiterImpl.pushHeader. +# ------------------------------------------------------------- + +debug-set-clock ts=135 +---- + +new-txn name=txn1 ts=100,1 epoch=0 +---- + +new-txn name=txn2 ts=120,1 epoch=0 maxts=150,1 +---- + +new-request name=req1 txn=txn2 ts=120,1 + get key=k +---- + +sequence req=req1 +---- +[1] sequence req1: sequencing request +[1] sequence req1: acquiring latches +[1] sequence req1: scanning lock table for conflicting locks +[1] sequence req1: sequencing complete, returned guard + +handle-write-intent-error req=req1 lease-seq=1 + intent txn=txn1 key=k +---- +[2] handle write intent error req1: handled conflicting intents on "k", released latches + +debug-lock-table +---- +global: num=1 + lock: "k" + holder: txn: 00000001-0000-0000-0000-000000000000, ts: 100.000000000,1, info: repl epoch: 0, seqs: [0] +local: num=0 + +sequence req=req1 +---- +[3] sequence req1: re-sequencing request +[3] sequence req1: acquiring latches +[3] sequence req1: scanning lock table for conflicting locks +[3] sequence req1: waiting in lock wait-queues +[3] sequence req1: pushing timestamp of txn 00000001 above 135.000000000,0 +[3] sequence req1: blocked on select in concurrency_test.(*cluster).PushTransaction + +on-txn-updated txn=txn1 status=pending ts=135,1 +---- +[-] update txn: increasing timestamp of txn1 +[3] sequence req1: resolving intent "k" for txn 00000001 with PENDING status +[3] sequence req1: conflicted with 00000001-0000-0000-0000-000000000000 on "k" for 1.23s +[3] sequence req1: acquiring latches +[3] sequence req1: scanning lock table for conflicting locks +[3] sequence req1: sequencing complete, returned guard + +finish req=req1 +---- +[-] finish req1: finishing request + +reset namespace +---- + # ------------------------------------------------------------- # A transactional (txn2) read-only request runs into a replicated # intent below its read timestamp and informs the lock table. diff --git a/pkg/kv/kvserver/intentresolver/intent_resolver.go b/pkg/kv/kvserver/intentresolver/intent_resolver.go index 4eece296aae0..c3deb6a4129c 100644 --- a/pkg/kv/kvserver/intentresolver/intent_resolver.go +++ b/pkg/kv/kvserver/intentresolver/intent_resolver.go @@ -376,6 +376,10 @@ func (ir *IntentResolver) MaybePushTransactions( return nil, b.MustPErr() } + // TODO(nvanbenschoten): if we succeed because the transaction has already + // been pushed _past_ where we were pushing, we need to set the synthetic + // bit. This is part of #36431. + br := b.RawResponse() pushedTxns := make(map[uuid.UUID]*roachpb.Transaction, len(br.Responses)) for _, resp := range br.Responses { diff --git a/pkg/kv/kvserver/replica.go b/pkg/kv/kvserver/replica.go index 7fc882c80136..9f8e9c045a64 100644 --- a/pkg/kv/kvserver/replica.go +++ b/pkg/kv/kvserver/replica.go @@ -849,7 +849,7 @@ func (r *Replica) GetRangeInfo(ctx context.Context) roachpb.RangeInfo { // is enabled and the TTL has passed. If this is an admin command or this range // contains data outside of the user keyspace, we return the true GC threshold. func (r *Replica) getImpliedGCThresholdRLocked( - st *kvserverpb.LeaseStatus, isAdmin bool, + st kvserverpb.LeaseStatus, isAdmin bool, ) hlc.Timestamp { threshold := *r.mu.state.GCThreshold @@ -1154,50 +1154,98 @@ func (r *Replica) assertStateLocked(ctx context.Context, reader storage.Reader) } } +// TODO(nvanbenschoten): move the following 5 methods to replica_send.go. + // checkExecutionCanProceed returns an error if a batch request cannot be // executed by the Replica. An error indicates that the Replica is not live and // able to serve traffic or that the request is not compatible with the state of -// the Range. +// the Range due to the range's key bounds, the range's lease, or the ranges GC +// threshold. // // The method accepts a concurrency Guard, which is used to indicate whether the // caller has acquired latches. When this condition is false, the batch request -// will not wait for a pending merge to conclude before proceeding. Callers might -// be ok with this if they know that they will end up checking for a pending -// merge at some later time. -// -// NB: We record and return the result of `mergeInProgress()` here because we use -// it to assert that no request that bumps the LeaseAppliedIndex of a range is -// proposed to Raft while a range is subsumed. This is a correctness invariant -// for range merges. See comment block inside Subsume() in cmd_subsume.go for -// more details. +// will not wait for a pending merge to conclude before proceeding. Callers +// might be ok with this if they know that they will end up checking for a +// pending merge at some later time. func (r *Replica) checkExecutionCanProceed( - ctx context.Context, ba *roachpb.BatchRequest, g *concurrency.Guard, st *kvserverpb.LeaseStatus, -) (bool, error) { + ctx context.Context, ba *roachpb.BatchRequest, g *concurrency.Guard, +) (kvserverpb.LeaseStatus, error) { + var st kvserverpb.LeaseStatus + var shouldExtend bool + defer func() { + if shouldExtend { + r.maybeExtendLeaseAsync(ctx, st) + } + }() + + now := r.Clock().NowAsClockTimestamp() rSpan, err := keys.Range(ba.Requests) if err != nil { - return false, err + return kvserverpb.LeaseStatus{}, err } + r.mu.RLock() defer r.mu.RUnlock() - mergeInProgress := r.mergeInProgressRLocked() + + // Is the replica destroyed? if _, err := r.isDestroyedRLocked(); err != nil { - return mergeInProgress, err - } else if err := r.checkSpanInRangeRLocked(ctx, rSpan); err != nil { - return mergeInProgress, err - } else if err := r.checkTSAboveGCThresholdRLocked( - ba.EarliestActiveTimestamp(), st, ba.IsAdmin(), - ); err != nil { - return mergeInProgress, err - } else if mergeInProgress && g.HoldingLatches() { - // We only call `shouldWaitForPendingMergeRLocked` if we're currently holding - // latches. In practice, this means that any request where - // concurrency.shouldAcquireLatches() == false (e.g. RequestLeaseRequests) will - // not wait for a pending merge before executing and, as such, can execute while - // a range is in a merge's critical phase (i.e. while the RHS of the merge is + return kvserverpb.LeaseStatus{}, err + } + + // Is the request fully contained in the range? + // NB: we only need to check that the request is in the Range's key bounds + // at evaluation time, not at application time, because the spanlatch manager + // will synchronize all requests (notably EndTxn with SplitTrigger) that may + // cause this condition to change. + if err := r.checkSpanInRangeRLocked(ctx, rSpan); err != nil { + return kvserverpb.LeaseStatus{}, err + } + + // Is the lease valid? + if !ba.ReadConsistency.RequiresReadLease() { + // For INCONSISTENT requests, we don't need the lease. + st = kvserverpb.LeaseStatus{ + Now: now, + } + } else if ba.IsSingleSkipLeaseCheckRequest() { + // For lease commands, use the provided previous lease for verification. + st = kvserverpb.LeaseStatus{ + Lease: ba.GetPrevLeaseForLeaseRequest(), + Now: now, + } + } else { + // If the request is a write or a consistent read, it requires the + // replica serving it to hold the range lease. + st, shouldExtend, err = r.leaseGoodToGoRLocked(ctx, now, ba.Timestamp) + if err != nil { + // If not, can we serve this request on a follower? + // TODO(nvanbenschoten): once we make this check cheaper + // than leaseGoodToGoRLocked, invert these checks. + if !r.canServeFollowerReadRLocked(ctx, ba, err) { + return st, err + } + } + } + + // Is the request below the GC threshold? + if err := r.checkTSAboveGCThresholdRLocked(ba.EarliestActiveTimestamp(), st, ba.IsAdmin()); err != nil { + return kvserverpb.LeaseStatus{}, err + } + + // Is there a merge in progress? + if r.mergeInProgressRLocked() && g.HoldingLatches() { + // We only check for a merge if we are holding latches. In practice, + // this means that any request where concurrency.shouldAcquireLatches() + // is false (e.g. RequestLeaseRequests) will not wait for a pending + // merge before executing and, as such, can execute while a range is in + // a merge's critical phase (i.e. while the RHS of the merge is // subsumed). - return mergeInProgress, r.shouldWaitForPendingMergeRLocked(ctx, ba) + if err := r.shouldWaitForPendingMergeRLocked(ctx, ba); err != nil { + return kvserverpb.LeaseStatus{}, err + } } - return mergeInProgress, nil + + return st, nil } // checkExecutionCanProceedForRangeFeed returns an error if a rangefeed request @@ -1213,7 +1261,7 @@ func (r *Replica) checkExecutionCanProceedForRangeFeed( return err } else if err := r.checkSpanInRangeRLocked(ctx, rSpan); err != nil { return err - } else if err := r.checkTSAboveGCThresholdRLocked(ts, &status, false /* isAdmin */); err != nil { + } else if err := r.checkTSAboveGCThresholdRLocked(ts, status, false /* isAdmin */); err != nil { return err } else if r.requiresExpiringLeaseRLocked() { // Ensure that the range does not require an expiration-based lease. If it @@ -1239,7 +1287,7 @@ func (r *Replica) checkSpanInRangeRLocked(ctx context.Context, rspan roachpb.RSp // checkTSAboveGCThresholdRLocked returns an error if a request (identified // by its MVCC timestamp) can be run on the replica. func (r *Replica) checkTSAboveGCThresholdRLocked( - ts hlc.Timestamp, st *kvserverpb.LeaseStatus, isAdmin bool, + ts hlc.Timestamp, st kvserverpb.LeaseStatus, isAdmin bool, ) error { threshold := r.getImpliedGCThresholdRLocked(st, isAdmin) if threshold.Less(ts) { @@ -1268,6 +1316,9 @@ func (r *Replica) shouldWaitForPendingMergeRLocked( return nil } + // TODO(nvanbenschoten): this isn't quite right. We shouldn't allow non-txn + // requests through here for the same reason why lease transfers can only + // allow concurrent reads > max_offset below the lease transfer time. if ba.IsReadOnly() { freezeStart := r.getFreezeStartRLocked() ts := ba.Timestamp @@ -1406,6 +1457,8 @@ func (r *Replica) isNewerThanSplitRLocked(split *roachpb.SplitTrigger) bool { r.mu.replicaID > rightDesc.ReplicaID } +// TODO(nvanbenschoten): move endCmds to replica_send.go. + // endCmds holds necessary information to end a batch after Raft // command processing. type endCmds struct { diff --git a/pkg/kv/kvserver/replica_follower_read.go b/pkg/kv/kvserver/replica_follower_read.go index c4f96effc5b1..02e5e1779e2d 100644 --- a/pkg/kv/kvserver/replica_follower_read.go +++ b/pkg/kv/kvserver/replica_follower_read.go @@ -20,6 +20,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/settings" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/cockroachdb/errors" ) // FollowerReadsEnabled controls whether replicas attempt to serve follower @@ -32,16 +33,16 @@ var FollowerReadsEnabled = settings.RegisterBoolSetting( true, ).WithPublic() -// canServeFollowerRead tests, when a range lease could not be acquired, whether -// the batch can be served as a follower read despite the error. Only +// canServeFollowerReadRLocked tests, when a range lease could not be acquired, +// whether the batch can be served as a follower read despite the error. Only // non-locking, read-only requests can be served as follower reads. The batch // must be composed exclusively only this kind of request to be accepted as a // follower read. -func (r *Replica) canServeFollowerRead( - ctx context.Context, ba *roachpb.BatchRequest, pErr *roachpb.Error, -) *roachpb.Error { - lErr, ok := pErr.GetDetail().(*roachpb.NotLeaseHolderError) - eligible := ok && +func (r *Replica) canServeFollowerReadRLocked( + ctx context.Context, ba *roachpb.BatchRequest, err error, +) bool { + var lErr *roachpb.NotLeaseHolderError + eligible := errors.As(err, &lErr) && lErr.LeaseHolder != nil && lErr.Lease.Type() == roachpb.LeaseEpoch && (!ba.IsLocking() && ba.IsAllTransactional()) && // followerreadsccl.batchCanBeEvaluatedOnFollower (ba.Txn == nil || !ba.Txn.IsLocking()) && // followerreadsccl.txnCanPerformFollowerRead @@ -49,19 +50,19 @@ func (r *Replica) canServeFollowerRead( if !eligible { // We couldn't do anything with the error, propagate it. - return pErr + return false } - repDesc, err := r.GetReplicaDescriptor() + repDesc, err := r.getReplicaDescriptorRLocked() if err != nil { - return roachpb.NewError(err) + return false } switch typ := repDesc.GetType(); typ { case roachpb.VOTER_FULL, roachpb.VOTER_INCOMING, roachpb.NON_VOTER: default: log.Eventf(ctx, "%s replicas cannot serve follower reads", typ) - return pErr + return false } ts := ba.Timestamp @@ -69,7 +70,7 @@ func (r *Replica) canServeFollowerRead( ts.Forward(ba.Txn.MaxTimestamp) } - maxClosed, _ := r.maxClosed(ctx) + maxClosed, _ := r.maxClosedRLocked(ctx) canServeFollowerRead := ts.LessEq(maxClosed) tsDiff := ts.GoTime().Sub(maxClosed.GoTime()) if !canServeFollowerRead { @@ -93,7 +94,7 @@ func (r *Replica) canServeFollowerRead( r.store.cfg.ClosedTimestamp.Storage.(*ctstorage.MultiStorage).StringForNodes(lErr.LeaseHolder.NodeID), ) } - return pErr + return false } // This replica can serve this read! @@ -102,7 +103,7 @@ func (r *Replica) canServeFollowerRead( // serve reads for that and smaller timestamps forever. log.Eventf(ctx, "%s; query timestamp below closed timestamp by %s", kvbase.FollowerReadServingMsg, -tsDiff) r.store.metrics.FollowerReadsCount.Inc(1) - return nil + return true } // maxClosed returns the maximum closed timestamp for this range. @@ -118,10 +119,15 @@ func (r *Replica) canServeFollowerRead( // is false. func (r *Replica) maxClosed(ctx context.Context) (_ hlc.Timestamp, ok bool) { r.mu.RLock() + defer r.mu.RUnlock() + return r.maxClosedRLocked(ctx) +} + +func (r *Replica) maxClosedRLocked(ctx context.Context) (_ hlc.Timestamp, ok bool) { lai := r.mu.state.LeaseAppliedIndex lease := *r.mu.state.Lease initialMaxClosed := r.mu.initialMaxClosed - r.mu.RUnlock() + if lease.Expiration != nil { return hlc.Timestamp{}, false } diff --git a/pkg/kv/kvserver/replica_raft.go b/pkg/kv/kvserver/replica_raft.go index baea23e13120..bfa8ea29ae1d 100644 --- a/pkg/kv/kvserver/replica_raft.go +++ b/pkg/kv/kvserver/replica_raft.go @@ -77,6 +77,7 @@ func (r *Replica) evalAndPropose( // If the request hit a server-side concurrency retry error, immediately // proagate the error. Don't assume ownership of the concurrency guard. if isConcurrencyRetryError(pErr) { + pErr = maybeAttachLease(pErr, lease) return nil, nil, 0, pErr } diff --git a/pkg/kv/kvserver/replica_range_lease.go b/pkg/kv/kvserver/replica_range_lease.go index 21dc12ed9ea5..9167083c9827 100644 --- a/pkg/kv/kvserver/replica_range_lease.go +++ b/pkg/kv/kvserver/replica_range_lease.go @@ -477,13 +477,12 @@ func (p *pendingLeaseRequest) JoinRequest() *leaseRequestHandle { return llHandle } -// TransferInProgress returns the next lease, if the replica is in the process -// of transferring away its range lease. This next lease indicates the next -// lease holder. The second return val is true if a transfer is in progress. -// Note that the return values are best-effort and shouldn't be relied upon for -// correctness: if a previous transfer has returned an error, TransferInProgress -// will return `false`, but that doesn't necessarily mean that the transfer -// cannot still apply (see replica.mu.minLeaseProposedTS). +// TransferInProgress returns whether the replica is in the process of +// transferring away its range lease. Note that the return values are +// best-effort and shouldn't be relied upon for correctness: if a previous +// transfer has returned an error, TransferInProgress will return `false`, but +// that doesn't necessarily mean that the transfer cannot still apply (see +// replica.mu.minLeaseProposedTS). // // It is assumed that the replica owning this pendingLeaseRequest owns the // LeaderLease. @@ -491,16 +490,12 @@ func (p *pendingLeaseRequest) JoinRequest() *leaseRequestHandle { // replicaID is the ID of the parent replica. // // Requires repl.mu is read locked. -func (p *pendingLeaseRequest) TransferInProgress( - replicaID roachpb.ReplicaID, -) (roachpb.Lease, bool) { +func (p *pendingLeaseRequest) TransferInProgress(replicaID roachpb.ReplicaID) bool { if nextLease, ok := p.RequestPending(); ok { // Is the lease being transferred? (as opposed to just extended) - if replicaID != nextLease.Replica.ReplicaID { - return nextLease, true - } + return replicaID != nextLease.Replica.ReplicaID } - return roachpb.Lease{}, false + return false } // newHandle creates a new leaseRequestHandle referencing the pending lease @@ -714,8 +709,6 @@ func (r *Replica) requiresExpiringLeaseRLocked() bool { // there's already a request in progress, we join in waiting for the results of // that request. Unless an error is returned, the obtained lease will be valid // for a time interval containing the requested timestamp. -// If a transfer is in progress, a NotLeaseHolderError directing to the recipient is -// sent on the returned chan. func (r *Replica) requestLeaseLocked( ctx context.Context, status kvserverpb.LeaseStatus, ) *leaseRequestHandle { @@ -724,16 +717,6 @@ func (r *Replica) requestLeaseLocked( return r.mu.pendingLeaseRequest.newResolvedHandle(err) } } - // Propose a Raft command to get a lease for this replica. - repDesc, err := r.getReplicaDescriptorRLocked() - if err != nil { - return r.mu.pendingLeaseRequest.newResolvedHandle(roachpb.NewError(err)) - } - if transferLease, ok := r.mu.pendingLeaseRequest.TransferInProgress(repDesc.ReplicaID); ok { - return r.mu.pendingLeaseRequest.newResolvedHandle(roachpb.NewError( - newNotLeaseHolderError(transferLease, r.store.StoreID(), r.mu.state.Desc, - "lease transfer in progress"))) - } // If we're draining, we'd rather not take any new leases (since we're also // trying to move leases away elsewhere). But if we're the leader, we don't // really have a choice and we take the lease - there might not be any other @@ -746,27 +729,35 @@ func (r *Replica) requestLeaseLocked( newNotLeaseHolderError(roachpb.Lease{}, r.store.StoreID(), r.mu.state.Desc, "refusing to take the lease; node is draining"))) } + // Propose a Raft command to get a lease for this replica. + repDesc, err := r.getReplicaDescriptorRLocked() + if err != nil { + return r.mu.pendingLeaseRequest.newResolvedHandle(roachpb.NewError(err)) + } return r.mu.pendingLeaseRequest.InitOrJoinRequest( ctx, repDesc, status, r.mu.state.Desc.StartKey.AsRawKey(), false /* transfer */) } -// AdminTransferLease transfers the LeaderLease to another replica. A -// valid LeaseStatus must be supplied. Only the current holder of the -// LeaderLease can do a transfer, because it needs to stop serving -// reads and proposing Raft commands (CPut is a read) after sending -// the transfer command. If it did not stop serving reads immediately, -// it would potentially serve reads with timestamps greater than the -// start timestamp of the new (transferred) lease. More subtly, the -// replica can't even serve reads or propose commands with timestamps -// lower than the start of the new lease because it could lead to read -// your own write violations (see comments on the stasis period in -// IsLeaseValid). We could, in principle, serve reads more than the -// maximum clock offset in the past. +// AdminTransferLease transfers the LeaderLease to another replica. Only the +// current holder of the LeaderLease can do a transfer, because it needs to stop +// serving reads and proposing Raft commands (CPut is a read) while evaluating +// and proposing the TransferLease request. This synchronization with all other +// requests on the leaseholder is enforced through latching. The TransferLease +// request grabs a write latch over all keys in the range. +// +// If the leaseholder did not respect latching and did not stop serving reads +// during the lease transfer, it would potentially serve reads with timestamps +// greater than the start timestamp of the new (transferred) lease, which is +// determined during the evaluation of the TransferLease request. More subtly, +// the replica can't even serve reads or propose commands with timestamps lower +// than the start of the new lease because it could lead to read your own write +// violations (see comments on the stasis period on leaseStatus). We could, in +// principle, serve reads more than the maximum clock offset in the past. // // The method waits for any in-progress lease extension to be done, and it also -// blocks until the transfer is done. If a transfer is already in progress, -// this method joins in waiting for it to complete if it's transferring to the -// same replica. Otherwise, a NotLeaseHolderError is returned. +// blocks until the transfer is done. If a transfer is already in progress, this +// method joins in waiting for it to complete if it's transferring to the same +// replica. Otherwise, a NotLeaseHolderError is returned. func (r *Replica) AdminTransferLease(ctx context.Context, target roachpb.StoreID) error { // initTransferHelper inits a transfer if no extension is in progress. // It returns a channel for waiting for the result of a pending @@ -822,7 +813,14 @@ func (r *Replica) AdminTransferLease(ctx context.Context, target roachpb.StoreID return nil, nil, newNotLeaseHolderError(nextLease, r.store.StoreID(), desc, "another transfer to a different store is in progress") } - // Stop using the current lease. + // Stop using the current lease. All future calls to leaseStatus on this + // node with the current lease will now return a PROSCRIBED status. + // + // TODO(nvanbenschoten): since we aren't pulling the transfer time here + // anymore, we could also move this below latching as well, similar to + // how a Subsume request sets the FreezeStart time and pauses closed + // timestamps during evaluation and communicates this back up using the + // LocalResult. r.mu.minLeaseProposedTS = now transfer = r.mu.pendingLeaseRequest.InitOrJoinRequest( ctx, nextLeaseHolder, status, desc.StartKey.AsRawKey(), true, /* transfer */ @@ -911,35 +909,6 @@ func newNotLeaseHolderError( return err } -// leaseGoodToGo is a fast-path for lease checks which verifies that an -// existing lease is valid, owned by the current store, and usable to -// serve requests at the specified timestamp. This method should not be -// called directly. Use redirectOnOrAcquireLease instead. -func (r *Replica) leaseGoodToGo( - ctx context.Context, reqTS hlc.Timestamp, -) (kvserverpb.LeaseStatus, bool) { - now := r.store.Clock().NowAsClockTimestamp() - r.mu.RLock() - defer r.mu.RUnlock() - - if r.requiresExpiringLeaseRLocked() { - // Slow-path for expiration-based leases. - return kvserverpb.LeaseStatus{}, false - } - - status := r.leaseStatusForRequestRLocked(ctx, now, reqTS) - if status.IsValid() && status.Lease.OwnedBy(r.store.StoreID()) { - // We own the lease... - if repDesc, err := r.getReplicaDescriptorRLocked(); err == nil { - if _, ok := r.mu.pendingLeaseRequest.TransferInProgress(repDesc.ReplicaID); !ok { - // ...and there is no transfer pending. - return status, true - } - } - } - return kvserverpb.LeaseStatus{}, false -} - // checkRequestTimeRLocked checks that the provided request timestamp is not // too far in the future. We define "too far" as a time that would require a // lease extension even if we were perfectly proactive about extending our @@ -950,9 +919,7 @@ func (r *Replica) leaseGoodToGo( // This serves as a stricter version of a check that if we were to perform // a lease extension at now, the request would be contained within the new // lease's expiration (and stasis period). -func (r *Replica) checkRequestTimeRLocked( - now hlc.ClockTimestamp, reqTS hlc.Timestamp, -) *roachpb.Error { +func (r *Replica) checkRequestTimeRLocked(now hlc.ClockTimestamp, reqTS hlc.Timestamp) error { var leaseRenewal time.Duration if r.requiresExpiringLeaseRLocked() { _, leaseRenewal = r.store.cfg.RangeLeaseDurations() @@ -968,17 +935,120 @@ func (r *Replica) checkRequestTimeRLocked( } maxReqTS := now.ToTimestamp().Add(leaseRenewalMinusStasis.Nanoseconds(), 0) if maxReqTS.Less(reqTS) { - return roachpb.NewErrorf("request timestamp %s too far in future (> %s)", reqTS, maxReqTS) + return errors.Errorf("request timestamp %s too far in future (> %s)", reqTS, maxReqTS) } return nil } -// redirectOnOrAcquireLease checks whether this replica has the lease at the -// current timestamp. If it does, returns the lease and its status. If -// another replica currently holds the lease, redirects by returning -// NotLeaseHolderError. If the lease is expired, a renewal is synchronously -// requested. Leases are eagerly renewed when a request with a timestamp -// within RangeLeaseRenewalDuration of the lease expiration is served. +// leaseGoodToGoRLocked verifies that the replica has a lease that is +// valid, owned by the current replica, and usable to serve requests at +// the specified timestamp. The method will return the lease status, +// along with an error indicating whether any of these conditions is +// unsatisfied. The lease status is either empty or fully populated. +// +// Latches must be acquired on the range before calling this method. +// This ensures that callers are properly sequenced with TransferLease +// requests, which declare a conflict with all other commands. +// +// The method can has four possible outcomes: +// +// (1) the request timestamp is too far in the future. In this case, +// a nonstructured error is returned. This shouldn't happen. +// (2) the lease is invalid or otherwise unable to serve a request at +// the specified timestamp. In this case, an InvalidLeaseError is +// returned, which is caught in executeBatchWithConcurrencyRetries +// and used to trigger a lease acquisition/extension. +// (3) the lease is valid but held by a different replica. In this case, +// a NotLeaseHolderError is returned, which is propagated back up to +// the DistSender and triggers a redirection of the request. +// (4) the lease is valid, held locally, and capable of serving the +// given request. In this case, no error is returned. +// +// In addition to the lease status, the method also returns whether the +// lease should be considered for extension using maybeExtendLeaseAsync +// after the replica's read lock has been dropped. +func (r *Replica) leaseGoodToGoRLocked( + ctx context.Context, now hlc.ClockTimestamp, reqTS hlc.Timestamp, +) (_ kvserverpb.LeaseStatus, shouldExtend bool, _ error) { + if err := r.checkRequestTimeRLocked(now, reqTS); err != nil { + // Case (1): invalid request. + return kvserverpb.LeaseStatus{}, false, err + } + + st := r.leaseStatusForRequestRLocked(ctx, now, reqTS) + if !st.IsValid() { + // Case (2): invalid lease. + return st, false, &roachpb.InvalidLeaseError{} + } + if !st.Lease.OwnedBy(r.store.StoreID()) { + // Case (3): not leaseholder. + _, stillMember := r.mu.state.Desc.GetReplicaDescriptor(st.Lease.Replica.StoreID) + if !stillMember { + // This would be the situation in which the lease holder gets removed when + // holding the lease, or in which a lease request erroneously gets accepted + // for a replica that is not in the replica set. Neither of the two can + // happen in normal usage since appropriate mechanisms have been added: + // + // 1. Only the lease holder (at the time) schedules removal of a replica, + // but the lease can change hands and so the situation in which a follower + // coordinates a replica removal of the (new) lease holder is possible (if + // unlikely) in practice. In this situation, the new lease holder would at + // some point be asked to propose the replica change's EndTxn to Raft. A + // check has been added that prevents proposals that amount to the removal + // of the proposer's (and hence lease holder's) Replica, preventing this + // scenario. + // + // 2. A lease is accepted for a Replica that has been removed. Without + // precautions, this could happen because lease requests are special in + // that they are the only command that is proposed on a follower (other + // commands may be proposed from followers, but not successfully so). For + // all proposals, processRaftCommand checks that their ProposalLease is + // compatible with the active lease for the log position. For commands + // proposed on the lease holder, the spanlatch manager then serializes + // everything. But lease requests get created on followers based on their + // local state and thus without being sequenced through latching. Thus + // a recently removed follower (unaware of its own removal) could submit + // a proposal for the lease (correctly using as a ProposerLease the last + // active lease), and would receive it given the up-to-date ProposerLease. + // Hence, an extra check is in order: processRaftCommand makes sure that + // lease requests for a replica not in the descriptor are bounced. + // + // However, this is possible if the `cockroach debug + // unsafe-remove-dead-replicas` command has been used, so + // this is just a logged error instead of a fatal + // assertion. + log.Errorf(ctx, "lease %s owned by replica %+v that no longer exists", + st.Lease, st.Lease.Replica) + } + // Otherwise, if the lease is currently held by another replica, redirect + // to the holder. + return st, false, newNotLeaseHolderError( + st.Lease, r.store.StoreID(), r.descRLocked(), "lease held by different store", + ) + } + // Case (4): all good. + return st, r.shouldExtendLeaseRLocked(st), nil +} + +// leaseGoodToGo is like leaseGoodToGoRLocked, but will acquire the replica read +// lock. +func (r *Replica) leaseGoodToGo( + ctx context.Context, now hlc.ClockTimestamp, reqTS hlc.Timestamp, +) (_ kvserverpb.LeaseStatus, shouldExtend bool, _ error) { + r.mu.RLock() + defer r.mu.RUnlock() + return r.leaseGoodToGoRLocked(ctx, now, reqTS) +} + +// redirectOnOrAcquireLease checks whether this replica has the lease at +// the current timestamp. If it does, returns the lease and its status. +// If another replica currently holds the lease, redirects by returning +// NotLeaseHolderError and an empty lease status. +// +// If the lease is expired, a renewal is synchronously requested. +// Expiration-based leases are eagerly renewed when a request with a +// timestamp within RangeLeaseRenewalDuration of the lease expiration is +// served. // // TODO(spencer): for write commands, don't wait while requesting // the range lease. If the lease acquisition fails, the write cmd @@ -998,25 +1068,36 @@ func (r *Replica) redirectOnOrAcquireLeaseForRequest( ctx context.Context, reqTS hlc.Timestamp, ) (kvserverpb.LeaseStatus, *roachpb.Error) { // Try fast-path. - if status, ok := r.leaseGoodToGo(ctx, reqTS); ok { + now := r.store.Clock().NowAsClockTimestamp() + status, shouldExtend, err := r.leaseGoodToGo(ctx, now, reqTS) + if err == nil { + if shouldExtend { + r.maybeExtendLeaseAsync(ctx, status) + } return status, nil + } else if !errors.HasType(err, (*roachpb.InvalidLeaseError)(nil)) { + return kvserverpb.LeaseStatus{}, roachpb.NewError(err) } - // If fast-path fails, loop until the lease is held or the replica - // ascertains the actual lease holder. Returns also on context.Done() - // (timeout or cancellation). - var status kvserverpb.LeaseStatus + // Loop until the lease is held or the replica ascertains the actual lease + // holder. Returns also on context.Done() (timeout or cancellation). for attempt := 1; ; attempt++ { - now := r.store.Clock().NowAsClockTimestamp() - llHandle, pErr := func() (*leaseRequestHandle, *roachpb.Error) { + now = r.store.Clock().NowAsClockTimestamp() + llHandle, transfer, pErr := func() (*leaseRequestHandle, bool, *roachpb.Error) { r.mu.Lock() defer r.mu.Unlock() - // Check to make sure we aren't trying to perform a request at an - // invalid timestamp too far in the future. If so, reject instead - // of trying to acquire a lease to satisfy the request. - if pErr := r.checkRequestTimeRLocked(now, reqTS); pErr != nil { - return nil, pErr + // Check that we're not in the process of transferring the lease + // away. If we are doing so, we can't serve reads or propose Raft + // commands - see comments on AdminTransferLease and TransferLease. + // So wait on the lease transfer to complete either successfully or + // unsuccessfully before redirecting or retrying. + repDesc, err := r.getReplicaDescriptorRLocked() + if err != nil { + return nil, false, roachpb.NewError(err) + } + if ok := r.mu.pendingLeaseRequest.TransferInProgress(repDesc.ReplicaID); ok { + return r.mu.pendingLeaseRequest.JoinRequest(), true /* transfer */, nil } status = r.leaseStatusForRequestRLocked(ctx, now, reqTS) @@ -1024,7 +1105,7 @@ func (r *Replica) redirectOnOrAcquireLeaseForRequest( case kvserverpb.LeaseState_ERROR: // Lease state couldn't be determined. log.VEventf(ctx, 2, "lease state couldn't be determined") - return nil, roachpb.NewError( + return nil, false, roachpb.NewError( newNotLeaseHolderError(roachpb.Lease{}, r.store.StoreID(), r.mu.state.Desc, "lease state couldn't be determined")) @@ -1032,94 +1113,30 @@ func (r *Replica) redirectOnOrAcquireLeaseForRequest( if !status.Lease.OwnedBy(r.store.StoreID()) { _, stillMember := r.mu.state.Desc.GetReplicaDescriptor(status.Lease.Replica.StoreID) if !stillMember { - // This would be the situation in which the lease holder gets removed when - // holding the lease, or in which a lease request erroneously gets accepted - // for a replica that is not in the replica set. Neither of the two can - // happen in normal usage since appropriate mechanisms have been added: - // - // 1. Only the lease holder (at the time) schedules removal of a replica, - // but the lease can change hands and so the situation in which a follower - // coordinates a replica removal of the (new) lease holder is possible (if - // unlikely) in practice. In this situation, the new lease holder would at - // some point be asked to propose the replica change's EndTxn to Raft. A - // check has been added that prevents proposals that amount to the removal - // of the proposer's (and hence lease holder's) Replica, preventing this - // scenario. - // - // 2. A lease is accepted for a Replica that has been removed. Without - // precautions, this could happen because lease requests are special in - // that they are the only command that is proposed on a follower (other - // commands may be proposed from followers, but not successfully so). For - // all proposals, processRaftCommand checks that their ProposalLease is - // compatible with the active lease for the log position. For commands - // proposed on the lease holder, the spanlatch manager then serializes - // everything. But lease requests get created on followers based on their - // local state and thus without being sequenced through latching. Thus - // a recently removed follower (unaware of its own removal) could submit - // a proposal for the lease (correctly using as a ProposerLease the last - // active lease), and would receive it given the up-to-date ProposerLease. - // Hence, an extra check is in order: processRaftCommand makes sure that - // lease requests for a replica not in the descriptor are bounced. - // - // However, this is possible if the `cockroach debug - // unsafe-remove-dead-replicas` command has been used, so - // this is just a logged error instead of a fatal - // assertion. + // See corresponding comment in leaseGoodToGoRLocked. log.Errorf(ctx, "lease %s owned by replica %+v that no longer exists", status.Lease, status.Lease.Replica) } // Otherwise, if the lease is currently held by another replica, redirect // to the holder. - return nil, roachpb.NewError( + return nil, false, roachpb.NewError( newNotLeaseHolderError(status.Lease, r.store.StoreID(), r.mu.state.Desc, "lease held by different store")) } - // Check that we're not in the process of transferring the lease away. - // If we are transferring the lease away, we can't serve reads or - // propose Raft commands - see comments on TransferLease. - // TODO(andrei): If the lease is being transferred, consider returning a - // new error type so the client backs off until the transfer is - // completed. - repDesc, err := r.getReplicaDescriptorRLocked() - if err != nil { - return nil, roachpb.NewError(err) - } - if transferLease, ok := r.mu.pendingLeaseRequest.TransferInProgress( - repDesc.ReplicaID); ok { - return nil, roachpb.NewError( - newNotLeaseHolderError(transferLease, r.store.StoreID(), r.mu.state.Desc, - "lease transfer in progress")) - } // If the lease is in stasis, we can't serve requests until we've // renewed the lease, so we return the handle to block on renewal. - // Otherwise, we don't need to wait for the extension and simply - // ignore the returned handle (whose channel is buffered) and continue. if status.State == kvserverpb.LeaseState_UNUSABLE { - return r.requestLeaseLocked(ctx, status), nil + return r.requestLeaseLocked(ctx, status), false, nil } - // Extend the lease if this range uses expiration-based - // leases, the lease is in need of renewal, and there's not - // already an extension pending. - _, requestPending := r.mu.pendingLeaseRequest.RequestPending() - if !requestPending && r.requiresExpiringLeaseRLocked() { - renewal := status.Lease.Expiration.Add(-r.store.cfg.RangeLeaseRenewalDuration().Nanoseconds(), 0) - if renewal.LessEq(now.ToTimestamp()) { - if log.V(2) { - log.Infof(ctx, "extending lease %s at %s", status.Lease, now) - } - // We had an active lease to begin with, but we want to trigger - // a lease extension. We explicitly ignore the returned handle - // as we won't block on it. - _ = r.requestLeaseLocked(ctx, status) - } - } + // Return a nil handle to signal that we have a valid lease. + return nil, false, nil case kvserverpb.LeaseState_EXPIRED: // No active lease: Request renewal if a renewal is not already pending. log.VEventf(ctx, 2, "request range lease (attempt #%d)", attempt) - return r.requestLeaseLocked(ctx, status), nil + return r.requestLeaseLocked(ctx, status), false, nil case kvserverpb.LeaseState_PROSCRIBED: // Lease proposed timestamp is earlier than the min proposed @@ -1127,15 +1144,15 @@ func (r *Replica) redirectOnOrAcquireLeaseForRequest( // owns the lease, re-request. Otherwise, redirect. if status.Lease.OwnedBy(r.store.StoreID()) { log.VEventf(ctx, 2, "request range lease (attempt #%d)", attempt) - return r.requestLeaseLocked(ctx, status), nil + return r.requestLeaseLocked(ctx, status), false, nil } // If lease is currently held by another, redirect to holder. - return nil, roachpb.NewError( + return nil, false, roachpb.NewError( newNotLeaseHolderError(status.Lease, r.store.StoreID(), r.mu.state.Desc, "lease proscribed")) - } - // Return a nil handle to signal that we have a valid lease. - return nil, nil + default: + return nil, false, roachpb.NewErrorf("unknown lease status state %v", status) + } }() if pErr != nil { return kvserverpb.LeaseStatus{}, pErr @@ -1145,7 +1162,15 @@ func (r *Replica) redirectOnOrAcquireLeaseForRequest( return status, nil } - // Wait for the range lease to finish, or the context to expire. + // Wait for the range lease acquisition/transfer to finish, or the + // context to expire. + // + // Note that even if the operation completes successfully, we can't + // assume that we have the lease. This is clearly not the case when + // waiting on a lease transfer and also not the case if our request + // timestamp is not covered by the new lease (though we try to protect + // against this in checkRequestTimeRLocked). So instead of assuming + // anything, we iterate and check again. pErr = func() (pErr *roachpb.Error) { slowTimer := timeutil.NewTimer() defer slowTimer.Stop() @@ -1154,6 +1179,12 @@ func (r *Replica) redirectOnOrAcquireLeaseForRequest( for { select { case pErr = <-llHandle.C(): + if transfer { + // We were waiting on a transfer to finish. Ignore its + // result and try again. + return nil + } + if pErr != nil { switch tErr := pErr.GetDetail().(type) { case *roachpb.AmbiguousResultError: @@ -1216,5 +1247,40 @@ func (r *Replica) redirectOnOrAcquireLeaseForRequest( if pErr != nil { return kvserverpb.LeaseStatus{}, pErr } + // Retry... + } +} + +// shouldExtendLeaseRLocked determines whether the lease should be +// extended asynchronously, even if it is currently valid. The method +// returns true if this range uses expiration-based leases, the lease is +// in need of renewal, and there's not already an extension pending. +func (r *Replica) shouldExtendLeaseRLocked(st kvserverpb.LeaseStatus) bool { + if !r.requiresExpiringLeaseRLocked() { + return false + } + if _, ok := r.mu.pendingLeaseRequest.RequestPending(); ok { + return false + } + renewal := st.Lease.Expiration.Add(-r.store.cfg.RangeLeaseRenewalDuration().Nanoseconds(), 0) + return renewal.LessEq(st.Now.ToTimestamp()) +} + +// maybeExtendLeaseAsync attempts to extend the expiration-based lease +// asynchronously, if doing so is deemed beneficial by an earlier call +// to shouldExtendLeaseRLocked. +func (r *Replica) maybeExtendLeaseAsync(ctx context.Context, st kvserverpb.LeaseStatus) { + r.mu.Lock() + defer r.mu.Unlock() + // Check shouldExtendLeaseRLocked again, because others may have raced to + // extend the lease and beaten us here after we made the determination + // (under a shared lock) that the extension was needed. + if !r.shouldExtendLeaseRLocked(st) { + return + } + if log.ExpensiveLogEnabled(ctx, 2) { + log.Infof(ctx, "extending lease %s at %s", st.Lease, st.Now) } + // We explicitly ignore the returned handle as we won't block on it. + _ = r.requestLeaseLocked(ctx, st) } diff --git a/pkg/kv/kvserver/replica_read.go b/pkg/kv/kvserver/replica_read.go index 03984471f1ce..3a86c36caf0a 100644 --- a/pkg/kv/kvserver/replica_read.go +++ b/pkg/kv/kvserver/replica_read.go @@ -17,7 +17,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval/result" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/concurrency" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase" - "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverpb" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/observedts" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/spanset" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/storage" @@ -32,16 +32,23 @@ import ( // iterator to evaluate the batch and then updates the timestamp cache to // reflect the key spans that it read. func (r *Replica) executeReadOnlyBatch( - ctx context.Context, ba *roachpb.BatchRequest, st kvserverpb.LeaseStatus, g *concurrency.Guard, + ctx context.Context, ba *roachpb.BatchRequest, g *concurrency.Guard, ) (br *roachpb.BatchResponse, _ *concurrency.Guard, pErr *roachpb.Error) { r.readOnlyCmdMu.RLock() defer r.readOnlyCmdMu.RUnlock() // Verify that the batch can be executed. - if _, err := r.checkExecutionCanProceed(ctx, ba, g, &st); err != nil { + st, err := r.checkExecutionCanProceed(ctx, ba, g) + if err != nil { return nil, g, roachpb.NewError(err) } + // Limit the transaction's maximum timestamp using observed timestamps. + // TODO(nvanbenschoten): now that we've pushed this down here, consider + // keeping the "local max timestamp" on the stack and never modifying the + // batch. + ba.Txn = observedts.LimitTxnMaxTimestamp(ctx, ba.Txn, st) + // Evaluate read-only batch command. spans := g.LatchSpans() rec := NewReplicaEvalContext(r, spans) @@ -74,6 +81,7 @@ func (r *Replica) executeReadOnlyBatch( // If the request hit a server-side concurrency retry error, immediately // proagate the error. Don't assume ownership of the concurrency guard. if isConcurrencyRetryError(pErr) { + pErr = maybeAttachLease(pErr, &st.Lease) return nil, g, pErr } diff --git a/pkg/kv/kvserver/replica_send.go b/pkg/kv/kvserver/replica_send.go index e10ac9dd4f8c..50b8467ad085 100644 --- a/pkg/kv/kvserver/replica_send.go +++ b/pkg/kv/kvserver/replica_send.go @@ -17,7 +17,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/concurrency" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverpb" - "github.com/cockroachdb/cockroach/pkg/kv/kvserver/observedts" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/spanset" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/txnwait" "github.com/cockroachdb/cockroach/pkg/roachpb" @@ -178,10 +177,9 @@ func (r *Replica) maybeAddRangeInfoToResponse( } } -// batchExecutionFn is a method on Replica that is able to execute a -// BatchRequest. It is called with the batch, along with the status of -// the lease that the batch is operating under and a guard for the -// latches protecting the request. +// batchExecutionFn is a method on Replica that executes a BatchRequest. It is +// called with the batch, along with a guard for the latches protecting the +// request. // // The function will return either a batch response or an error. The function // also has the option to pass ownership of the concurrency guard back to the @@ -204,7 +202,7 @@ func (r *Replica) maybeAddRangeInfoToResponse( // the function returns one of these errors, it must also pass ownership of the // concurrency guard back to the caller. type batchExecutionFn func( - *Replica, context.Context, *roachpb.BatchRequest, kvserverpb.LeaseStatus, *concurrency.Guard, + *Replica, context.Context, *roachpb.BatchRequest, *concurrency.Guard, ) (*roachpb.BatchResponse, *concurrency.Guard, *roachpb.Error) var _ batchExecutionFn = (*Replica).executeWriteBatch @@ -227,9 +225,18 @@ var _ batchExecutionFn = (*Replica).executeReadOnlyBatch func (r *Replica) executeBatchWithConcurrencyRetries( ctx context.Context, ba *roachpb.BatchRequest, fn batchExecutionFn, ) (br *roachpb.BatchResponse, pErr *roachpb.Error) { + // Determine the maximal set of key spans that the batch will operate on. + // This is used below to sequence the request in the concurrency manager. + latchSpans, lockSpans, err := r.collectSpans(ba) + if err != nil { + return nil, roachpb.NewError(err) + } + + // Handle load-based splitting. + r.recordBatchForLoadBasedSplitting(ctx, ba, latchSpans) + // Try to execute command; exit retry loop on success. var g *concurrency.Guard - var latchSpans, lockSpans *spanset.SpanSet defer func() { // NB: wrapped to delay g evaluation to its value when returning. if g != nil { @@ -242,41 +249,6 @@ func (r *Replica) executeBatchWithConcurrencyRetries( return nil, roachpb.NewError(errors.Wrap(err, "aborted during Replica.Send")) } - // Determine the lease under which to evaluate the request. - var status kvserverpb.LeaseStatus - if !ba.ReadConsistency.RequiresReadLease() { - // Get a clock reading for checkExecutionCanProceed. - status.Now = r.Clock().NowAsClockTimestamp() - } else if ba.IsSingleSkipLeaseCheckRequest() { - // For lease commands, use the provided previous lease for verification. - status.Lease = ba.GetPrevLeaseForLeaseRequest() - status.Now = r.Clock().NowAsClockTimestamp() - } else { - // If the request is a write or a consistent read, it requires the - // range lease or permission to serve via follower reads. - if status, pErr = r.redirectOnOrAcquireLeaseForRequest(ctx, ba.Timestamp); pErr != nil { - if nErr := r.canServeFollowerRead(ctx, ba, pErr); nErr != nil { - return nil, nErr - } - } - } - // Limit the transaction's maximum timestamp using observed timestamps. - ba.Txn = observedts.LimitTxnMaxTimestamp(ctx, ba.Txn, status) - - // Determine the maximal set of key spans that the batch will operate - // on. We only need to do this once and we make sure to do so after we - // have limited the transaction's maximum timestamp. - if latchSpans == nil { - var err error - latchSpans, lockSpans, err = r.collectSpans(ba) - if err != nil { - return nil, roachpb.NewError(err) - } - - // Handle load-based splitting. - r.recordBatchForLoadBasedSplitting(ctx, ba, latchSpans) - } - // Acquire latches to prevent overlapping requests from executing until // this request completes. After latching, wait on any conflicting locks // to ensure that the request has full isolation during evaluation. This @@ -300,13 +272,7 @@ func (r *Replica) executeBatchWithConcurrencyRetries( return br, nil } - if filter := r.store.cfg.TestingKnobs.TestingLatchFilter; filter != nil { - if pErr := filter(ctx, *ba); pErr != nil { - return nil, pErr - } - } - - br, g, pErr = fn(r, ctx, ba, status, g) + br, g, pErr = fn(r, ctx, ba, g) if pErr == nil { // Success. return br, nil @@ -325,7 +291,7 @@ func (r *Replica) executeBatchWithConcurrencyRetries( switch t := pErr.GetDetail().(type) { case *roachpb.WriteIntentError: // Drop latches, but retain lock wait-queues. - if g, pErr = r.handleWriteIntentError(ctx, ba, g, status.Lease, pErr, t); pErr != nil { + if g, pErr = r.handleWriteIntentError(ctx, ba, g, pErr, t); pErr != nil { return nil, pErr } case *roachpb.TransactionPushError: @@ -341,6 +307,16 @@ func (r *Replica) executeBatchWithConcurrencyRetries( if pErr = r.handleIndeterminateCommitError(ctx, ba, pErr, t); pErr != nil { return nil, pErr } + case *roachpb.InvalidLeaseError: + // Drop latches and lock wait-queues. + r.concMgr.FinishReq(g) + g = nil + // Then attempt to acquire the lease if not currently held by any + // replica or redirect to the current leaseholder if currently held + // by a different replica. + if pErr = r.handleInvalidLeaseError(ctx, ba, pErr, t); pErr != nil { + return nil, pErr + } case *roachpb.MergeInProgressError: // Drop latches and lock wait-queues. r.concMgr.FinishReq(g) @@ -379,6 +355,13 @@ func isConcurrencyRetryError(pErr *roachpb.Error) bool { // the pushee is aborted or committed, so the request must kick off the // "transaction recovery procedure" to resolve this ambiguity before // retrying. + case *roachpb.InvalidLeaseError: + // If a request hits an InvalidLeaseError, the replica it is being + // evaluated against does not have a valid lease under which it can + // serve the request. The request cannot proceed until a new lease is + // acquired. If the acquisition process determines that the lease now + // lives elsewhere, the request should be redirected (using a + // NotLeaseHolderError) to the new leaseholder. case *roachpb.MergeInProgressError: // If a request hits a MergeInProgressError, the replica it is being // evaluted against is in the process of being merged into its left-hand @@ -392,11 +375,21 @@ func isConcurrencyRetryError(pErr *roachpb.Error) bool { return true } +// maybeAttachLease is used to augment a concurrency retry error with +// information about the lease that the operation which hit this error was +// operating under. +func maybeAttachLease(pErr *roachpb.Error, lease *roachpb.Lease) *roachpb.Error { + if wiErr, ok := pErr.GetDetail().(*roachpb.WriteIntentError); ok { + wiErr.LeaseSequence = lease.Sequence + return roachpb.NewErrorWithTxn(wiErr, pErr.GetTxn()) + } + return pErr +} + func (r *Replica) handleWriteIntentError( ctx context.Context, ba *roachpb.BatchRequest, g *concurrency.Guard, - lease roachpb.Lease, pErr *roachpb.Error, t *roachpb.WriteIntentError, ) (*concurrency.Guard, *roachpb.Error) { @@ -404,7 +397,7 @@ func (r *Replica) handleWriteIntentError( return g, pErr } // g's latches will be dropped, but it retains its spot in lock wait-queues. - return r.concMgr.HandleWriterIntentError(ctx, g, lease.Sequence, t) + return r.concMgr.HandleWriterIntentError(ctx, g, t.LeaseSequence, t) } func (r *Replica) handleTransactionPushError( @@ -456,6 +449,31 @@ func (r *Replica) handleIndeterminateCommitError( return nil } +func (r *Replica) handleInvalidLeaseError( + ctx context.Context, ba *roachpb.BatchRequest, _ *roachpb.Error, t *roachpb.InvalidLeaseError, +) *roachpb.Error { + // On an invalid lease error, attempt to acquire a new lease. If in the + // process of doing so, we determine that the lease now lives elsewhere, + // redirect. + _, pErr := r.redirectOnOrAcquireLeaseForRequest(ctx, ba.Timestamp) + if pErr == nil { + // Lease valid. Retry command. + return nil + } + // If we failed to acquire the lease, check to see whether the request can + // still be served as a follower read on this replica. Doing so here will + // not be necessary once we break the dependency between closed timestamps + // and leases and address the TODO in checkExecutionCanProceed to check the + // closed timestamp before consulting the lease. + r.mu.RLock() + defer r.mu.RUnlock() + if r.canServeFollowerReadRLocked(ctx, ba, pErr.GoError()) { + // Follower read possible. Retry command. + return nil + } + return pErr +} + func (r *Replica) handleMergeInProgressError( ctx context.Context, ba *roachpb.BatchRequest, @@ -501,7 +519,7 @@ func (r *Replica) executeAdminBatch( } // Admin commands always require the range lease. - status, pErr := r.redirectOnOrAcquireLease(ctx) + _, pErr := r.redirectOnOrAcquireLease(ctx) if pErr != nil { return nil, pErr } @@ -511,7 +529,7 @@ func (r *Replica) executeAdminBatch( // NB: we pass nil for the spanlatch guard because we haven't acquired // latches yet. This is ok because each individual request that the admin // request sends will acquire latches. - if _, err := r.checkExecutionCanProceed(ctx, ba, nil /* g */, &status); err != nil { + if _, err := r.checkExecutionCanProceed(ctx, ba, nil /* g */); err != nil { return nil, roachpb.NewError(err) } diff --git a/pkg/kv/kvserver/replica_test.go b/pkg/kv/kvserver/replica_test.go index ec752f9c9462..af4f93da533f 100644 --- a/pkg/kv/kvserver/replica_test.go +++ b/pkg/kv/kvserver/replica_test.go @@ -86,7 +86,7 @@ var allSpans = func() spanset.SpanSet { Key: roachpb.KeyMin, EndKey: roachpb.KeyMax, }) - // Local keys (see `keys.localPrefix`). + // Local keys (see `keys.LocalPrefix`). ss.AddNonMVCC(spanset.SpanReadWrite, roachpb.Span{ Key: append([]byte("\x01"), roachpb.KeyMin...), EndKey: append([]byte("\x01"), roachpb.KeyMax...), @@ -705,8 +705,8 @@ func TestReplicaReadConsistency(t *testing.T) { } // Test the behavior of a replica while a range lease transfer is in progress: -// - while the transfer is in progress, reads should return errors pointing to -// the transfer target. +// - while the transfer is in progress, reads should block while attempting to +// acquire latches. // - if a transfer fails, the pre-existing lease does not start being used // again. Instead, a new lease needs to be obtained. This is because, even // though the transfer got an error, that error is considered ambiguous as the @@ -714,140 +714,152 @@ func TestReplicaReadConsistency(t *testing.T) { func TestBehaviorDuringLeaseTransfer(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) - manual := hlc.NewManualClock(123) - clock := hlc.NewClock(manual.UnixNano, 100*time.Millisecond) - tc := testContext{manualClock: manual} - tsc := TestStoreConfig(clock) - var leaseAcquisitionTrap atomic.Value - tsc.TestingKnobs.DisableAutomaticLeaseRenewal = true - tsc.TestingKnobs.LeaseRequestEvent = func(ts hlc.Timestamp, _ roachpb.StoreID, _ roachpb.RangeID) *roachpb.Error { - val := leaseAcquisitionTrap.Load() - if val == nil { + + testutils.RunTrueAndFalse(t, "transferSucceeds", func(t *testing.T, transferSucceeds bool) { + manual := hlc.NewManualClock(123) + clock := hlc.NewClock(manual.UnixNano, 100*time.Millisecond) + tc := testContext{manualClock: manual} + tsc := TestStoreConfig(clock) + var leaseAcquisitionTrap atomic.Value + tsc.TestingKnobs.DisableAutomaticLeaseRenewal = true + tsc.TestingKnobs.LeaseRequestEvent = func(ts hlc.Timestamp, _ roachpb.StoreID, _ roachpb.RangeID) *roachpb.Error { + val := leaseAcquisitionTrap.Load() + if val == nil { + return nil + } + trapCallback := val.(func(ts hlc.Timestamp)) + if trapCallback != nil { + trapCallback(ts) + } return nil } - trapCallback := val.(func(ts hlc.Timestamp)) - if trapCallback != nil { - trapCallback(ts) - } - return nil - } - transferSem := make(chan struct{}) - tsc.TestingKnobs.EvalKnobs.TestingEvalFilter = - func(filterArgs kvserverbase.FilterArgs) *roachpb.Error { - if _, ok := filterArgs.Req.(*roachpb.TransferLeaseRequest); ok { - // Notify the test that the transfer has been trapped. - transferSem <- struct{}{} - // Wait for the test to unblock the transfer. - <-transferSem - // Return an error, so that the pendingLeaseRequest considers the - // transfer failed. - return roachpb.NewErrorf("injected transfer error") + transferSem := make(chan struct{}) + tsc.TestingKnobs.EvalKnobs.TestingEvalFilter = + func(filterArgs kvserverbase.FilterArgs) *roachpb.Error { + if _, ok := filterArgs.Req.(*roachpb.TransferLeaseRequest); ok { + // Notify the test that the transfer has been trapped. + transferSem <- struct{}{} + // Wait for the test to unblock the transfer. + <-transferSem + if !transferSucceeds { + // Return an error, so that the pendingLeaseRequest + // considers the transfer failed. + return roachpb.NewErrorf("injected transfer error") + } + return nil + } + return nil } - return nil + stopper := stop.NewStopper() + defer stopper.Stop(context.Background()) + tc.StartWithStoreConfig(t, stopper, tsc) + secondReplica, err := tc.addBogusReplicaToRangeDesc(context.Background()) + if err != nil { + t.Fatal(err) } - stopper := stop.NewStopper() - defer stopper.Stop(context.Background()) - tc.StartWithStoreConfig(t, stopper, tsc) - secondReplica, err := tc.addBogusReplicaToRangeDesc(context.Background()) - if err != nil { - t.Fatal(err) - } - // Do a read to acquire the lease. - gArgs := getArgs(roachpb.Key("a")) - if _, err := tc.SendWrapped(&gArgs); err != nil { - t.Fatal(err) - } + // Do a read to acquire the lease. + gArgs := getArgs(roachpb.Key("a")) + if _, err := tc.SendWrapped(&gArgs); err != nil { + t.Fatal(err) + } - // Advance the clock so that the transfer we're going to perform sets a higher - // minLeaseProposedTS. - tc.manualClock.Increment((500 * time.Nanosecond).Nanoseconds()) + // Advance the clock so that the transfer we're going to perform sets a higher + // minLeaseProposedTS. + tc.manualClock.Increment((500 * time.Nanosecond).Nanoseconds()) - // Initiate a transfer (async) and wait for it to be blocked. - transferResChan := make(chan error) - go func() { - err := tc.repl.AdminTransferLease(context.Background(), secondReplica.StoreID) - if !testutils.IsError(err, "injected") { - transferResChan <- err - } else { - transferResChan <- nil + // Initiate a transfer (async) and wait for it to be blocked. + transferResChan := make(chan error) + go func() { + err := tc.repl.AdminTransferLease(context.Background(), secondReplica.StoreID) + if !testutils.IsError(err, "injected") { + transferResChan <- err + } else { + transferResChan <- nil + } + }() + <-transferSem + // Check that a transfer is indeed on-going. + tc.repl.mu.Lock() + repDesc, err := tc.repl.getReplicaDescriptorRLocked() + if err != nil { + tc.repl.mu.Unlock() + t.Fatal(err) } - }() - <-transferSem - // Check that a transfer is indeed on-going. - tc.repl.mu.Lock() - repDesc, err := tc.repl.getReplicaDescriptorRLocked() - if err != nil { + pending := tc.repl.mu.pendingLeaseRequest.TransferInProgress(repDesc.ReplicaID) tc.repl.mu.Unlock() - t.Fatal(err) - } - _, pending := tc.repl.mu.pendingLeaseRequest.TransferInProgress(repDesc.ReplicaID) - tc.repl.mu.Unlock() - if !pending { - t.Fatalf("expected transfer to be in progress, and it wasn't") - } - - // Check that, while the transfer is on-going, the replica redirects to the - // transfer target. - _, pErr := tc.SendWrapped(&gArgs) - nlhe, ok := pErr.GetDetail().(*roachpb.NotLeaseHolderError) - if !ok || nlhe.LeaseHolder.StoreID != secondReplica.StoreID { - t.Fatalf("expected not lease holder error pointing to store %d, got %v", - secondReplica.StoreID, pErr) - } + if !pending { + t.Fatalf("expected transfer to be in progress, and it wasn't") + } - // Unblock the transfer and wait for the pendingLeaseRequest to clear the - // transfer state. - transferSem <- struct{}{} - if err := <-transferResChan; err != nil { - t.Fatal(err) - } + // Check that, while the transfer is on-going, reads are blocked. + readResChan := make(chan error) + go func() { + _, pErr := tc.SendWrapped(&gArgs) + readResChan <- pErr.GoError() + }() + select { + case err := <-readResChan: + t.Fatalf("expected read to block, completed with err=%v", err) + case <-time.After(10 * time.Millisecond): + } + + // Set up a hook to detect lease requests, in case this transfer fails. + expectedLeaseStartTS := tc.manualClock.UnixNano() + leaseAcquisitionCh := make(chan error) + leaseAcquisitionTrap.Store(func(ts hlc.Timestamp) { + if ts.WallTime == expectedLeaseStartTS { + close(leaseAcquisitionCh) + } else { + leaseAcquisitionCh <- errors.Errorf( + "expected acquisition of lease with start: %d but got start: %s", + expectedLeaseStartTS, ts) + } + }) - testutils.SucceedsSoon(t, func() error { - tc.repl.mu.Lock() - defer tc.repl.mu.Unlock() - _, pending := tc.repl.mu.pendingLeaseRequest.TransferInProgress(repDesc.ReplicaID) - if pending { - return errors.New("transfer pending") + // Unblock the transfer and wait for the pendingLeaseRequest to clear + // the transfer state. + transferSem <- struct{}{} + if err := <-transferResChan; err != nil { + t.Fatal(err) } - return nil - }) - // Check that the replica doesn't use its lease, even though there's no longer - // a transfer in progress. This is because, even though the transfer got an - // error, that error is considered ambiguous as the transfer might still - // apply. - // Concretely, we're going to check that a read triggers a new lease - // acquisition. - tc.repl.mu.Lock() - minLeaseProposedTS := tc.repl.mu.minLeaseProposedTS - leaseStartTS := tc.repl.mu.state.Lease.Start - tc.repl.mu.Unlock() - if minLeaseProposedTS.LessEq(leaseStartTS) { - t.Fatalf("expected minLeaseProposedTS > lease start. minLeaseProposedTS: %s, "+ - "leas start: %s", minLeaseProposedTS, leaseStartTS) - } - expectedLeaseStartTS := tc.manualClock.UnixNano() - leaseAcquisitionCh := make(chan error) - leaseAcquisitionTrap.Store(func(ts hlc.Timestamp) { - if ts.WallTime == expectedLeaseStartTS { - close(leaseAcquisitionCh) + testutils.SucceedsSoon(t, func() error { + tc.repl.mu.Lock() + defer tc.repl.mu.Unlock() + pending := tc.repl.mu.pendingLeaseRequest.TransferInProgress(repDesc.ReplicaID) + if pending { + return errors.New("transfer pending") + } + return nil + }) + + if transferSucceeds { + // Check that the read is rejected and redirected to the transfer + // target. + err = <-readResChan + require.Error(t, err) + var lErr *roachpb.NotLeaseHolderError + require.True(t, errors.As(err, &lErr)) + require.Equal(t, secondReplica.StoreID, lErr.LeaseHolder.StoreID) } else { - leaseAcquisitionCh <- errors.Errorf( - "expected acquisition of lease with start: %d but got start: %s", - expectedLeaseStartTS, ts) + // Check that the replica doesn't use its lease, even though there's + // no longer a transfer in progress. This is because, even though + // the transfer got an error, that error is considered ambiguous as + // the transfer might still apply. + // + // Concretely, we're going to check that a read triggered a new + // lease acquisition and that is then succeeds. + select { + case <-leaseAcquisitionCh: + case <-time.After(time.Second): + t.Fatalf("read did not acquire a new lease") + } + + err = <-readResChan + require.NoError(t, err) } }) - // We expect this call to succeed, but after acquiring a new lease. - if _, err := tc.SendWrapped(&gArgs); err != nil { - t.Fatal(err) - } - // Check that the Send above triggered a lease acquisition. - select { - case <-leaseAcquisitionCh: - case <-time.After(time.Second): - t.Fatalf("read did not acquire a new lease") - } } // TestApplyCmdLeaseError verifies that when during application of a Raft @@ -1458,12 +1470,10 @@ func TestReplicaDrainLease(t *testing.T) { // expired already. // Stop n1's heartbeats and wait for the lease to expire. - log.Infof(ctx, "test: suspending heartbeats for n1") cleanup := s1.NodeLiveness().(*liveness.NodeLiveness).PauseAllHeartbeatsForTest() defer cleanup() - require.NoError(t, err) testutils.SucceedsSoon(t, func() error { status := r1.CurrentLeaseStatus(ctx) require.True(t, status.Lease.OwnedBy(store1.StoreID()), "someone else got the lease: %s", status) diff --git a/pkg/kv/kvserver/replica_write.go b/pkg/kv/kvserver/replica_write.go index a160ca1102df..231bf462588f 100644 --- a/pkg/kv/kvserver/replica_write.go +++ b/pkg/kv/kvserver/replica_write.go @@ -22,6 +22,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/liveness" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/observedts" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/spanset" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/storage" @@ -64,7 +65,7 @@ import ( // as this method makes the assumption that it operates on a shallow copy (see // call to applyTimestampCache). func (r *Replica) executeWriteBatch( - ctx context.Context, ba *roachpb.BatchRequest, st kvserverpb.LeaseStatus, g *concurrency.Guard, + ctx context.Context, ba *roachpb.BatchRequest, g *concurrency.Guard, ) (br *roachpb.BatchResponse, _ *concurrency.Guard, pErr *roachpb.Error) { startTime := timeutil.Now() @@ -75,15 +76,17 @@ func (r *Replica) executeWriteBatch( // if we hit an error during evaluation (e.g. a ConditionFailedError)? // Verify that the batch can be executed. - // NB: we only need to check that the request is in the Range's key bounds - // at proposal time, not at application time, because the spanlatch manager - // will synchronize all requests (notably EndTxn with SplitTrigger) that may - // cause this condition to change. - mergeInProgress, err := r.checkExecutionCanProceed(ctx, ba, g, &st) + st, err := r.checkExecutionCanProceed(ctx, ba, g) if err != nil { return nil, g, roachpb.NewError(err) } + // Limit the transaction's maximum timestamp using observed timestamps. + // TODO(nvanbenschoten): now that we've pushed this down here, consider + // keeping the "local max timestamp" on the stack and never modifying the + // batch. + ba.Txn = observedts.LimitTxnMaxTimestamp(ctx, ba.Txn, st) + minTS, untrack := r.store.cfg.ClosedTimestamp.Tracker.Track(ctx) defer untrack(ctx, 0, 0, 0) // covers all error returns below @@ -116,19 +119,6 @@ func (r *Replica) executeWriteBatch( return nil, g, roachpb.NewError(errors.Wrap(err, "aborted before proposing")) } - // Check that the lease is still valid before proposing to avoid discovering - // this after replication and potentially missing out on the chance to retry - // if the request is using AsyncConsensus. This is best-effort, but can help - // in cases where the request waited arbitrarily long for locks acquired by - // other transactions to be released while sequencing in the concurrency - // manager. - if curLease, _ := r.GetLease(); curLease.Sequence > st.Lease.Sequence { - err := newNotLeaseHolderError(curLease, r.store.StoreID(), r.Desc(), - "stale lease discovered before proposing") - log.VEventf(ctx, 2, "%s before proposing: %s", err, ba.Summary()) - return nil, g, roachpb.NewError(err) - } - // If the command is proposed to Raft, ownership of and responsibility for // the concurrency guard will be assumed by Raft, so provide the guard to // evalAndPropose. @@ -150,18 +140,6 @@ func (r *Replica) executeWriteBatch( // cannot communicate under the lease's epoch. Instead the code calls EmitMLAI // explicitly as a side effect of stepping up as leaseholder. if maxLeaseIndex != 0 { - if mergeInProgress { - // The correctness of range merges relies on the invariant that the - // LeaseAppliedIndex of the range is not bumped while a range is in its - // subsumed state. If this invariant is ever violated, the follower - // replicas of the subsumed range (RHS) are free to activate any future - // closed timestamp updates even before the merge completes. This would be - // a serializability violation. - // - // See comment block in Subsume() in cmd_subsume.go for details. - log.Fatalf(ctx, - "lease applied index bumped by %v while the range was subsumed", ba) - } untrack(ctx, ctpb.Epoch(st.Lease.Epoch), r.RangeID, ctpb.LAI(maxLeaseIndex)) } diff --git a/pkg/kv/kvserver/store_test.go b/pkg/kv/kvserver/store_test.go index 2a7bf51e2473..b943e45c1375 100644 --- a/pkg/kv/kvserver/store_test.go +++ b/pkg/kv/kvserver/store_test.go @@ -721,8 +721,7 @@ func TestStoreRemoveReplicaDestroy(t *testing.T) { t.Fatal("replica was not marked as destroyed") } - st := &kvserverpb.LeaseStatus{Now: repl1.Clock().NowAsClockTimestamp()} - if _, err = repl1.checkExecutionCanProceed(ctx, &roachpb.BatchRequest{}, nil /* g */, st); !errors.Is(err, expErr) { + if _, err = repl1.checkExecutionCanProceed(ctx, &roachpb.BatchRequest{}, nil /* g */); !errors.Is(err, expErr) { t.Fatalf("expected error %s, but got %v", expErr, err) } } diff --git a/pkg/kv/kvserver/testing_knobs.go b/pkg/kv/kvserver/testing_knobs.go index d195fb5dbe85..96ad6bff8468 100644 --- a/pkg/kv/kvserver/testing_knobs.go +++ b/pkg/kv/kvserver/testing_knobs.go @@ -41,12 +41,6 @@ type StoreTestingKnobs struct { // returns an error, the command will not be evaluated. TestingRequestFilter kvserverbase.ReplicaRequestFilter - // TestingLatchFilter is called before evaluating each command on a replica - // but after acquiring latches for the command. Blocking in the filter will - // block interfering requests. If it returns an error, the command will not - // be evaluated. - TestingLatchFilter kvserverbase.ReplicaRequestFilter - // TestingConcurrencyRetryFilter is called before a concurrency retry error is // handled and the batch is retried. TestingConcurrencyRetryFilter kvserverbase.ReplicaConcurrencyRetryFilter diff --git a/pkg/roachpb/batch_generated.go b/pkg/roachpb/batch_generated.go index 99556c74f790..c6d948bc2486 100644 --- a/pkg/roachpb/batch_generated.go +++ b/pkg/roachpb/batch_generated.go @@ -68,6 +68,8 @@ func (ru ErrorDetail) GetInner() error { return t.RangefeedRetry case *ErrorDetail_IndeterminateCommit: return t.IndeterminateCommit + case *ErrorDetail_InvalidLeaseError: + return t.InvalidLeaseError default: return nil } @@ -328,6 +330,8 @@ func (ru *ErrorDetail) MustSetInner(r error) { union = &ErrorDetail_RangefeedRetry{t} case *IndeterminateCommitError: union = &ErrorDetail_IndeterminateCommit{t} + case *InvalidLeaseError: + union = &ErrorDetail_InvalidLeaseError{t} default: panic(fmt.Sprintf("unsupported type %T for %T", r, ru)) } diff --git a/pkg/roachpb/data.go b/pkg/roachpb/data.go index 034243264f8f..01ac8f9beaa0 100644 --- a/pkg/roachpb/data.go +++ b/pkg/roachpb/data.go @@ -884,6 +884,9 @@ func MakeTransaction( name string, baseKey Key, userPriority UserPriority, now hlc.Timestamp, maxOffsetNs int64, ) Transaction { u := uuid.FastMakeV4() + // TODO(nvanbenschoten): technically, maxTS should be a synthetic timestamp. + // Make this change in v21.2 when all nodes in a cluster are guaranteed to + // be aware of synthetic timestamps by addressing the TODO in Timestamp.Add. maxTS := now.Add(maxOffsetNs, 0) return Transaction{ diff --git a/pkg/roachpb/errordetailtype_string.go b/pkg/roachpb/errordetailtype_string.go index 11a59bf5e82a..0ddcaacd392e 100644 --- a/pkg/roachpb/errordetailtype_string.go +++ b/pkg/roachpb/errordetailtype_string.go @@ -36,6 +36,7 @@ func _() { _ = x[MergeInProgressErrType-37] _ = x[RangeFeedRetryErrType-38] _ = x[IndeterminateCommitErrType-39] + _ = x[InvalidLeaseErrType-40] _ = x[CommunicationErrType-22] _ = x[InternalErrType-25] } @@ -46,7 +47,7 @@ const ( _ErrorDetailType_name_2 = "CommunicationErrType" _ErrorDetailType_name_3 = "InternalErrTypeAmbiguousResultErrTypeStoreNotFoundErrTypeTransactionRetryWithProtoRefreshErrType" _ErrorDetailType_name_4 = "IntegerOverflowErrTypeUnsupportedRequestErrType" - _ErrorDetailType_name_5 = "BatchTimestampBeforeGCErrTypeTxnAlreadyEncounteredErrTypeIntentMissingErrTypeMergeInProgressErrTypeRangeFeedRetryErrTypeIndeterminateCommitErrType" + _ErrorDetailType_name_5 = "BatchTimestampBeforeGCErrTypeTxnAlreadyEncounteredErrTypeIntentMissingErrTypeMergeInProgressErrTypeRangeFeedRetryErrTypeIndeterminateCommitErrTypeInvalidLeaseErrType" ) var ( @@ -54,7 +55,7 @@ var ( _ErrorDetailType_index_1 = [...]uint8{0, 23, 47, 67} _ErrorDetailType_index_3 = [...]uint8{0, 15, 37, 57, 96} _ErrorDetailType_index_4 = [...]uint8{0, 22, 47} - _ErrorDetailType_index_5 = [...]uint8{0, 29, 57, 77, 99, 120, 146} + _ErrorDetailType_index_5 = [...]uint8{0, 29, 57, 77, 99, 120, 146, 165} ) func (i ErrorDetailType) String() string { @@ -73,7 +74,7 @@ func (i ErrorDetailType) String() string { case 31 <= i && i <= 32: i -= 31 return _ErrorDetailType_name_4[_ErrorDetailType_index_4[i]:_ErrorDetailType_index_4[i+1]] - case 34 <= i && i <= 39: + case 34 <= i && i <= 40: i -= 34 return _ErrorDetailType_name_5[_ErrorDetailType_index_5[i]:_ErrorDetailType_index_5[i+1]] default: diff --git a/pkg/roachpb/errors.go b/pkg/roachpb/errors.go index 4aee815c3e65..cc19ae235519 100644 --- a/pkg/roachpb/errors.go +++ b/pkg/roachpb/errors.go @@ -316,6 +316,7 @@ const ( MergeInProgressErrType ErrorDetailType = 37 RangeFeedRetryErrType ErrorDetailType = 38 IndeterminateCommitErrType ErrorDetailType = 39 + InvalidLeaseErrType ErrorDetailType = 40 // When adding new error types, don't forget to update NumErrors below. // CommunicationErrType indicates a gRPC error; this is not an ErrorDetail. @@ -325,7 +326,7 @@ const ( // detail. The value 25 is chosen because it's reserved in the errors proto. InternalErrType ErrorDetailType = 25 - NumErrors int = 40 + NumErrors int = 41 ) // GoError returns a Go error converted from Error. If the error is a transaction @@ -1238,3 +1239,18 @@ func (e *IndeterminateCommitError) Type() ErrorDetailType { } var _ ErrorDetailInterface = &IndeterminateCommitError{} + +func (e *InvalidLeaseError) Error() string { + return e.message(nil) +} + +func (e *InvalidLeaseError) message(_ *Error) string { + return "invalid lease" +} + +// Type is part of the ErrorDetailInterface. +func (e *InvalidLeaseError) Type() ErrorDetailType { + return InvalidLeaseErrType +} + +var _ ErrorDetailInterface = &InvalidLeaseError{} diff --git a/pkg/roachpb/errors.pb.go b/pkg/roachpb/errors.pb.go index c38a0dae0955..49a3e03e599b 100644 --- a/pkg/roachpb/errors.pb.go +++ b/pkg/roachpb/errors.pb.go @@ -140,7 +140,7 @@ func (x *TransactionAbortedReason) UnmarshalJSON(data []byte) error { return nil } func (TransactionAbortedReason) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_errors_52db6b6dc82e2737, []int{0} + return fileDescriptor_errors_c3dba2b22d5a2b16, []int{0} } // TransactionRetryReason specifies what caused a transaction retry. @@ -191,7 +191,7 @@ func (x *TransactionRetryReason) UnmarshalJSON(data []byte) error { return nil } func (TransactionRetryReason) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_errors_52db6b6dc82e2737, []int{1} + return fileDescriptor_errors_c3dba2b22d5a2b16, []int{1} } // TransactionRestart indicates how an error should be handled in a @@ -242,7 +242,7 @@ func (x *TransactionRestart) UnmarshalJSON(data []byte) error { return nil } func (TransactionRestart) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_errors_52db6b6dc82e2737, []int{2} + return fileDescriptor_errors_c3dba2b22d5a2b16, []int{2} } // Reason specifies what caused the error. @@ -281,7 +281,7 @@ func (x *TransactionStatusError_Reason) UnmarshalJSON(data []byte) error { return nil } func (TransactionStatusError_Reason) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_errors_52db6b6dc82e2737, []int{9, 0} + return fileDescriptor_errors_c3dba2b22d5a2b16, []int{9, 0} } // Reason specifies what caused the error. @@ -337,7 +337,7 @@ func (x *RangeFeedRetryError_Reason) UnmarshalJSON(data []byte) error { return nil } func (RangeFeedRetryError_Reason) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_errors_52db6b6dc82e2737, []int{27, 0} + return fileDescriptor_errors_c3dba2b22d5a2b16, []int{27, 0} } // A NotLeaseHolderError indicates that the current range is not the lease @@ -368,7 +368,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_52db6b6dc82e2737, []int{0} + return fileDescriptor_errors_c3dba2b22d5a2b16, []int{0} } func (m *NotLeaseHolderError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -403,7 +403,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_52db6b6dc82e2737, []int{1} + return fileDescriptor_errors_c3dba2b22d5a2b16, []int{1} } func (m *NodeUnavailableError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -437,7 +437,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_52db6b6dc82e2737, []int{2} + return fileDescriptor_errors_c3dba2b22d5a2b16, []int{2} } func (m *UnsupportedRequestError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -474,7 +474,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_52db6b6dc82e2737, []int{3} + return fileDescriptor_errors_c3dba2b22d5a2b16, []int{3} } func (m *RangeNotFoundError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -525,7 +525,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_52db6b6dc82e2737, []int{4} + return fileDescriptor_errors_c3dba2b22d5a2b16, []int{4} } func (m *RangeKeyMismatchError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -570,7 +570,7 @@ type ReadWithinUncertaintyIntervalError struct { func (m *ReadWithinUncertaintyIntervalError) Reset() { *m = ReadWithinUncertaintyIntervalError{} } func (*ReadWithinUncertaintyIntervalError) ProtoMessage() {} func (*ReadWithinUncertaintyIntervalError) Descriptor() ([]byte, []int) { - return fileDescriptor_errors_52db6b6dc82e2737, []int{5} + return fileDescriptor_errors_c3dba2b22d5a2b16, []int{5} } func (m *ReadWithinUncertaintyIntervalError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -613,7 +613,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_52db6b6dc82e2737, []int{6} + return fileDescriptor_errors_c3dba2b22d5a2b16, []int{6} } func (m *TransactionAbortedError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -649,7 +649,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_52db6b6dc82e2737, []int{7} + return fileDescriptor_errors_c3dba2b22d5a2b16, []int{7} } func (m *TransactionPushError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -685,7 +685,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_52db6b6dc82e2737, []int{8} + return fileDescriptor_errors_c3dba2b22d5a2b16, []int{8} } func (m *TransactionRetryError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -725,7 +725,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_52db6b6dc82e2737, []int{9} + return fileDescriptor_errors_c3dba2b22d5a2b16, []int{9} } func (m *TransactionStatusError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -757,13 +757,17 @@ var xxx_messageInfo_TransactionStatusError proto.InternalMessageInfo // transactions. type WriteIntentError struct { Intents []Intent `protobuf:"bytes,1,rep,name=intents" json:"intents"` + // The sequence of the lease that the operation which hit this error was + // operating under. Used on the server to avoid adding discovered locks + // which were discovered under old leases to the lock table. + LeaseSequence LeaseSequence `protobuf:"varint,3,opt,name=lease_sequence,json=leaseSequence,casttype=LeaseSequence" json:"lease_sequence"` } 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_52db6b6dc82e2737, []int{10} + return fileDescriptor_errors_c3dba2b22d5a2b16, []int{10} } func (m *WriteIntentError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -808,7 +812,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_52db6b6dc82e2737, []int{11} + return fileDescriptor_errors_c3dba2b22d5a2b16, []int{11} } func (m *WriteTooOldError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -844,7 +848,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_52db6b6dc82e2737, []int{12} + return fileDescriptor_errors_c3dba2b22d5a2b16, []int{12} } func (m *OpRequiresTxnError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -881,7 +885,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_52db6b6dc82e2737, []int{13} + return fileDescriptor_errors_c3dba2b22d5a2b16, []int{13} } func (m *ConditionFailedError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -918,7 +922,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_52db6b6dc82e2737, []int{14} + return fileDescriptor_errors_c3dba2b22d5a2b16, []int{14} } func (m *LeaseRejectedError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -956,7 +960,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_52db6b6dc82e2737, []int{15} + return fileDescriptor_errors_c3dba2b22d5a2b16, []int{15} } func (m *AmbiguousResultError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -990,7 +994,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_52db6b6dc82e2737, []int{16} + return fileDescriptor_errors_c3dba2b22d5a2b16, []int{16} } func (m *RaftGroupDeletedError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1028,7 +1032,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_52db6b6dc82e2737, []int{17} + return fileDescriptor_errors_c3dba2b22d5a2b16, []int{17} } func (m *ReplicaCorruptionError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1065,7 +1069,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_52db6b6dc82e2737, []int{18} + return fileDescriptor_errors_c3dba2b22d5a2b16, []int{18} } func (m *ReplicaTooOldError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1100,7 +1104,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_52db6b6dc82e2737, []int{19} + return fileDescriptor_errors_c3dba2b22d5a2b16, []int{19} } func (m *StoreNotFoundError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1145,7 +1149,7 @@ type UnhandledRetryableError struct { func (m *UnhandledRetryableError) Reset() { *m = UnhandledRetryableError{} } func (*UnhandledRetryableError) ProtoMessage() {} func (*UnhandledRetryableError) Descriptor() ([]byte, []int) { - return fileDescriptor_errors_52db6b6dc82e2737, []int{20} + return fileDescriptor_errors_c3dba2b22d5a2b16, []int{20} } func (m *UnhandledRetryableError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1197,7 +1201,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_52db6b6dc82e2737, []int{21} + return fileDescriptor_errors_c3dba2b22d5a2b16, []int{21} } func (m *TransactionRetryWithProtoRefreshError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1235,7 +1239,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_52db6b6dc82e2737, []int{22} + return fileDescriptor_errors_c3dba2b22d5a2b16, []int{22} } func (m *TxnAlreadyEncounteredErrorError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1272,7 +1276,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_52db6b6dc82e2737, []int{23} + return fileDescriptor_errors_c3dba2b22d5a2b16, []int{23} } func (m *IntegerOverflowError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1308,7 +1312,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_52db6b6dc82e2737, []int{24} + return fileDescriptor_errors_c3dba2b22d5a2b16, []int{24} } func (m *BatchTimestampBeforeGCError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1347,7 +1351,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_52db6b6dc82e2737, []int{25} + return fileDescriptor_errors_c3dba2b22d5a2b16, []int{25} } func (m *IntentMissingError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1384,7 +1388,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_52db6b6dc82e2737, []int{26} + return fileDescriptor_errors_c3dba2b22d5a2b16, []int{26} } func (m *MergeInProgressError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1419,7 +1423,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_52db6b6dc82e2737, []int{27} + return fileDescriptor_errors_c3dba2b22d5a2b16, []int{27} } func (m *RangeFeedRetryError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1458,7 +1462,7 @@ func (m *IndeterminateCommitError) Reset() { *m = IndeterminateCommitErr func (m *IndeterminateCommitError) String() string { return proto.CompactTextString(m) } func (*IndeterminateCommitError) ProtoMessage() {} func (*IndeterminateCommitError) Descriptor() ([]byte, []int) { - return fileDescriptor_errors_52db6b6dc82e2737, []int{28} + return fileDescriptor_errors_c3dba2b22d5a2b16, []int{28} } func (m *IndeterminateCommitError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1483,6 +1487,45 @@ func (m *IndeterminateCommitError) XXX_DiscardUnknown() { var xxx_messageInfo_IndeterminateCommitError proto.InternalMessageInfo +// An InvalidLeaseError indicates that the request could not be completed +// because the replica does not have a valid lease. The request should be +// resubmitted on the current replica after a lease is acquired or should be +// redirected (using a NotLeaseHolderError) to the current leaseholder if one +// already exists. +// +// This error is handled by the Store and should not escape to higher levels. +type InvalidLeaseError struct { +} + +func (m *InvalidLeaseError) Reset() { *m = InvalidLeaseError{} } +func (m *InvalidLeaseError) String() string { return proto.CompactTextString(m) } +func (*InvalidLeaseError) ProtoMessage() {} +func (*InvalidLeaseError) Descriptor() ([]byte, []int) { + return fileDescriptor_errors_c3dba2b22d5a2b16, []int{29} +} +func (m *InvalidLeaseError) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *InvalidLeaseError) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *InvalidLeaseError) XXX_Merge(src proto.Message) { + xxx_messageInfo_InvalidLeaseError.Merge(dst, src) +} +func (m *InvalidLeaseError) XXX_Size() int { + return m.Size() +} +func (m *InvalidLeaseError) XXX_DiscardUnknown() { + xxx_messageInfo_InvalidLeaseError.DiscardUnknown(m) +} + +var xxx_messageInfo_InvalidLeaseError proto.InternalMessageInfo + // ErrorDetail is a union type containing all available errors. type ErrorDetail struct { // Types that are valid to be assigned to Value: @@ -1514,6 +1557,7 @@ type ErrorDetail struct { // *ErrorDetail_MergeInProgress // *ErrorDetail_RangefeedRetry // *ErrorDetail_IndeterminateCommit + // *ErrorDetail_InvalidLeaseError Value isErrorDetail_Value `protobuf_oneof:"value"` } @@ -1521,7 +1565,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_52db6b6dc82e2737, []int{29} + return fileDescriptor_errors_c3dba2b22d5a2b16, []int{30} } func (m *ErrorDetail) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1636,6 +1680,9 @@ type ErrorDetail_RangefeedRetry struct { type ErrorDetail_IndeterminateCommit struct { IndeterminateCommit *IndeterminateCommitError `protobuf:"bytes,39,opt,name=indeterminate_commit,json=indeterminateCommit,oneof"` } +type ErrorDetail_InvalidLeaseError struct { + InvalidLeaseError *InvalidLeaseError `protobuf:"bytes,40,opt,name=invalid_lease_error,json=invalidLeaseError,oneof"` +} func (*ErrorDetail_NotLeaseHolder) isErrorDetail_Value() {} func (*ErrorDetail_RangeNotFound) isErrorDetail_Value() {} @@ -1665,6 +1712,7 @@ func (*ErrorDetail_IntentMissing) isErrorDetail_Value() {} func (*ErrorDetail_MergeInProgress) isErrorDetail_Value() {} func (*ErrorDetail_RangefeedRetry) isErrorDetail_Value() {} func (*ErrorDetail_IndeterminateCommit) isErrorDetail_Value() {} +func (*ErrorDetail_InvalidLeaseError) isErrorDetail_Value() {} func (m *ErrorDetail) GetValue() isErrorDetail_Value { if m != nil { @@ -1869,6 +1917,13 @@ func (m *ErrorDetail) GetIndeterminateCommit() *IndeterminateCommitError { return nil } +func (m *ErrorDetail) GetInvalidLeaseError() *InvalidLeaseError { + if x, ok := m.GetValue().(*ErrorDetail_InvalidLeaseError); ok { + return x.InvalidLeaseError + } + return nil +} + // XXX_OneofFuncs is for the internal use of the proto package. func (*ErrorDetail) XXX_OneofFuncs() (func(msg proto.Message, b *proto.Buffer) error, func(msg proto.Message, tag, wire int, b *proto.Buffer) (bool, error), func(msg proto.Message) (n int), []interface{}) { return _ErrorDetail_OneofMarshaler, _ErrorDetail_OneofUnmarshaler, _ErrorDetail_OneofSizer, []interface{}{ @@ -1900,6 +1955,7 @@ func (*ErrorDetail) XXX_OneofFuncs() (func(msg proto.Message, b *proto.Buffer) e (*ErrorDetail_MergeInProgress)(nil), (*ErrorDetail_RangefeedRetry)(nil), (*ErrorDetail_IndeterminateCommit)(nil), + (*ErrorDetail_InvalidLeaseError)(nil), } } @@ -2047,6 +2103,11 @@ func _ErrorDetail_OneofMarshaler(msg proto.Message, b *proto.Buffer) error { if err := b.EncodeMessage(x.IndeterminateCommit); err != nil { return err } + case *ErrorDetail_InvalidLeaseError: + _ = b.EncodeVarint(40<<3 | proto.WireBytes) + if err := b.EncodeMessage(x.InvalidLeaseError); err != nil { + return err + } case nil: default: return fmt.Errorf("ErrorDetail.Value has unexpected type %T", x) @@ -2281,6 +2342,14 @@ func _ErrorDetail_OneofUnmarshaler(msg proto.Message, tag, wire int, b *proto.Bu err := b.DecodeMessage(msg) m.Value = &ErrorDetail_IndeterminateCommit{msg} return true, err + case 40: // value.invalid_lease_error + if wire != proto.WireBytes { + return true, proto.ErrInternalBadWireType + } + msg := new(InvalidLeaseError) + err := b.DecodeMessage(msg) + m.Value = &ErrorDetail_InvalidLeaseError{msg} + return true, err default: return false, nil } @@ -2430,6 +2499,11 @@ func _ErrorDetail_OneofSizer(msg proto.Message) (n int) { n += 2 // tag and wire n += proto.SizeVarint(uint64(s)) n += s + case *ErrorDetail_InvalidLeaseError: + s := proto.Size(x.InvalidLeaseError) + n += 2 // tag and wire + n += proto.SizeVarint(uint64(s)) + n += s case nil: default: panic(fmt.Sprintf("proto: unexpected type %T in oneof", x)) @@ -2448,7 +2522,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_52db6b6dc82e2737, []int{30} + return fileDescriptor_errors_c3dba2b22d5a2b16, []int{31} } func (m *ErrPosition) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2510,7 +2584,7 @@ type Error struct { func (m *Error) Reset() { *m = Error{} } func (*Error) ProtoMessage() {} func (*Error) Descriptor() ([]byte, []int) { - return fileDescriptor_errors_52db6b6dc82e2737, []int{31} + return fileDescriptor_errors_c3dba2b22d5a2b16, []int{32} } func (m *Error) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2565,6 +2639,7 @@ func init() { proto.RegisterType((*MergeInProgressError)(nil), "cockroach.kv.kvpb.MergeInProgressError") proto.RegisterType((*RangeFeedRetryError)(nil), "cockroach.kv.kvpb.RangeFeedRetryError") proto.RegisterType((*IndeterminateCommitError)(nil), "cockroach.kv.kvpb.IndeterminateCommitError") + proto.RegisterType((*InvalidLeaseError)(nil), "cockroach.kv.kvpb.InvalidLeaseError") proto.RegisterType((*ErrorDetail)(nil), "cockroach.kv.kvpb.ErrorDetail") proto.RegisterType((*ErrPosition)(nil), "cockroach.kv.kvpb.ErrPosition") proto.RegisterType((*Error)(nil), "cockroach.kv.kvpb.Error") @@ -2927,6 +3002,9 @@ func (m *WriteIntentError) MarshalTo(dAtA []byte) (int, error) { i += n } } + dAtA[i] = 0x18 + i++ + i = encodeVarintErrors(dAtA, i, uint64(m.LeaseSequence)) return i, nil } @@ -3419,6 +3497,24 @@ func (m *IndeterminateCommitError) MarshalTo(dAtA []byte) (int, error) { return i, nil } +func (m *InvalidLeaseError) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalTo(dAtA) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *InvalidLeaseError) MarshalTo(dAtA []byte) (int, error) { + var i int + _ = i + var l int + _ = l + return i, nil +} + func (m *ErrorDetail) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) @@ -3864,6 +3960,22 @@ func (m *ErrorDetail_IndeterminateCommit) MarshalTo(dAtA []byte) (int, error) { } return i, nil } +func (m *ErrorDetail_InvalidLeaseError) MarshalTo(dAtA []byte) (int, error) { + i := 0 + if m.InvalidLeaseError != nil { + dAtA[i] = 0xc2 + i++ + dAtA[i] = 0x2 + i++ + i = encodeVarintErrors(dAtA, i, uint64(m.InvalidLeaseError.Size())) + n52, err := m.InvalidLeaseError.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n52 + } + return i, nil +} func (m *ErrPosition) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) @@ -3911,11 +4023,11 @@ func (m *Error) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x22 i++ i = encodeVarintErrors(dAtA, i, uint64(m.UnexposedTxn.Size())) - n52, err := m.UnexposedTxn.MarshalTo(dAtA[i:]) + n53, err := m.UnexposedTxn.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n52 + i += n53 } dAtA[i] = 0x28 i++ @@ -3923,37 +4035,37 @@ func (m *Error) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x32 i++ i = encodeVarintErrors(dAtA, i, uint64(m.deprecatedDetail.Size())) - n53, err := m.deprecatedDetail.MarshalTo(dAtA[i:]) + n54, err := m.deprecatedDetail.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n53 + i += n54 if m.Index != nil { dAtA[i] = 0x3a i++ i = encodeVarintErrors(dAtA, i, uint64(m.Index.Size())) - n54, err := m.Index.MarshalTo(dAtA[i:]) + n55, err := m.Index.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n54 + i += n55 } dAtA[i] = 0x42 i++ i = encodeVarintErrors(dAtA, i, uint64(m.Now.Size())) - n55, err := m.Now.MarshalTo(dAtA[i:]) + n56, err := m.Now.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n55 + i += n56 dAtA[i] = 0x4a i++ i = encodeVarintErrors(dAtA, i, uint64(m.EncodedError.Size())) - n56, err := m.EncodedError.MarshalTo(dAtA[i:]) + n57, err := m.EncodedError.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n56 + i += n57 return i, nil } @@ -4126,6 +4238,7 @@ func (m *WriteIntentError) Size() (n int) { n += 1 + l + sovErrors(uint64(l)) } } + n += 1 + sovErrors(uint64(m.LeaseSequence)) return n } @@ -4347,6 +4460,15 @@ func (m *IndeterminateCommitError) Size() (n int) { return n } +func (m *InvalidLeaseError) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + return n +} + func (m *ErrorDetail) Size() (n int) { if m == nil { return 0 @@ -4695,6 +4817,18 @@ func (m *ErrorDetail_IndeterminateCommit) Size() (n int) { } return n } +func (m *ErrorDetail_InvalidLeaseError) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.InvalidLeaseError != nil { + l = m.InvalidLeaseError.Size() + n += 2 + l + sovErrors(uint64(l)) + } + return n +} func (m *ErrPosition) Size() (n int) { if m == nil { return 0 @@ -5912,6 +6046,25 @@ func (m *WriteIntentError) Unmarshal(dAtA []byte) error { return err } iNdEx = postIndex + case 3: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field LeaseSequence", wireType) + } + m.LeaseSequence = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowErrors + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.LeaseSequence |= (LeaseSequence(b) & 0x7F) << shift + if b < 0x80 { + break + } + } default: iNdEx = preIndex skippy, err := skipErrors(dAtA[iNdEx:]) @@ -7554,6 +7707,56 @@ func (m *IndeterminateCommitError) Unmarshal(dAtA []byte) error { } return nil } +func (m *InvalidLeaseError) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowErrors + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: InvalidLeaseError: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: InvalidLeaseError: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + default: + iNdEx = preIndex + skippy, err := skipErrors(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return ErrInvalidLengthErrors + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} func (m *ErrorDetail) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 @@ -8479,6 +8682,38 @@ func (m *ErrorDetail) Unmarshal(dAtA []byte) error { } m.Value = &ErrorDetail_IndeterminateCommit{v} iNdEx = postIndex + case 40: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field InvalidLeaseError", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowErrors + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthErrors + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + v := &InvalidLeaseError{} + if err := v.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.Value = &ErrorDetail_InvalidLeaseError{v} + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipErrors(dAtA[iNdEx:]) @@ -8947,195 +9182,199 @@ var ( ErrIntOverflowErrors = fmt.Errorf("proto: integer overflow") ) -func init() { proto.RegisterFile("roachpb/errors.proto", fileDescriptor_errors_52db6b6dc82e2737) } - -var fileDescriptor_errors_52db6b6dc82e2737 = []byte{ - // 2987 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x9c, 0x59, 0xcf, 0x6f, 0x1b, 0xc7, - 0xf5, 0x27, 0x29, 0x4a, 0xa2, 0x9e, 0x7e, 0x78, 0x3d, 0x96, 0xe5, 0x95, 0x1c, 0x53, 0xca, 0xda, - 0x4e, 0x6c, 0x07, 0xa1, 0xbe, 0x70, 0xbe, 0x01, 0x9a, 0xd4, 0x39, 0xf0, 0xc7, 0x4a, 0xa4, 0xc4, - 0x5f, 0x5d, 0x52, 0xb1, 0x1d, 0xa3, 0x98, 0xac, 0xb8, 0x23, 0x6a, 0x6b, 0x72, 0x97, 0x99, 0xdd, - 0x95, 0x25, 0xa0, 0x87, 0xa2, 0xbd, 0x04, 0x2d, 0x50, 0xf4, 0xd6, 0x1e, 0x0b, 0x04, 0x3d, 0x14, - 0x28, 0x8a, 0xfe, 0x05, 0x3d, 0xfb, 0x98, 0x63, 0x50, 0x14, 0x46, 0xeb, 0xf4, 0xd6, 0xff, 0x20, - 0xa7, 0x62, 0x7e, 0x2c, 0xb9, 0x14, 0x97, 0xb2, 0x92, 0x1b, 0xf7, 0xcd, 0x7b, 0x6f, 0xde, 0xbc, - 0x99, 0xf9, 0xbc, 0xcf, 0x1b, 0xc2, 0x2a, 0x75, 0xcd, 0xce, 0xf1, 0xe0, 0x70, 0x9b, 0x50, 0xea, - 0x52, 0x2f, 0x37, 0xa0, 0xae, 0xef, 0xa2, 0xab, 0x1d, 0xb7, 0xf3, 0x9c, 0x8f, 0xe4, 0x9e, 0x9f, - 0xe4, 0x9e, 0x9f, 0x0c, 0x0e, 0x37, 0xae, 0x0b, 0x85, 0x73, 0x9a, 0x1b, 0x28, 0xb4, 0xb7, 0x4c, - 0xdf, 0x94, 0xb2, 0xb5, 0x50, 0xd6, 0x27, 0xbe, 0x19, 0x91, 0xab, 0x81, 0x6f, 0xf7, 0xb6, 0x8f, - 0x7b, 0x9d, 0x6d, 0xdf, 0xee, 0x13, 0xcf, 0x37, 0xfb, 0x03, 0x39, 0xb2, 0xda, 0x75, 0xbb, 0x2e, - 0xff, 0xb9, 0xcd, 0x7e, 0x09, 0xa9, 0xf6, 0xb7, 0x14, 0x5c, 0xab, 0xbb, 0x7e, 0x95, 0x98, 0x1e, - 0x29, 0xbb, 0x3d, 0x8b, 0x50, 0x9d, 0x4d, 0x8d, 0x4a, 0x30, 0x4f, 0xc9, 0xa0, 0x67, 0x77, 0x4c, - 0x35, 0xb9, 0x95, 0xbc, 0xb7, 0xf8, 0xf0, 0x4e, 0x6e, 0x14, 0xaf, 0x9c, 0x3b, 0x67, 0x08, 0x8d, - 0x12, 0xf1, 0x3a, 0xd4, 0x1e, 0xf8, 0x2e, 0x2d, 0xa4, 0x5f, 0xbe, 0xda, 0x4c, 0x18, 0xa1, 0x29, - 0xda, 0x85, 0xa5, 0x1e, 0xf3, 0x8c, 0x8f, 0xb9, 0x6b, 0x35, 0x75, 0x79, 0x57, 0xc6, 0x62, 0x6f, - 0x14, 0x13, 0xfa, 0x10, 0x32, 0xd4, 0x74, 0xba, 0x04, 0xdb, 0x96, 0x3a, 0xb3, 0x95, 0xbc, 0x37, - 0x53, 0xd8, 0x60, 0x33, 0xbd, 0x7e, 0xb5, 0x39, 0x6f, 0x30, 0x79, 0xa5, 0xf4, 0xdd, 0xe8, 0xa7, - 0x31, 0xcf, 0x75, 0x2b, 0x16, 0xca, 0xc1, 0x2c, 0xf7, 0xa2, 0xa6, 0xf9, 0xc4, 0x6a, 0xcc, 0xc4, - 0x7c, 0xe5, 0x86, 0x50, 0x43, 0xb7, 0x01, 0x3a, 0x81, 0xe7, 0xbb, 0x7d, 0xdc, 0xf7, 0xba, 0xea, - 0xec, 0x56, 0xf2, 0xde, 0x82, 0x5c, 0xd2, 0x82, 0x90, 0xd7, 0xbc, 0xae, 0xb6, 0x06, 0xab, 0x75, - 0xd7, 0x22, 0x07, 0x8e, 0x79, 0x62, 0xda, 0x3d, 0xf3, 0xb0, 0x47, 0x78, 0xca, 0xb4, 0x75, 0xb8, - 0x71, 0xe0, 0x78, 0xc1, 0x60, 0xe0, 0x52, 0x9f, 0x58, 0x06, 0xf9, 0x22, 0x20, 0x9e, 0x2f, 0x86, - 0x7e, 0x99, 0x04, 0xc4, 0x83, 0xab, 0xbb, 0xfe, 0x8e, 0x1b, 0x38, 0x96, 0x48, 0x72, 0x74, 0x55, - 0xc9, 0xcb, 0xaf, 0xea, 0x43, 0xc8, 0x78, 0xbe, 0x4b, 0xb9, 0x59, 0x6a, 0xdc, 0xac, 0xc5, 0xe4, - 0xc2, 0x4c, 0xfe, 0x34, 0xe6, 0xb9, 0x6e, 0xc5, 0xd2, 0x7e, 0x3f, 0x03, 0xd7, 0xb9, 0xaf, 0x7d, - 0x72, 0x56, 0xb3, 0xbd, 0xbe, 0xe9, 0x77, 0x8e, 0x45, 0x1c, 0x1f, 0xc0, 0x55, 0x2a, 0xc2, 0xc5, - 0x9e, 0x6f, 0x52, 0x1f, 0x3f, 0x27, 0x67, 0x3c, 0xa0, 0xa5, 0xc2, 0xfc, 0x77, 0xaf, 0x36, 0x67, - 0xf6, 0xc9, 0x99, 0x71, 0x45, 0x6a, 0xb4, 0x98, 0xc2, 0x3e, 0x39, 0x43, 0xdb, 0x10, 0x8a, 0x30, - 0x71, 0x2c, 0x6e, 0x92, 0x1a, 0x37, 0x59, 0x96, 0xe3, 0xba, 0x63, 0x31, 0x83, 0x63, 0xb8, 0x69, - 0x91, 0x01, 0x25, 0x1d, 0xd3, 0x27, 0x16, 0xee, 0xcb, 0x08, 0x88, 0x85, 0xf9, 0xba, 0xf8, 0xb6, - 0x2e, 0x3e, 0xd4, 0xe2, 0xce, 0x06, 0x1b, 0x9f, 0x38, 0x64, 0xeb, 0x23, 0x67, 0xb5, 0xa1, 0x2f, - 0xae, 0x8a, 0x3e, 0x87, 0x8d, 0xc8, 0x4c, 0x5e, 0xd0, 0xed, 0x12, 0xcf, 0x1f, 0x4e, 0x94, 0xbe, - 0xec, 0x44, 0x86, 0x3a, 0xf2, 0xd2, 0x0a, 0x9d, 0x88, 0x19, 0xaa, 0x30, 0xc7, 0x9d, 0x79, 0xea, - 0xec, 0xd6, 0xcc, 0xbd, 0xc5, 0x87, 0x6f, 0x4d, 0xf3, 0x56, 0x71, 0x8e, 0xdc, 0xc2, 0x9a, 0xdc, - 0x9e, 0x15, 0x61, 0x53, 0x71, 0x7c, 0x42, 0x1d, 0xb3, 0x67, 0x48, 0x1f, 0xda, 0x7f, 0x53, 0xa0, - 0x19, 0xc4, 0xb4, 0x1e, 0xdb, 0xfe, 0xb1, 0xed, 0x1c, 0x38, 0x1d, 0x42, 0x7d, 0xd3, 0x76, 0xfc, - 0x33, 0xae, 0x79, 0x62, 0xf6, 0xc4, 0x36, 0xed, 0xc1, 0x0a, 0x25, 0xa6, 0x85, 0x87, 0x37, 0x5b, - 0x5e, 0xcd, 0x5b, 0x91, 0xc9, 0xd9, 0xf5, 0xcf, 0x1d, 0xf7, 0x3a, 0xb9, 0x76, 0xa8, 0x24, 0xd3, - 0xb5, 0xcc, 0x4c, 0x87, 0x42, 0x64, 0x00, 0x22, 0xa7, 0xb6, 0xe7, 0xdb, 0x4e, 0x37, 0xe2, 0x2f, - 0x75, 0x79, 0x7f, 0x57, 0x43, 0xf3, 0x91, 0xcf, 0x02, 0x2c, 0xf7, 0xcd, 0xd3, 0x88, 0xbb, 0x99, - 0x4b, 0xb8, 0x33, 0x96, 0xfa, 0xe6, 0xe9, 0xc8, 0xc7, 0x33, 0xb8, 0xe6, 0x1e, 0x7a, 0x84, 0x9e, - 0x90, 0xc8, 0x3a, 0x3d, 0x35, 0xcd, 0xb3, 0x1c, 0x07, 0x1c, 0x0d, 0xa9, 0x7d, 0x3e, 0x3e, 0xe4, - 0x9e, 0x1f, 0xf0, 0x3e, 0x4e, 0xff, 0xe1, 0x8f, 0x9b, 0x09, 0xcd, 0x82, 0x1b, 0x6d, 0x6a, 0x3a, - 0x9e, 0xd9, 0xf1, 0x6d, 0xd7, 0xc9, 0x1f, 0xf2, 0xeb, 0x2a, 0x32, 0x5c, 0x81, 0x39, 0x4a, 0x4c, - 0xcf, 0x75, 0x78, 0x66, 0x57, 0x1e, 0xbe, 0x97, 0x9b, 0x00, 0xe9, 0xdc, 0xa4, 0xad, 0xc1, 0x4d, - 0xe4, 0xbc, 0xd2, 0x81, 0xf6, 0x0c, 0x56, 0x23, 0x9a, 0xcd, 0xc0, 0x93, 0x77, 0xad, 0x08, 0x30, - 0x08, 0xbc, 0x63, 0x42, 0xb0, 0x7f, 0xea, 0xc8, 0x0d, 0xcc, 0xc6, 0xac, 0x2b, 0x62, 0x1c, 0x42, - 0x90, 0xb0, 0x6b, 0x9f, 0x3a, 0xda, 0xaf, 0x92, 0x70, 0x3d, 0xa2, 0x60, 0x10, 0x9f, 0x9e, 0x09, - 0xf7, 0xbb, 0xe7, 0x56, 0x70, 0xff, 0xe2, 0x15, 0x70, 0xcb, 0xb8, 0xf8, 0xd1, 0xdb, 0xb0, 0x40, - 0x4e, 0x7d, 0x6a, 0x72, 0x24, 0x4c, 0x45, 0x90, 0x30, 0xc3, 0xc5, 0x0c, 0x08, 0xff, 0x9e, 0x84, - 0xb5, 0x88, 0xaf, 0x96, 0x6f, 0xfa, 0x81, 0x27, 0xc2, 0x58, 0x83, 0x19, 0x66, 0x97, 0x8c, 0xd8, - 0x31, 0x01, 0xaa, 0x0f, 0xc3, 0x4b, 0xf1, 0xf0, 0xfe, 0xef, 0xe2, 0xf0, 0x22, 0x2e, 0x73, 0xb1, - 0x59, 0x7e, 0x04, 0x73, 0x42, 0x8e, 0x10, 0xac, 0x18, 0x7a, 0xbe, 0xd5, 0xa8, 0xe3, 0x83, 0xfa, - 0x7e, 0xbd, 0xf1, 0xb8, 0xae, 0x24, 0x90, 0x0a, 0xab, 0x52, 0xd6, 0x7e, 0x52, 0xc7, 0xc5, 0x46, - 0xad, 0x56, 0x69, 0xb7, 0xf5, 0x92, 0x92, 0xd2, 0xd2, 0x99, 0xa4, 0x92, 0xd4, 0x5a, 0xa0, 0x3c, - 0xa6, 0xb6, 0x4f, 0xd8, 0x35, 0x73, 0x04, 0x54, 0xa3, 0x8f, 0x60, 0xde, 0xe6, 0x9f, 0x9e, 0x9a, - 0xe4, 0x87, 0x6e, 0x3d, 0x66, 0x73, 0x84, 0x41, 0x58, 0xed, 0xa4, 0xfe, 0x5e, 0x3a, 0x93, 0x52, - 0x66, 0xb4, 0x3f, 0x25, 0xa5, 0xd7, 0xb6, 0xeb, 0x36, 0x7a, 0xf2, 0x60, 0xe5, 0x61, 0xe1, 0x07, - 0xdd, 0xda, 0x91, 0x15, 0xaa, 0x83, 0x62, 0x76, 0xfc, 0xc0, 0xec, 0xfd, 0xb0, 0xfb, 0x7a, 0x45, - 0x18, 0x0f, 0xc5, 0xda, 0x2a, 0xa0, 0xc6, 0x80, 0x55, 0x29, 0x9b, 0x12, 0xaf, 0x7d, 0xea, 0x88, - 0x4a, 0xd5, 0x82, 0xd5, 0xa2, 0xeb, 0x58, 0x36, 0xcb, 0xfe, 0x8e, 0x69, 0xf7, 0xc2, 0x9b, 0xf1, - 0x63, 0x58, 0x92, 0xb3, 0x9f, 0x98, 0xbd, 0x80, 0xc8, 0x35, 0xc4, 0x15, 0xd4, 0x4f, 0xd9, 0xb8, - 0xb1, 0x28, 0xb4, 0xf9, 0x87, 0xf6, 0xd7, 0x24, 0x20, 0x51, 0x67, 0xc9, 0xcf, 0x48, 0x67, 0x78, - 0xdb, 0xb2, 0x30, 0xdf, 0x27, 0x9e, 0x67, 0x76, 0xc9, 0xd8, 0x41, 0x09, 0x85, 0xe8, 0x11, 0x2c, - 0xc8, 0x0a, 0x42, 0x2c, 0xb9, 0xd4, 0xa9, 0x15, 0x3c, 0xcc, 0xd7, 0xd0, 0x00, 0x7d, 0x0c, 0x99, - 0x10, 0xa2, 0x24, 0x10, 0xbd, 0xc9, 0x78, 0xa8, 0xaf, 0x7d, 0x01, 0xab, 0xf9, 0xfe, 0xa1, 0xdd, - 0x0d, 0xdc, 0xc0, 0x33, 0x88, 0x17, 0xf4, 0xfc, 0xcb, 0x45, 0xfc, 0x11, 0x2c, 0xbe, 0xa0, 0xe6, - 0x60, 0x40, 0x2c, 0x4c, 0x28, 0x8d, 0x89, 0x39, 0x3c, 0xe3, 0xdc, 0x9d, 0x01, 0x52, 0x59, 0xa7, - 0x54, 0xbb, 0xc1, 0x8a, 0xf3, 0x91, 0xbf, 0x4b, 0xdd, 0x60, 0x50, 0x22, 0x3d, 0x12, 0x66, 0x49, - 0xc3, 0xb0, 0x26, 0xc9, 0x51, 0xd1, 0xa5, 0x34, 0x18, 0xb0, 0x9d, 0x11, 0xd1, 0xb0, 0x2b, 0xca, - 0x7e, 0xe0, 0xf3, 0x57, 0x2d, 0xc3, 0xc5, 0x35, 0xaf, 0x8b, 0x34, 0x58, 0x18, 0x50, 0xb7, 0x43, - 0x3c, 0x4f, 0xa6, 0x30, 0x33, 0x04, 0x93, 0x50, 0xac, 0xb5, 0x00, 0xc9, 0x09, 0xa2, 0x27, 0xf6, - 0x13, 0x00, 0xc9, 0xe2, 0x42, 0x76, 0x32, 0x5b, 0xc8, 0xca, 0x3a, 0xb6, 0x20, 0xf5, 0x39, 0xd1, - 0x18, 0x7d, 0xb0, 0xec, 0x8b, 0x9f, 0x96, 0xb6, 0x0f, 0x88, 0x13, 0x90, 0x09, 0xc2, 0x33, 0x64, - 0x2e, 0xc9, 0xcb, 0x33, 0x97, 0x16, 0x63, 0x56, 0xc7, 0xa6, 0x63, 0xf5, 0x18, 0xd8, 0xfa, 0xf4, - 0x6c, 0x48, 0xba, 0xd0, 0x43, 0x48, 0x0f, 0x74, 0x4a, 0x63, 0xce, 0xe3, 0x58, 0xaa, 0xe5, 0xaa, - 0xb9, 0xae, 0x2c, 0x03, 0xff, 0x49, 0xc2, 0xdd, 0xf3, 0x48, 0xc8, 0x0a, 0x70, 0x93, 0xf1, 0x62, - 0x83, 0x1c, 0x51, 0x12, 0x42, 0xf6, 0x34, 0x30, 0x7b, 0x06, 0x73, 0xfe, 0xa9, 0x13, 0xb2, 0xb0, - 0xa5, 0x42, 0x89, 0x0d, 0xfd, 0xe3, 0xd5, 0xe6, 0x07, 0x5d, 0xdb, 0x3f, 0x0e, 0x0e, 0x73, 0x1d, - 0xb7, 0xbf, 0x3d, 0x8c, 0xc7, 0x3a, 0x1c, 0xfd, 0xde, 0x1e, 0x3c, 0xef, 0x6e, 0x73, 0xa2, 0x1e, - 0x04, 0xb6, 0x95, 0x3b, 0x38, 0xa8, 0x94, 0x5e, 0xbf, 0xda, 0x9c, 0x6d, 0x9f, 0x3a, 0x95, 0x92, - 0x31, 0xeb, 0x9f, 0x3a, 0x15, 0x0b, 0xed, 0xc0, 0xa2, 0x3f, 0x8a, 0x4e, 0x9e, 0xe0, 0xcb, 0x15, - 0x8a, 0xa8, 0xa1, 0xb6, 0x03, 0x9b, 0xed, 0x53, 0x27, 0xdf, 0x63, 0xe5, 0xff, 0x4c, 0x77, 0x3a, - 0x6e, 0xc0, 0x38, 0x85, 0x3c, 0x5c, 0x62, 0x7d, 0xb7, 0x01, 0x06, 0x94, 0x9c, 0x60, 0x7e, 0x6a, - 0xc6, 0x96, 0xb9, 0xc0, 0xe4, 0xe2, 0x18, 0xfe, 0x26, 0x09, 0xab, 0x0c, 0xf6, 0xba, 0x84, 0x36, - 0x4e, 0x08, 0x3d, 0xea, 0xb9, 0x2f, 0x84, 0xf5, 0x3a, 0xcc, 0xc4, 0xd0, 0x45, 0x26, 0x43, 0xf7, - 0x61, 0xb9, 0x13, 0x50, 0x4a, 0x1c, 0x5f, 0xa2, 0x86, 0x60, 0xab, 0xc2, 0xf7, 0x92, 0x1c, 0xe2, - 0x10, 0x81, 0xde, 0x87, 0x2b, 0xb6, 0xd3, 0xa1, 0xa4, 0x3f, 0x52, 0x9e, 0x89, 0x28, 0xaf, 0x0c, - 0x07, 0x05, 0xa2, 0x7c, 0x95, 0x84, 0x9b, 0x05, 0x46, 0xf9, 0x46, 0x30, 0x47, 0x8e, 0x5c, 0x4a, - 0x76, 0x8b, 0x43, 0xbc, 0x6d, 0xff, 0x20, 0xbc, 0x1d, 0x31, 0x11, 0xe6, 0xe2, 0x98, 0x1d, 0x02, - 0xb7, 0x67, 0x7d, 0x1f, 0xa0, 0x1d, 0x59, 0x69, 0x7d, 0x40, 0xa2, 0x52, 0xd4, 0x6c, 0xcf, 0xb3, - 0x9d, 0xae, 0x88, 0xed, 0x11, 0x2c, 0xbd, 0xa0, 0xae, 0xd3, 0xc5, 0xa2, 0x6e, 0xc8, 0xf0, 0xa6, - 0x97, 0x19, 0x63, 0x91, 0xab, 0x8b, 0x8f, 0x30, 0xdd, 0xa9, 0xc9, 0x74, 0xb3, 0xc6, 0xa4, 0x46, - 0x28, 0xe3, 0x9c, 0x4d, 0xea, 0x76, 0x29, 0xf1, 0x44, 0xe5, 0xd4, 0x7e, 0x9b, 0x82, 0x6b, 0x9c, - 0x8c, 0xee, 0x10, 0x79, 0x7f, 0x44, 0x20, 0xfb, 0xe7, 0xb8, 0xc2, 0xfb, 0x31, 0xb7, 0x27, 0xc6, - 0x2e, 0xbe, 0x12, 0xff, 0x39, 0x39, 0x2c, 0xc5, 0x1b, 0xb0, 0x26, 0xcb, 0xae, 0xa1, 0x37, 0xab, - 0x95, 0x62, 0x1e, 0x1b, 0x7a, 0xad, 0xf1, 0xa9, 0x5e, 0x52, 0x12, 0x68, 0x0d, 0x50, 0x38, 0x96, - 0xaf, 0xef, 0xea, 0xb8, 0xd5, 0xac, 0x56, 0xda, 0x4a, 0x12, 0xdd, 0x80, 0x6b, 0x63, 0xf2, 0x9a, - 0x6e, 0xec, 0xb2, 0x4a, 0x1d, 0xa9, 0xe1, 0x46, 0x7e, 0xa7, 0x8d, 0x5b, 0xf5, 0x7c, 0xb3, 0x55, - 0x6e, 0xb4, 0x95, 0x19, 0x94, 0x85, 0x0d, 0x39, 0x52, 0x6d, 0xec, 0x56, 0x8a, 0xf9, 0x2a, 0x6e, - 0x34, 0x5b, 0xb8, 0x56, 0x69, 0xb5, 0x2a, 0xf5, 0x5d, 0x25, 0x1d, 0xb1, 0x6c, 0x55, 0x1b, 0x8f, - 0x71, 0xb1, 0x51, 0x6f, 0x1d, 0xd4, 0x74, 0x43, 0x99, 0xd5, 0x4c, 0x50, 0x2b, 0x8e, 0x45, 0x7c, - 0x42, 0xfb, 0xb6, 0x63, 0xfa, 0xa4, 0xe8, 0xf6, 0xfb, 0xb6, 0x84, 0x78, 0x1d, 0x16, 0x3d, 0xdf, - 0xec, 0x72, 0x5e, 0xfc, 0x3d, 0x09, 0x1a, 0x48, 0x43, 0xc6, 0xd0, 0x5e, 0xae, 0xc2, 0x22, 0x77, - 0x58, 0x22, 0xbe, 0x69, 0xf7, 0x90, 0x01, 0x8a, 0xe3, 0xfa, 0x78, 0xac, 0x1b, 0x16, 0xbe, 0xdf, - 0x89, 0xc9, 0x7a, 0x4c, 0x47, 0x5e, 0x4e, 0x18, 0x2b, 0xce, 0x98, 0x18, 0x35, 0xe0, 0x8a, 0x68, - 0x1f, 0x99, 0xe7, 0x23, 0x86, 0xb2, 0xf2, 0x9c, 0xde, 0x9d, 0xb6, 0x91, 0x63, 0x68, 0x5c, 0x66, - 0x4d, 0x41, 0x54, 0x8a, 0x9e, 0x00, 0x12, 0x0e, 0x9f, 0x93, 0xb3, 0x61, 0x83, 0x26, 0xa1, 0xe7, - 0xde, 0x34, 0x9f, 0xe7, 0xbb, 0xc9, 0x72, 0xc2, 0x50, 0xe8, 0xb9, 0x01, 0xf4, 0x8b, 0x24, 0x6c, - 0xf1, 0xde, 0xe5, 0x05, 0x6f, 0x71, 0x70, 0x30, 0xea, 0x71, 0xf8, 0x35, 0x60, 0x4d, 0x8e, 0x6c, - 0xcc, 0x3e, 0x8c, 0x9b, 0xe8, 0x8d, 0xcd, 0x51, 0x39, 0x61, 0xdc, 0xa2, 0x17, 0x69, 0xa1, 0x9f, - 0xc2, 0xb5, 0x08, 0x2e, 0x62, 0x53, 0x70, 0x77, 0xde, 0xe4, 0x2f, 0x3e, 0x7c, 0x70, 0x29, 0xa2, - 0x1f, 0xce, 0x84, 0xfc, 0x89, 0x21, 0xd4, 0x06, 0x25, 0xea, 0x9e, 0x71, 0x75, 0x75, 0x8e, 0xfb, - 0x7e, 0xf7, 0x62, 0xdf, 0xc3, 0xd6, 0xa0, 0x9c, 0x30, 0xae, 0xf8, 0xe3, 0x72, 0xf4, 0x18, 0xae, - 0x46, 0xbd, 0x52, 0x76, 0x09, 0xd5, 0xf9, 0xa9, 0x1b, 0x12, 0xdb, 0x13, 0xb0, 0x0d, 0xf1, 0xcf, - 0x0d, 0xa0, 0xcf, 0x20, 0xba, 0x08, 0xd6, 0xf6, 0xfb, 0x81, 0xa7, 0x66, 0xb8, 0xe7, 0xfb, 0x97, - 0x26, 0xe5, 0xe5, 0x84, 0x11, 0x8d, 0x4f, 0x8c, 0xa0, 0x32, 0x03, 0x38, 0xdb, 0x27, 0x21, 0xc0, - 0x2d, 0x70, 0xaf, 0xb7, 0x63, 0xbc, 0x9e, 0x67, 0xdf, 0xe5, 0x04, 0x03, 0xbb, 0xa1, 0x0c, 0x55, - 0x60, 0x59, 0x78, 0xf2, 0x5d, 0x17, 0x33, 0x1c, 0x86, 0x8b, 0x5d, 0x45, 0x08, 0xcc, 0xd0, 0x95, - 0x90, 0xb1, 0xcb, 0xe2, 0x0e, 0x30, 0x95, 0x7c, 0x97, 0xdf, 0xed, 0xc5, 0xa9, 0x97, 0x65, 0x92, - 0x18, 0xb3, 0xcb, 0xe2, 0x46, 0xa5, 0x6c, 0xc3, 0x3b, 0x21, 0x53, 0xc6, 0x47, 0x9c, 0x2a, 0xab, - 0x4b, 0x53, 0x37, 0x3c, 0x8e, 0x54, 0xb3, 0x0d, 0xef, 0x8c, 0xcb, 0x51, 0x1d, 0x56, 0x04, 0x46, - 0x50, 0x49, 0x95, 0xd5, 0xe5, 0xa9, 0x51, 0x4e, 0x52, 0x6a, 0x16, 0x65, 0x2f, 0x2a, 0x65, 0x51, - 0x3a, 0xae, 0x45, 0x70, 0x30, 0x7a, 0xad, 0x52, 0x57, 0xa6, 0x46, 0x19, 0xf7, 0xae, 0xc5, 0xa2, - 0x74, 0xc6, 0xe5, 0x02, 0x28, 0x8e, 0x7c, 0xdc, 0x65, 0x6c, 0x15, 0x5b, 0x82, 0xae, 0xaa, 0xca, - 0x05, 0x40, 0x11, 0xc3, 0x6c, 0x05, 0x50, 0x8c, 0x0f, 0xb0, 0x73, 0x19, 0xd2, 0xce, 0xce, 0x90, - 0xee, 0xaa, 0x57, 0xa7, 0x9e, 0xcb, 0x78, 0x6a, 0xcc, 0xce, 0x25, 0x3d, 0x3f, 0xc2, 0xf1, 0x52, - 0xfa, 0x0e, 0xcf, 0x13, 0x9a, 0x8e, 0x97, 0x13, 0x94, 0x98, 0xe3, 0x65, 0x54, 0xca, 0x92, 0x6b, - 0x86, 0x6d, 0x02, 0xa6, 0xbc, 0x4f, 0x50, 0x37, 0xa6, 0x26, 0x37, 0xae, 0xa3, 0x60, 0xc9, 0x35, - 0xc7, 0xe5, 0x2c, 0x4c, 0x41, 0x92, 0x47, 0xb0, 0x7e, 0x73, 0x6a, 0x98, 0x93, 0x24, 0x9b, 0x85, - 0xe9, 0x45, 0xa5, 0xe8, 0xd7, 0x49, 0xb8, 0x33, 0x81, 0x22, 0x1c, 0x89, 0x31, 0x7f, 0x04, 0xc6, - 0x54, 0xb0, 0x5d, 0xf5, 0x2d, 0x3e, 0xcd, 0x8f, 0x2e, 0x01, 0x2c, 0xb1, 0x44, 0xb9, 0x9c, 0x30, - 0xb6, 0xfc, 0x37, 0x28, 0xb2, 0x9c, 0xd9, 0x82, 0x46, 0x62, 0x57, 0xf2, 0x48, 0x75, 0x73, 0x6a, - 0xce, 0xe2, 0x18, 0x27, 0xcb, 0x99, 0x3d, 0x2e, 0x67, 0xe0, 0x1e, 0x8c, 0xde, 0x5e, 0xb1, 0xec, - 0x02, 0xd5, 0xad, 0xa9, 0xe0, 0x3e, 0xe5, 0xa5, 0x96, 0x81, 0x7b, 0x30, 0x31, 0x84, 0x9e, 0x81, - 0x32, 0x6c, 0xba, 0xf1, 0x21, 0x67, 0x9a, 0xaa, 0xc6, 0x7d, 0xe7, 0x62, 0x7c, 0x5f, 0x40, 0x4c, - 0x39, 0xc6, 0x8f, 0x8f, 0xa0, 0x17, 0x70, 0x8b, 0xb5, 0x11, 0xa6, 0xa0, 0xe8, 0x98, 0x8c, 0x38, - 0xba, 0x64, 0xe4, 0xb7, 0xf9, 0x4c, 0x0f, 0xe3, 0xb6, 0xe5, 0x62, 0x66, 0x5f, 0x4e, 0x18, 0x1b, - 0xfe, 0x54, 0x15, 0x86, 0x35, 0x02, 0xa1, 0x59, 0xad, 0x67, 0xfc, 0x54, 0xbd, 0x33, 0xf5, 0x9c, - 0x4d, 0xf2, 0x58, 0x76, 0xce, 0xec, 0xa8, 0x14, 0x1d, 0xc0, 0xd5, 0x3e, 0xe3, 0x9f, 0xd8, 0x76, - 0xd8, 0xc1, 0xe2, 0x0c, 0x54, 0xbd, 0x3b, 0x75, 0x6f, 0xe3, 0xb8, 0x2a, 0xcb, 0x4f, 0x7f, 0x5c, - 0x8e, 0x7e, 0x22, 0x69, 0xce, 0x11, 0xe1, 0x3b, 0xcb, 0x2a, 0xe0, 0x3b, 0x53, 0x99, 0x53, 0x0c, - 0x5f, 0x65, 0xcc, 0x69, 0xe8, 0x40, 0x54, 0xbf, 0xcf, 0x61, 0xd5, 0x8e, 0x12, 0x40, 0xdc, 0xe1, - 0x0c, 0x50, 0x7d, 0x97, 0xfb, 0x7d, 0x2f, 0x76, 0xfd, 0xf1, 0x7c, 0xb1, 0x9c, 0x30, 0xae, 0xd9, - 0x93, 0x63, 0x85, 0x79, 0x98, 0xe5, 0x5d, 0xcc, 0x5e, 0x3a, 0x73, 0x45, 0x51, 0xf6, 0xd2, 0x99, - 0x6b, 0xca, 0xea, 0x5e, 0x3a, 0xb3, 0xaa, 0x5c, 0xdf, 0x4b, 0x67, 0xae, 0x2b, 0x6b, 0x7b, 0xe9, - 0xcc, 0x9a, 0x72, 0x63, 0x2f, 0x9d, 0xb9, 0xa1, 0xa8, 0x7b, 0xe9, 0x8c, 0xaa, 0xac, 0xef, 0xa5, - 0x33, 0xeb, 0xca, 0xc6, 0x5e, 0x3a, 0x73, 0x4b, 0xc9, 0xee, 0xa5, 0x33, 0x59, 0x65, 0x73, 0x2f, - 0x9d, 0x79, 0x5b, 0xd1, 0xb4, 0xfb, 0x9c, 0x49, 0x36, 0x5d, 0x8f, 0xd7, 0x09, 0xb4, 0x01, 0xb3, - 0x6c, 0xc2, 0x53, 0xd9, 0x93, 0x0b, 0xea, 0x29, 0x44, 0xda, 0x97, 0xb3, 0x30, 0x1b, 0x3e, 0xe9, - 0x9f, 0x7b, 0xa9, 0x58, 0x97, 0x8d, 0xf6, 0xd5, 0xc8, 0xb3, 0xb9, 0x50, 0x18, 0x3d, 0x5f, 0xfc, - 0x7c, 0x9c, 0x22, 0x51, 0xc2, 0xff, 0x0d, 0xe0, 0x04, 0x70, 0x25, 0xf6, 0x54, 0x8c, 0xc1, 0x02, - 0x57, 0x2e, 0xdc, 0x91, 0xf3, 0xbc, 0x35, 0x9a, 0x67, 0x52, 0x6b, 0x8c, 0x41, 0x49, 0x19, 0x2a, - 0xc2, 0x72, 0xe0, 0x90, 0xd3, 0x81, 0xeb, 0x11, 0x8b, 0xd7, 0xe7, 0xf4, 0x65, 0xb8, 0xb7, 0xb1, - 0x34, 0x34, 0x62, 0x55, 0x79, 0x1b, 0x16, 0x5d, 0x6a, 0x77, 0x6d, 0x07, 0xb3, 0x9a, 0xc5, 0xd9, - 0xdd, 0x6c, 0x61, 0x85, 0xc5, 0xf4, 0xdd, 0xab, 0xcd, 0x39, 0x56, 0xdf, 0x2a, 0x25, 0x03, 0x84, - 0x0a, 0xfb, 0x42, 0x4d, 0x98, 0xb3, 0x38, 0x45, 0x97, 0x6c, 0x2d, 0x3b, 0xed, 0x09, 0x41, 0x10, - 0xf9, 0x82, 0x2a, 0xd7, 0xa7, 0x8c, 0xd6, 0x27, 0x46, 0x0c, 0xe9, 0x07, 0xfd, 0x7f, 0xb8, 0x41, - 0xf3, 0x17, 0x39, 0x0c, 0xf7, 0x53, 0x6e, 0x1d, 0x0a, 0x60, 0xc6, 0x71, 0x5f, 0x48, 0x06, 0xf6, - 0x86, 0x46, 0xb3, 0x24, 0xd7, 0xf3, 0xe8, 0xf2, 0x0f, 0x0d, 0xcc, 0x41, 0xb1, 0xe7, 0x76, 0x9e, - 0x8f, 0x1e, 0xde, 0xd9, 0x7c, 0x68, 0x1f, 0x96, 0x19, 0xe0, 0x58, 0x43, 0xb0, 0x11, 0x64, 0x6d, - 0x2b, 0x12, 0x40, 0xf8, 0xa7, 0x64, 0x4e, 0x17, 0x8a, 0xd1, 0x07, 0x95, 0x25, 0x12, 0x91, 0x89, - 0x87, 0x15, 0xf1, 0x0c, 0xfa, 0xe0, 0x9f, 0x29, 0x50, 0xa7, 0x3d, 0x95, 0xb3, 0xd6, 0x2c, 0x5f, - 0x68, 0x18, 0x6d, 0x3c, 0xf1, 0x64, 0x7b, 0x17, 0xde, 0x1e, 0x1b, 0xe1, 0x1f, 0x7a, 0x09, 0x1b, - 0x7a, 0xb1, 0x61, 0x94, 0xf0, 0x4e, 0xe3, 0xa0, 0x5e, 0x52, 0x92, 0xac, 0xf7, 0x1b, 0x53, 0x2b, - 0x56, 0x2b, 0x7a, 0x9d, 0x7d, 0xed, 0xe9, 0x45, 0xd6, 0x1b, 0x6e, 0xc2, 0xcd, 0xb1, 0xf1, 0xe6, - 0x41, 0xab, 0xac, 0x1b, 0xa1, 0x37, 0x25, 0x8d, 0x6e, 0xc2, 0x8d, 0xc9, 0x79, 0x70, 0xab, 0x99, - 0xaf, 0x2b, 0xb3, 0x28, 0x0f, 0x9f, 0x8c, 0x0f, 0x56, 0x0d, 0x3d, 0x5f, 0x7a, 0x3a, 0x7a, 0x41, - 0xc6, 0x0d, 0x03, 0x1b, 0x8d, 0x6a, 0x55, 0x2f, 0xe1, 0x42, 0xbe, 0xb8, 0x8f, 0x9b, 0x8d, 0x56, - 0xab, 0x52, 0xa8, 0xea, 0xbc, 0xe1, 0xcd, 0x3f, 0x55, 0xe6, 0xd0, 0xbb, 0x70, 0x7b, 0xcc, 0x45, - 0x5d, 0x7f, 0x8c, 0xab, 0x7a, 0xbe, 0xa5, 0xe3, 0xa6, 0xa1, 0x7f, 0xaa, 0xd7, 0xdb, 0x2d, 0xdc, - 0x7e, 0x52, 0x57, 0x32, 0xe8, 0x3e, 0xdc, 0x1d, 0x53, 0x6c, 0x57, 0x6a, 0x7a, 0xab, 0x9d, 0xaf, - 0x35, 0x71, 0x31, 0x5f, 0x2c, 0xeb, 0x72, 0x49, 0x7a, 0x49, 0x99, 0xdf, 0x48, 0x7f, 0xf9, 0x55, - 0x36, 0xa1, 0xb1, 0xf4, 0xa6, 0x1e, 0xfc, 0x65, 0xfc, 0xed, 0x3d, 0xf2, 0x8e, 0x2f, 0xfa, 0xde, - 0xb6, 0xf1, 0x74, 0x32, 0xb9, 0xbc, 0xc9, 0x66, 0x23, 0x8f, 0x8d, 0x4a, 0x5b, 0xc7, 0xed, 0x46, - 0x03, 0x37, 0xaa, 0x2c, 0x9d, 0xbc, 0x2b, 0x67, 0x03, 0x2d, 0xdd, 0xa8, 0xe4, 0xab, 0x95, 0xcf, - 0xf2, 0x85, 0xaa, 0xae, 0xcc, 0xa0, 0x5b, 0xb0, 0x2e, 0xe4, 0xf9, 0xd6, 0xd3, 0x7a, 0x51, 0x9a, - 0xed, 0xe4, 0x2b, 0xd5, 0x03, 0x43, 0x57, 0x66, 0x91, 0x06, 0x59, 0x31, 0x2c, 0x12, 0x83, 0x4b, - 0x7a, 0xbe, 0x54, 0xad, 0xd4, 0x75, 0xac, 0x3f, 0x29, 0xea, 0x7a, 0x49, 0x2f, 0x29, 0x73, 0x22, - 0xe8, 0x07, 0x1f, 0x03, 0x9a, 0x44, 0x01, 0x94, 0x81, 0x74, 0xbd, 0x51, 0xd7, 0x95, 0x04, 0x5a, - 0x84, 0x79, 0x96, 0xc8, 0xc6, 0xce, 0x8e, 0x92, 0x44, 0xcb, 0xb0, 0x50, 0xa9, 0xd5, 0xf4, 0x52, - 0x25, 0xdf, 0xd6, 0x95, 0x54, 0xe1, 0xfe, 0xcb, 0x7f, 0x67, 0x13, 0x2f, 0x5f, 0x67, 0x93, 0x5f, - 0xbf, 0xce, 0x26, 0xbf, 0x79, 0x9d, 0x4d, 0xfe, 0xeb, 0x75, 0x36, 0xf9, 0xbb, 0x6f, 0xb3, 0x89, - 0xaf, 0xbf, 0xcd, 0x26, 0xbe, 0xf9, 0x36, 0x9b, 0xf8, 0x6c, 0x5e, 0xe2, 0xc2, 0xff, 0x02, 0x00, - 0x00, 0xff, 0xff, 0x14, 0x07, 0xfe, 0xb0, 0x6a, 0x1f, 0x00, 0x00, +func init() { proto.RegisterFile("roachpb/errors.proto", fileDescriptor_errors_c3dba2b22d5a2b16) } + +var fileDescriptor_errors_c3dba2b22d5a2b16 = []byte{ + // 3051 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x9c, 0x59, 0xcb, 0x6f, 0x1b, 0xd7, + 0xb9, 0x27, 0x25, 0x4a, 0xa2, 0x3e, 0x3d, 0x3c, 0x3a, 0x92, 0xe5, 0x91, 0x1c, 0x53, 0xca, 0xd8, + 0x4e, 0x6c, 0x07, 0xa1, 0x2e, 0x9c, 0x1b, 0xe0, 0x26, 0xd7, 0x59, 0xf0, 0x31, 0x12, 0x47, 0xe2, + 0x43, 0x77, 0x48, 0xf9, 0x11, 0xe3, 0xe2, 0x64, 0xc4, 0x39, 0xa2, 0xa6, 0x26, 0x67, 0x98, 0x33, + 0x43, 0x59, 0x02, 0xba, 0x28, 0xda, 0x4d, 0xd0, 0x02, 0x45, 0xbb, 0x6a, 0x97, 0x01, 0x82, 0x2e, + 0x0a, 0x14, 0x45, 0xff, 0x82, 0xae, 0xb3, 0xcc, 0x32, 0x28, 0x0a, 0xa3, 0x75, 0xba, 0xeb, 0x7f, + 0x90, 0x55, 0x71, 0x1e, 0x43, 0x0e, 0xc9, 0xa1, 0xac, 0x64, 0xc7, 0xf9, 0x5e, 0xe7, 0x3b, 0xaf, + 0xdf, 0xf7, 0xfb, 0x0e, 0x61, 0x8d, 0x7a, 0x56, 0xf3, 0xb4, 0x7b, 0xbc, 0x43, 0x28, 0xf5, 0xa8, + 0x9f, 0xed, 0x52, 0x2f, 0xf0, 0xd0, 0x4a, 0xd3, 0x6b, 0xbe, 0xe0, 0x9a, 0xec, 0x8b, 0xb3, 0xec, + 0x8b, 0xb3, 0xee, 0xf1, 0xe6, 0x75, 0x61, 0x30, 0x62, 0xb9, 0x89, 0x42, 0x7f, 0xdb, 0x0a, 0x2c, + 0x29, 0x5b, 0x0f, 0x65, 0x1d, 0x12, 0x58, 0x11, 0xb9, 0xda, 0x0b, 0x9c, 0xf6, 0xce, 0x69, 0xbb, + 0xb9, 0x13, 0x38, 0x1d, 0xe2, 0x07, 0x56, 0xa7, 0x2b, 0x35, 0x6b, 0x2d, 0xaf, 0xe5, 0xf1, 0x9f, + 0x3b, 0xec, 0x97, 0x90, 0x6a, 0x7f, 0x99, 0x82, 0xd5, 0xaa, 0x17, 0x94, 0x89, 0xe5, 0x93, 0x92, + 0xd7, 0xb6, 0x09, 0xd5, 0xd9, 0xd0, 0xa8, 0x08, 0x73, 0x94, 0x74, 0xdb, 0x4e, 0xd3, 0x52, 0x93, + 0xdb, 0xc9, 0x7b, 0x0b, 0x0f, 0xef, 0x64, 0x07, 0xf9, 0xca, 0xb1, 0xb3, 0xa6, 0xb0, 0x28, 0x12, + 0xbf, 0x49, 0x9d, 0x6e, 0xe0, 0xd1, 0x7c, 0xea, 0xeb, 0x57, 0x5b, 0x09, 0x33, 0x74, 0x45, 0x7b, + 0xb0, 0xd8, 0x66, 0x91, 0xf1, 0x29, 0x0f, 0xad, 0x4e, 0x5d, 0x3d, 0x94, 0xb9, 0xd0, 0x1e, 0xe4, + 0x84, 0x3e, 0x84, 0x34, 0xb5, 0xdc, 0x16, 0xc1, 0x8e, 0xad, 0x4e, 0x6f, 0x27, 0xef, 0x4d, 0xe7, + 0x37, 0xd9, 0x48, 0xaf, 0x5f, 0x6d, 0xcd, 0x99, 0x4c, 0x6e, 0x14, 0xbf, 0x1f, 0xfc, 0x34, 0xe7, + 0xb8, 0xad, 0x61, 0xa3, 0x2c, 0xcc, 0xf0, 0x28, 0x6a, 0x8a, 0x0f, 0xac, 0xc6, 0x0c, 0xcc, 0x67, + 0x6e, 0x0a, 0x33, 0x74, 0x1b, 0xa0, 0xd9, 0xf3, 0x03, 0xaf, 0x83, 0x3b, 0x7e, 0x4b, 0x9d, 0xd9, + 0x4e, 0xde, 0x9b, 0x97, 0x53, 0x9a, 0x17, 0xf2, 0x8a, 0xdf, 0xd2, 0xd6, 0x61, 0xad, 0xea, 0xd9, + 0xe4, 0xc8, 0xb5, 0xce, 0x2c, 0xa7, 0x6d, 0x1d, 0xb7, 0x09, 0x5f, 0x32, 0x6d, 0x03, 0x6e, 0x1c, + 0xb9, 0x7e, 0xaf, 0xdb, 0xf5, 0x68, 0x40, 0x6c, 0x93, 0x7c, 0xde, 0x23, 0x7e, 0x20, 0x54, 0x3f, + 0x4f, 0x02, 0xe2, 0xc9, 0x55, 0xbd, 0x60, 0xd7, 0xeb, 0xb9, 0xb6, 0x58, 0xe4, 0xe8, 0xac, 0x92, + 0x57, 0x9f, 0xd5, 0x87, 0x90, 0xf6, 0x03, 0x8f, 0x72, 0xb7, 0xa9, 0x61, 0xb7, 0x3a, 0x93, 0x0b, + 0x37, 0xf9, 0xd3, 0x9c, 0xe3, 0xb6, 0x86, 0xad, 0xfd, 0x6e, 0x1a, 0xae, 0xf3, 0x58, 0x07, 0xe4, + 0xa2, 0xe2, 0xf8, 0x1d, 0x2b, 0x68, 0x9e, 0x8a, 0x3c, 0x3e, 0x80, 0x15, 0x2a, 0xd2, 0xc5, 0x7e, + 0x60, 0xd1, 0x00, 0xbf, 0x20, 0x17, 0x3c, 0xa1, 0xc5, 0xfc, 0xdc, 0xf7, 0xaf, 0xb6, 0xa6, 0x0f, + 0xc8, 0x85, 0x79, 0x4d, 0x5a, 0xd4, 0x99, 0xc1, 0x01, 0xb9, 0x40, 0x3b, 0x10, 0x8a, 0x30, 0x71, + 0x6d, 0xee, 0x32, 0x35, 0xec, 0xb2, 0x24, 0xf5, 0xba, 0x6b, 0x33, 0x87, 0x53, 0xb8, 0x69, 0x93, + 0x2e, 0x25, 0x4d, 0x2b, 0x20, 0x36, 0xee, 0xc8, 0x0c, 0x88, 0x8d, 0xf9, 0xbc, 0xf8, 0xb6, 0x2e, + 0x3c, 0xd4, 0xe2, 0xce, 0x06, 0xd3, 0x8f, 0x1d, 0xb2, 0x8d, 0x41, 0xb0, 0x4a, 0x3f, 0x16, 0x37, + 0x45, 0x9f, 0xc1, 0x66, 0x64, 0x24, 0xbf, 0xd7, 0x6a, 0x11, 0x3f, 0xe8, 0x0f, 0x94, 0xba, 0xea, + 0x40, 0xa6, 0x3a, 0x88, 0x52, 0x0f, 0x83, 0x88, 0x11, 0xca, 0x30, 0xcb, 0x83, 0xf9, 0xea, 0xcc, + 0xf6, 0xf4, 0xbd, 0x85, 0x87, 0x6f, 0x4d, 0x8a, 0x66, 0xb8, 0x27, 0x5e, 0x7e, 0x5d, 0x6e, 0xcf, + 0xb2, 0xf0, 0x31, 0xdc, 0x80, 0x50, 0xd7, 0x6a, 0x9b, 0x32, 0x86, 0xf6, 0xef, 0x29, 0xd0, 0x4c, + 0x62, 0xd9, 0x4f, 0x9c, 0xe0, 0xd4, 0x71, 0x8f, 0xdc, 0x26, 0xa1, 0x81, 0xe5, 0xb8, 0xc1, 0x05, + 0xb7, 0x3c, 0xb3, 0xda, 0x62, 0x9b, 0xf6, 0x61, 0x99, 0x12, 0xcb, 0xc6, 0xfd, 0x9b, 0x2d, 0xaf, + 0xe6, 0xad, 0xc8, 0xe0, 0xec, 0xfa, 0x67, 0x4f, 0xdb, 0xcd, 0x6c, 0x23, 0x34, 0x92, 0xcb, 0xb5, + 0xc4, 0x5c, 0xfb, 0x42, 0x64, 0x02, 0x22, 0xe7, 0x8e, 0x1f, 0x38, 0x6e, 0x2b, 0x12, 0x6f, 0xea, + 0xea, 0xf1, 0x56, 0x42, 0xf7, 0x41, 0xcc, 0x3c, 0x2c, 0x75, 0xac, 0xf3, 0x48, 0xb8, 0xe9, 0x2b, + 0x84, 0x33, 0x17, 0x3b, 0xd6, 0xf9, 0x20, 0xc6, 0x73, 0x58, 0xf5, 0x8e, 0x7d, 0x42, 0xcf, 0x48, + 0x64, 0x9e, 0xbe, 0x9a, 0xe2, 0xab, 0x1c, 0x07, 0x1c, 0x35, 0x69, 0x3d, 0x9a, 0x1f, 0xf2, 0x46, + 0x15, 0xfe, 0xc7, 0xa9, 0xdf, 0x7f, 0xb9, 0x95, 0xd0, 0x6c, 0xb8, 0xd1, 0xa0, 0x96, 0xeb, 0x5b, + 0xcd, 0xc0, 0xf1, 0xdc, 0xdc, 0x31, 0xbf, 0xae, 0x62, 0x85, 0x0d, 0x98, 0xa5, 0xc4, 0xf2, 0x3d, + 0x97, 0xaf, 0xec, 0xf2, 0xc3, 0xf7, 0xb2, 0x63, 0x20, 0x9d, 0x1d, 0xf7, 0x35, 0xb9, 0x8b, 0x1c, + 0x57, 0x06, 0xd0, 0x9e, 0xc3, 0x5a, 0xc4, 0xf2, 0xb0, 0xe7, 0xcb, 0xbb, 0x56, 0x00, 0xe8, 0xf6, + 0xfc, 0x53, 0x42, 0x70, 0x70, 0xee, 0xca, 0x0d, 0xcc, 0xc4, 0xcc, 0x2b, 0xe2, 0x1c, 0x42, 0x90, + 0xf0, 0x6b, 0x9c, 0xbb, 0xda, 0x2f, 0x92, 0x70, 0x3d, 0x62, 0x60, 0x92, 0x80, 0x5e, 0x88, 0xf0, + 0x7b, 0x23, 0x33, 0xb8, 0x7f, 0xf9, 0x0c, 0xb8, 0x67, 0x5c, 0xfe, 0xe8, 0x6d, 0x98, 0x27, 0xe7, + 0x01, 0xb5, 0x38, 0x12, 0x4e, 0x45, 0x90, 0x30, 0xcd, 0xc5, 0x0c, 0x08, 0xff, 0x9a, 0x84, 0xf5, + 0x48, 0xac, 0x7a, 0x60, 0x05, 0x3d, 0x5f, 0xa4, 0xb1, 0x0e, 0xd3, 0xcc, 0x2f, 0x19, 0xf1, 0x63, + 0x02, 0x54, 0xed, 0xa7, 0x37, 0xc5, 0xd3, 0xfb, 0xaf, 0xcb, 0xd3, 0x8b, 0x84, 0xcc, 0xc6, 0xae, + 0xf2, 0x23, 0x98, 0x15, 0x72, 0x84, 0x60, 0xd9, 0xd4, 0x73, 0xf5, 0x5a, 0x15, 0x1f, 0x55, 0x0f, + 0xaa, 0xb5, 0x27, 0x55, 0x25, 0x81, 0x54, 0x58, 0x93, 0xb2, 0xc6, 0xd3, 0x2a, 0x2e, 0xd4, 0x2a, + 0x15, 0xa3, 0xd1, 0xd0, 0x8b, 0xca, 0x94, 0x96, 0x4a, 0x27, 0x95, 0xa4, 0xf6, 0xdb, 0x24, 0x28, + 0x4f, 0xa8, 0x13, 0x10, 0x76, 0xcf, 0x5c, 0x81, 0xd5, 0xe8, 0x23, 0x98, 0x73, 0xf8, 0xa7, 0xaf, + 0x26, 0xf9, 0xa9, 0xdb, 0x88, 0xd9, 0x1d, 0xe1, 0x10, 0x96, 0x3b, 0x69, 0x8f, 0x1e, 0xc1, 0xb2, + 0x28, 0x77, 0x3e, 0x03, 0x3e, 0xb7, 0x49, 0x64, 0xad, 0xba, 0xce, 0xcc, 0xbe, 0x7f, 0xb5, 0xb5, + 0xc4, 0x8b, 0x4d, 0x5d, 0x2a, 0xcd, 0xa5, 0x76, 0xf4, 0x73, 0x3f, 0x95, 0x9e, 0x52, 0xa6, 0xb5, + 0x3f, 0x84, 0x39, 0x35, 0x3c, 0xaf, 0xd6, 0x96, 0xe7, 0x32, 0x07, 0xf3, 0x3f, 0xea, 0xd2, 0x0f, + 0xbc, 0x50, 0x15, 0x14, 0xab, 0x19, 0xf4, 0xac, 0xf6, 0x8f, 0xbb, 0xee, 0xd7, 0x84, 0x73, 0x5f, + 0xac, 0xad, 0x01, 0xaa, 0x75, 0x59, 0x91, 0x73, 0x28, 0xf1, 0x1b, 0xe7, 0xae, 0x28, 0x74, 0x75, + 0x58, 0x2b, 0x78, 0xae, 0xed, 0xb0, 0xcd, 0xdb, 0xb5, 0x9c, 0x76, 0x78, 0xb1, 0xfe, 0x17, 0x16, + 0xe5, 0xe8, 0x67, 0x56, 0xbb, 0x47, 0xe4, 0x1c, 0xe2, 0xea, 0xf1, 0x63, 0xa6, 0x37, 0x17, 0x84, + 0x35, 0xff, 0xd0, 0xfe, 0x9c, 0x04, 0x24, 0xca, 0x34, 0xf9, 0x09, 0x69, 0xf6, 0x2f, 0x6b, 0x06, + 0xe6, 0x3a, 0xc4, 0xf7, 0xad, 0x16, 0x19, 0x3a, 0x67, 0xa1, 0x10, 0x3d, 0x82, 0x79, 0x59, 0x80, + 0x88, 0x2d, 0xa7, 0x3a, 0x91, 0x00, 0x84, 0xeb, 0xd5, 0x77, 0x40, 0x1f, 0x43, 0x3a, 0x44, 0x38, + 0x89, 0x63, 0x6f, 0x72, 0xee, 0xdb, 0x6b, 0x9f, 0xc3, 0x5a, 0xae, 0x73, 0xec, 0xb4, 0x7a, 0x5e, + 0xcf, 0x37, 0x89, 0xdf, 0x6b, 0x07, 0x57, 0xcb, 0xf8, 0x23, 0x58, 0x78, 0x49, 0xad, 0x6e, 0x97, + 0xd8, 0x98, 0x50, 0x1a, 0x93, 0x73, 0x78, 0x45, 0x78, 0x38, 0x13, 0xa4, 0xb1, 0x4e, 0xa9, 0x76, + 0x83, 0xd5, 0xf6, 0x93, 0x60, 0x8f, 0x7a, 0xbd, 0x6e, 0x91, 0xb4, 0x49, 0xb8, 0x4a, 0x1a, 0x86, + 0x75, 0xc9, 0xad, 0x0a, 0x1e, 0xa5, 0xbd, 0x2e, 0xdb, 0x19, 0x91, 0x0d, 0xbb, 0xe1, 0xec, 0x07, + 0x1e, 0xbd, 0xa9, 0x69, 0x2e, 0xae, 0xf8, 0x2d, 0xa4, 0xc1, 0x7c, 0x97, 0x7a, 0x4d, 0xe2, 0xfb, + 0x72, 0x09, 0xd3, 0x7d, 0x2c, 0x0a, 0xc5, 0x5a, 0x1d, 0x90, 0x1c, 0x20, 0x7a, 0x62, 0x3f, 0x01, + 0x90, 0x24, 0x30, 0x24, 0x37, 0x33, 0xf9, 0x8c, 0x2c, 0x83, 0xf3, 0xd2, 0x9e, 0xf3, 0x94, 0xc1, + 0x07, 0x5b, 0x7d, 0xf1, 0xd3, 0xd6, 0x0e, 0x00, 0x71, 0xfe, 0x32, 0xc6, 0x97, 0xfa, 0xc4, 0x27, + 0x79, 0x75, 0xe2, 0x53, 0x67, 0xc4, 0xec, 0xd4, 0x72, 0xed, 0x36, 0xc3, 0xea, 0x80, 0x5e, 0xf4, + 0x39, 0x1b, 0x7a, 0x08, 0xa9, 0xae, 0x4e, 0x69, 0xcc, 0x79, 0x1c, 0x5a, 0x6a, 0x39, 0x6b, 0x6e, + 0x2b, 0xab, 0xc8, 0xbf, 0x92, 0x70, 0x77, 0x14, 0x48, 0x59, 0xfd, 0x3e, 0x64, 0xb4, 0xda, 0x24, + 0x27, 0x94, 0x84, 0x88, 0x3f, 0x09, 0x0b, 0x9f, 0xc3, 0x6c, 0x70, 0xee, 0x86, 0x24, 0x6e, 0x31, + 0x5f, 0x64, 0xaa, 0xbf, 0xbd, 0xda, 0xfa, 0xa0, 0xe5, 0x04, 0xa7, 0xbd, 0xe3, 0x6c, 0xd3, 0xeb, + 0xec, 0xf4, 0xf3, 0xb1, 0x8f, 0x07, 0xbf, 0x77, 0xba, 0x2f, 0x5a, 0x3b, 0x9c, 0xe7, 0xf7, 0x7a, + 0x8e, 0x9d, 0x3d, 0x3a, 0x32, 0x8a, 0xaf, 0x5f, 0x6d, 0xcd, 0x34, 0xce, 0x5d, 0xa3, 0x68, 0xce, + 0x04, 0xe7, 0xae, 0x61, 0xa3, 0x5d, 0x58, 0x08, 0x06, 0xd9, 0xc9, 0x13, 0x7c, 0xb5, 0x3a, 0x13, + 0x75, 0xd4, 0x76, 0x61, 0xab, 0x71, 0xee, 0xe6, 0xda, 0x8c, 0x3d, 0x5c, 0xe8, 0x6e, 0xd3, 0xeb, + 0x31, 0x4a, 0x22, 0x0f, 0x97, 0x98, 0xdf, 0x6d, 0x80, 0x2e, 0x25, 0x67, 0x98, 0x9f, 0x9a, 0xa1, + 0x69, 0xce, 0x33, 0xb9, 0x38, 0x86, 0xbf, 0x4a, 0xc2, 0x1a, 0x03, 0xcd, 0x16, 0xa1, 0xb5, 0x33, + 0x42, 0x4f, 0xda, 0xde, 0x4b, 0xe1, 0xbd, 0x01, 0xd3, 0x31, 0x6c, 0x93, 0xc9, 0xd0, 0x7d, 0x58, + 0x6a, 0xf6, 0x28, 0x25, 0x6e, 0x20, 0x51, 0x43, 0x90, 0x5d, 0x11, 0x7b, 0x51, 0xaa, 0x38, 0x44, + 0xa0, 0xf7, 0xe1, 0x9a, 0xe3, 0x36, 0x29, 0xe9, 0x0c, 0x8c, 0xa7, 0x23, 0xc6, 0xcb, 0x7d, 0xa5, + 0x40, 0x94, 0xaf, 0x92, 0x70, 0x33, 0xcf, 0x18, 0xe3, 0x00, 0xe6, 0xc8, 0x89, 0x47, 0xc9, 0x5e, + 0xa1, 0x8f, 0xb7, 0x8d, 0x1f, 0x85, 0xb7, 0x03, 0x22, 0xc3, 0x42, 0x9c, 0xb2, 0x43, 0xe0, 0xb5, + 0xed, 0x1f, 0x02, 0xb4, 0x03, 0x2f, 0xad, 0x03, 0x48, 0xd4, 0x99, 0x8a, 0xe3, 0xfb, 0x8e, 0xdb, + 0x12, 0xb9, 0x3d, 0x82, 0xc5, 0x97, 0xd4, 0x73, 0x5b, 0x58, 0x54, 0x1d, 0x99, 0xde, 0xe4, 0x22, + 0x65, 0x2e, 0x70, 0x73, 0xf1, 0x11, 0x2e, 0xf7, 0xd4, 0xf8, 0x72, 0xb3, 0xbe, 0xa6, 0x42, 0x28, + 0xa3, 0xac, 0x87, 0xd4, 0x6b, 0x51, 0xe2, 0x8b, 0xc2, 0xab, 0xfd, 0x7a, 0x0a, 0x56, 0x39, 0x97, + 0xdd, 0x25, 0xf2, 0xfe, 0x88, 0x44, 0x0e, 0x46, 0xa8, 0xc6, 0xfb, 0x31, 0xb7, 0x27, 0xc6, 0x2f, + 0xbe, 0x90, 0xff, 0x31, 0xd9, 0xaf, 0xe4, 0x9b, 0xb0, 0x2e, 0xab, 0xb6, 0xa9, 0x1f, 0x96, 0x8d, + 0x42, 0x0e, 0x9b, 0x7a, 0xa5, 0xf6, 0x58, 0x2f, 0x2a, 0x09, 0xb4, 0x0e, 0x28, 0xd4, 0xe5, 0xaa, + 0x7b, 0x3a, 0xae, 0x1f, 0x96, 0x8d, 0x86, 0x92, 0x44, 0x37, 0x60, 0x75, 0x48, 0x5e, 0xd1, 0xcd, + 0x3d, 0x56, 0xe8, 0x23, 0x14, 0xc0, 0xcc, 0xed, 0x36, 0x70, 0xbd, 0x9a, 0x3b, 0xac, 0x97, 0x6a, + 0x0d, 0x65, 0x1a, 0x65, 0x60, 0x53, 0x6a, 0xca, 0xb5, 0x3d, 0xa3, 0x90, 0x2b, 0xe3, 0xda, 0x61, + 0x1d, 0x57, 0x8c, 0x7a, 0xdd, 0xa8, 0xee, 0x29, 0xa9, 0x88, 0x67, 0xbd, 0x5c, 0x7b, 0x82, 0x0b, + 0xb5, 0x6a, 0xfd, 0xa8, 0xa2, 0x9b, 0xca, 0x8c, 0x66, 0x81, 0x6a, 0xb8, 0x36, 0x09, 0x08, 0xed, + 0x38, 0xae, 0x15, 0x90, 0x82, 0xd7, 0xe9, 0x38, 0x12, 0xe2, 0x75, 0x58, 0xf0, 0x03, 0xab, 0xc5, + 0x69, 0xf5, 0x0f, 0xe4, 0x77, 0x20, 0x1d, 0x19, 0xc1, 0x5b, 0x85, 0x15, 0xc3, 0x3d, 0xb3, 0xda, + 0x8e, 0xcd, 0x2b, 0x8c, 0xd8, 0x88, 0x2f, 0xaf, 0xc3, 0x02, 0xff, 0x55, 0x24, 0x81, 0xe5, 0xb4, + 0x91, 0x09, 0x8a, 0xeb, 0x05, 0x78, 0xa8, 0xc3, 0x16, 0x03, 0xbe, 0x13, 0xb3, 0x15, 0x31, 0x5d, + 0x7e, 0x29, 0x61, 0x2e, 0xbb, 0x43, 0x62, 0x54, 0x83, 0x6b, 0xa2, 0x25, 0x65, 0x91, 0x4f, 0x18, + 0xf4, 0xca, 0xc3, 0x7b, 0x77, 0xd2, 0xee, 0x0e, 0x41, 0x74, 0x89, 0x35, 0x1a, 0x51, 0x29, 0x7a, + 0x0a, 0x48, 0x04, 0x7c, 0x41, 0x2e, 0xfa, 0x4d, 0x9f, 0xc4, 0xa3, 0x7b, 0x93, 0x62, 0x8e, 0x76, + 0xa8, 0xa5, 0x84, 0xa9, 0xd0, 0x11, 0x05, 0xfa, 0x59, 0x12, 0xb6, 0x79, 0x3f, 0xf4, 0x92, 0xb7, + 0x4d, 0xb8, 0x37, 0xe8, 0x9b, 0xf8, 0xdd, 0x60, 0x8d, 0x93, 0x6c, 0xf6, 0x3e, 0x8c, 0x1b, 0xe8, + 0x8d, 0x0d, 0x57, 0x29, 0x61, 0xde, 0xa2, 0x97, 0x59, 0xa1, 0xff, 0x87, 0xd5, 0x08, 0x58, 0x62, + 0x4b, 0xf4, 0x03, 0xfc, 0xe1, 0x60, 0xe1, 0xe1, 0x83, 0x2b, 0x35, 0x0f, 0xe1, 0x48, 0x28, 0x18, + 0x53, 0xa1, 0x06, 0x28, 0xd1, 0xf0, 0x8c, 0xff, 0xab, 0xb3, 0x3c, 0xf6, 0xbb, 0x97, 0xc7, 0xee, + 0xb7, 0x1b, 0xa5, 0x84, 0x79, 0x2d, 0x18, 0x96, 0xa3, 0x27, 0xb0, 0x12, 0x8d, 0x4a, 0xd9, 0xcd, + 0x54, 0xe7, 0x26, 0x6e, 0x48, 0x6c, 0x9f, 0xc1, 0x36, 0x24, 0x18, 0x51, 0xa0, 0x4f, 0x21, 0x3a, + 0x09, 0xec, 0x73, 0xf2, 0xae, 0xa6, 0x79, 0xe4, 0xfb, 0x57, 0x26, 0xfa, 0xa5, 0x84, 0x19, 0xcd, + 0x4f, 0x68, 0x50, 0x89, 0xa1, 0x9e, 0x13, 0x90, 0x10, 0xf5, 0xe6, 0x79, 0xd4, 0xdb, 0x31, 0x51, + 0x47, 0x09, 0x7d, 0x29, 0xc1, 0x10, 0xb0, 0x2f, 0x43, 0x06, 0x2c, 0x89, 0x48, 0x81, 0xe7, 0x61, + 0x06, 0xce, 0x70, 0x79, 0xa8, 0x08, 0xab, 0xe9, 0x87, 0x12, 0x32, 0x76, 0x59, 0xbc, 0x2e, 0xa6, + 0x92, 0x04, 0xf3, 0x0b, 0xbf, 0x30, 0xf1, 0xb2, 0x8c, 0xb3, 0x65, 0x76, 0x59, 0xbc, 0xa8, 0x94, + 0x6d, 0x78, 0x33, 0xa4, 0xcf, 0xf8, 0x84, 0xf3, 0x67, 0x75, 0x71, 0xe2, 0x86, 0xc7, 0x31, 0x6d, + 0xb6, 0xe1, 0xcd, 0x61, 0x39, 0xaa, 0x86, 0x6d, 0x09, 0x95, 0xfc, 0x59, 0x5d, 0x9a, 0x98, 0xe5, + 0x38, 0xcf, 0x66, 0x59, 0xb6, 0xa3, 0x52, 0x96, 0xa5, 0xeb, 0xd9, 0x04, 0xf7, 0x06, 0x2f, 0x60, + 0xea, 0xf2, 0xc4, 0x2c, 0xe3, 0xde, 0xca, 0x58, 0x96, 0xee, 0xb0, 0x5c, 0x00, 0xc5, 0x49, 0x80, + 0x5b, 0x8c, 0xc2, 0x62, 0x5b, 0x70, 0x58, 0x55, 0xb9, 0x04, 0x28, 0x62, 0xe8, 0xae, 0x00, 0x8a, + 0x61, 0x05, 0x3b, 0x97, 0x21, 0x17, 0x6d, 0xf6, 0x39, 0xb0, 0xba, 0x32, 0xf1, 0x5c, 0xc6, 0xf3, + 0x65, 0x76, 0x2e, 0xe9, 0xa8, 0x86, 0xe3, 0xa5, 0x8c, 0x1d, 0x9e, 0x27, 0x34, 0x19, 0x2f, 0xc7, + 0x78, 0x32, 0xc7, 0xcb, 0xa8, 0x94, 0x2d, 0xae, 0x15, 0xf6, 0x0e, 0x98, 0xf2, 0xe6, 0x41, 0xdd, + 0x9c, 0xb8, 0xb8, 0x71, 0x6d, 0x06, 0x5b, 0x5c, 0x6b, 0x58, 0xce, 0xd2, 0x14, 0xcc, 0x79, 0x00, + 0xeb, 0x37, 0x27, 0xa6, 0x39, 0xce, 0xbc, 0x59, 0x9a, 0x7e, 0x54, 0x8a, 0x7e, 0x99, 0x84, 0x3b, + 0x63, 0x28, 0xc2, 0x91, 0x18, 0xf3, 0x87, 0x65, 0x4c, 0x05, 0x05, 0x56, 0xdf, 0xe2, 0xc3, 0xfc, + 0xcf, 0x15, 0x80, 0x25, 0x96, 0x3d, 0x97, 0x12, 0xe6, 0x76, 0xf0, 0x06, 0x43, 0xb6, 0x66, 0x8e, + 0xe0, 0x96, 0xd8, 0x93, 0xe4, 0x52, 0xdd, 0x9a, 0xb8, 0x66, 0x71, 0x34, 0x94, 0xad, 0x99, 0x33, + 0x2c, 0x67, 0xe0, 0xde, 0x1b, 0xbc, 0xe7, 0x62, 0xd9, 0x1a, 0xaa, 0xdb, 0x13, 0xc1, 0x7d, 0xc2, + 0xeb, 0x2f, 0x03, 0xf7, 0xde, 0x98, 0x0a, 0x3d, 0x07, 0xa5, 0xdf, 0x89, 0xe3, 0x63, 0x4e, 0x3f, + 0x55, 0x8d, 0xc7, 0xce, 0xc6, 0xc4, 0xbe, 0x84, 0xad, 0x72, 0x8c, 0x1f, 0xd6, 0xa0, 0x97, 0x70, + 0x8b, 0xf5, 0x16, 0x96, 0xe0, 0xed, 0x98, 0x0c, 0x88, 0xbb, 0xa4, 0xe9, 0xb7, 0xf9, 0x48, 0x0f, + 0xe3, 0xb6, 0xe5, 0x72, 0xba, 0x5f, 0x4a, 0x98, 0x9b, 0xc1, 0x44, 0x13, 0x86, 0x35, 0x02, 0xa1, + 0x59, 0xad, 0x67, 0xa4, 0x55, 0xbd, 0x33, 0xf1, 0x9c, 0x8d, 0x93, 0x5b, 0x76, 0xce, 0x9c, 0xa8, + 0x14, 0x1d, 0xc1, 0x4a, 0x87, 0x91, 0x52, 0xec, 0xb8, 0xec, 0x60, 0x71, 0x5a, 0xaa, 0xde, 0x9d, + 0xb8, 0xb7, 0x71, 0x04, 0x96, 0xad, 0x4f, 0x67, 0x58, 0x8e, 0xfe, 0x4f, 0xd2, 0x9c, 0x13, 0xc2, + 0x77, 0x96, 0x55, 0xc0, 0x77, 0x26, 0x32, 0xa7, 0x18, 0x12, 0xcb, 0x98, 0x53, 0x3f, 0x80, 0xa8, + 0x7e, 0x9f, 0xc1, 0x9a, 0x13, 0x65, 0x85, 0xb8, 0xc9, 0x69, 0xa1, 0xfa, 0x2e, 0x8f, 0xfb, 0x5e, + 0xec, 0xfc, 0xe3, 0x49, 0x64, 0x29, 0x61, 0xae, 0x3a, 0xe3, 0x3a, 0xf4, 0x18, 0x56, 0x1d, 0x41, + 0x0a, 0x25, 0xe7, 0x13, 0x5b, 0x79, 0x6f, 0xec, 0x4f, 0x95, 0xc1, 0x00, 0x23, 0x14, 0x92, 0x61, + 0x98, 0x33, 0x2a, 0xcc, 0xcf, 0xc1, 0x0c, 0x6f, 0x99, 0xf6, 0x53, 0xe9, 0x6b, 0x8a, 0xb2, 0x9f, + 0x4a, 0xaf, 0x2a, 0x6b, 0xfb, 0xa9, 0xf4, 0x9a, 0x72, 0x7d, 0x3f, 0x95, 0xbe, 0xae, 0xac, 0xef, + 0xa7, 0xd2, 0xeb, 0xca, 0x8d, 0xfd, 0x54, 0xfa, 0x86, 0xa2, 0xee, 0xa7, 0xd2, 0xaa, 0xb2, 0xb1, + 0x9f, 0x4a, 0x6f, 0x28, 0x9b, 0xfb, 0xa9, 0xf4, 0x2d, 0x25, 0xb3, 0x9f, 0x4a, 0x67, 0x94, 0xad, + 0xfd, 0x54, 0xfa, 0x6d, 0x45, 0xd3, 0xee, 0x73, 0x86, 0x7a, 0xe8, 0xf9, 0xbc, 0xfe, 0xa0, 0x4d, + 0x98, 0x61, 0x13, 0x39, 0x97, 0x0f, 0x00, 0x82, 0xe7, 0x0a, 0x91, 0xf6, 0xc5, 0x0c, 0xcc, 0x84, + 0x7f, 0x3f, 0x8c, 0x3c, 0x8b, 0x6c, 0xc8, 0xae, 0x7e, 0x25, 0xf2, 0xc4, 0x2f, 0x0c, 0x06, 0x6f, + 0x25, 0x3f, 0x1d, 0xa6, 0x5e, 0x94, 0xf0, 0x7f, 0x2e, 0x38, 0xb1, 0x5c, 0x8e, 0x3d, 0x6d, 0x43, + 0x70, 0xc3, 0x8d, 0xf3, 0x77, 0xe4, 0x38, 0x6f, 0x0d, 0xc6, 0x19, 0xb7, 0x1a, 0x62, 0x66, 0x52, + 0x86, 0x0a, 0xb0, 0xd4, 0x73, 0xc9, 0x79, 0xd7, 0xf3, 0x89, 0xcd, 0xeb, 0x7e, 0xea, 0x2a, 0x44, + 0xdf, 0x5c, 0xec, 0x3b, 0xb1, 0x6a, 0xbf, 0x03, 0x0b, 0x1e, 0x75, 0x5a, 0x8e, 0x8b, 0x59, 0x2d, + 0xe4, 0xac, 0x71, 0x26, 0xbf, 0x2c, 0xdf, 0x0a, 0x67, 0x59, 0xdd, 0x34, 0x8a, 0x26, 0x08, 0x13, + 0xf6, 0x85, 0x0e, 0x61, 0xd6, 0xe6, 0xd4, 0x5f, 0xb2, 0xc0, 0xcc, 0xa4, 0xf7, 0x0a, 0xd1, 0x20, + 0xe4, 0x55, 0x39, 0x3f, 0x65, 0x30, 0x3f, 0xa1, 0x31, 0x65, 0x1c, 0xf4, 0xdf, 0xe1, 0x06, 0xcd, + 0x5d, 0x16, 0x30, 0xdc, 0x4f, 0xb9, 0x75, 0xa8, 0x07, 0xd3, 0xae, 0xf7, 0x52, 0x32, 0xbb, 0x37, + 0x74, 0xb5, 0x45, 0x39, 0x9f, 0x47, 0x57, 0x7f, 0xd5, 0x60, 0x01, 0x0a, 0x6d, 0xaf, 0xf9, 0x62, + 0xf0, 0x27, 0x01, 0x1b, 0x0f, 0x1d, 0xc0, 0x12, 0x03, 0x32, 0xbb, 0x0f, 0x62, 0x82, 0x04, 0x6e, + 0x47, 0x12, 0x08, 0xff, 0x40, 0xcd, 0xea, 0xc2, 0x30, 0xfa, 0x7a, 0xb3, 0x48, 0x22, 0x32, 0xf1, + 0x8a, 0x23, 0xde, 0x5c, 0x1f, 0xfc, 0x7d, 0x0a, 0xd4, 0x49, 0xcf, 0xfa, 0xac, 0x0f, 0xcc, 0xe5, + 0x6b, 0x66, 0x03, 0x8f, 0x3d, 0x2f, 0xdf, 0x85, 0xb7, 0x87, 0x34, 0xfc, 0x43, 0x2f, 0x62, 0x53, + 0x2f, 0xd4, 0xcc, 0x22, 0xde, 0xad, 0x1d, 0x55, 0x8b, 0x4a, 0x92, 0x35, 0x9a, 0x43, 0x66, 0x85, + 0xb2, 0xa1, 0x57, 0xd9, 0xd7, 0xbe, 0x5e, 0x60, 0x8d, 0xe8, 0x16, 0xdc, 0x1c, 0xd2, 0x1f, 0x1e, + 0xd5, 0x4b, 0xba, 0x19, 0x46, 0x53, 0x52, 0xe8, 0x26, 0xdc, 0x18, 0x1f, 0x07, 0xd7, 0x0f, 0x73, + 0x55, 0x65, 0x06, 0xe5, 0xe0, 0x93, 0x61, 0x65, 0xd9, 0xd4, 0x73, 0xc5, 0x67, 0x83, 0xd7, 0x6e, + 0x5c, 0x33, 0xb1, 0x59, 0x2b, 0x97, 0xf5, 0x22, 0xce, 0xe7, 0x0a, 0x07, 0xf8, 0xb0, 0x56, 0xaf, + 0x1b, 0xf9, 0xb2, 0xce, 0xbb, 0xeb, 0xdc, 0x33, 0x65, 0x16, 0xbd, 0x0b, 0xb7, 0x87, 0x42, 0x54, + 0xf5, 0x27, 0xb8, 0xac, 0xe7, 0xea, 0x3a, 0x3e, 0x34, 0xf5, 0xc7, 0x7a, 0xb5, 0x51, 0xc7, 0x8d, + 0xa7, 0x55, 0x25, 0x8d, 0xee, 0xc3, 0xdd, 0x21, 0xc3, 0x86, 0x51, 0xd1, 0xeb, 0x8d, 0x5c, 0xe5, + 0x10, 0x17, 0x72, 0x85, 0x92, 0x2e, 0xa7, 0xa4, 0x17, 0x95, 0xb9, 0xcd, 0xd4, 0x17, 0x5f, 0x65, + 0x12, 0x1a, 0x5b, 0xde, 0xa9, 0x07, 0x7f, 0x1a, 0xfe, 0x9f, 0x20, 0xf2, 0x9f, 0x83, 0x68, 0xb2, + 0x1b, 0xe6, 0xb3, 0xf1, 0xc5, 0xe5, 0x1d, 0x3d, 0xd3, 0x3c, 0x31, 0x8d, 0x86, 0x8e, 0x1b, 0xb5, + 0x1a, 0xae, 0x95, 0xd9, 0x72, 0xf2, 0x27, 0x00, 0xa6, 0xa8, 0xeb, 0xa6, 0x91, 0x2b, 0x1b, 0x9f, + 0xe6, 0xf2, 0x65, 0x5d, 0x99, 0x46, 0xb7, 0x60, 0x43, 0xc8, 0x73, 0xf5, 0x67, 0xd5, 0x82, 0x74, + 0xdb, 0xcd, 0x19, 0xe5, 0x23, 0x53, 0x57, 0x66, 0x90, 0x06, 0x19, 0xa1, 0x16, 0x0b, 0x83, 0x8b, + 0x7a, 0xae, 0x58, 0x36, 0xaa, 0x3a, 0xd6, 0x9f, 0x16, 0x74, 0xbd, 0xa8, 0x17, 0x95, 0x59, 0x91, + 0xf4, 0x83, 0x8f, 0x01, 0x8d, 0xa3, 0x00, 0x4a, 0x43, 0xaa, 0x5a, 0xab, 0xea, 0x4a, 0x02, 0x2d, + 0xc0, 0x1c, 0x5b, 0xc8, 0xda, 0xee, 0xae, 0x92, 0x44, 0x4b, 0x30, 0x6f, 0x54, 0x2a, 0x7a, 0xd1, + 0xc8, 0x35, 0x74, 0x65, 0x2a, 0x7f, 0xff, 0xeb, 0x7f, 0x66, 0x12, 0x5f, 0xbf, 0xce, 0x24, 0xbf, + 0x79, 0x9d, 0x49, 0x7e, 0xfb, 0x3a, 0x93, 0xfc, 0xc7, 0xeb, 0x4c, 0xf2, 0x37, 0xdf, 0x65, 0x12, + 0xdf, 0x7c, 0x97, 0x49, 0x7c, 0xfb, 0x5d, 0x26, 0xf1, 0xe9, 0x9c, 0xc4, 0x85, 0xff, 0x04, 0x00, + 0x00, 0xff, 0xff, 0x19, 0xeb, 0x00, 0x91, 0x16, 0x20, 0x00, 0x00, } diff --git a/pkg/roachpb/errors.proto b/pkg/roachpb/errors.proto index f25f249e6370..b9ba62670a5a 100644 --- a/pkg/roachpb/errors.proto +++ b/pkg/roachpb/errors.proto @@ -271,6 +271,11 @@ message TransactionStatusError { message WriteIntentError { repeated roachpb.Intent intents = 1 [(gogoproto.nullable) = false]; reserved 2; + // The sequence of the lease that the operation which hit this error was + // operating under. Used on the server to avoid adding discovered locks + // which were discovered under old leases to the lock table. + optional int64 lease_sequence = 3 [(gogoproto.nullable) = false, + (gogoproto.casttype) = "LeaseSequence"]; } // A WriteTooOldError indicates that a write encountered a versioned @@ -472,6 +477,16 @@ message IndeterminateCommitError { optional roachpb.Transaction staging_txn = 1 [(gogoproto.nullable) = false]; } +// An InvalidLeaseError indicates that the request could not be completed +// because the replica does not have a valid lease. The request should be +// resubmitted on the current replica after a lease is acquired or should be +// redirected (using a NotLeaseHolderError) to the current leaseholder if one +// already exists. +// +// This error is handled by the Store and should not escape to higher levels. +message InvalidLeaseError { +} + // ErrorDetail is a union type containing all available errors. message ErrorDetail { reserved 15, 19, 20, 21, 22, 23, 24, 25, 29, 30, 33; @@ -509,6 +524,7 @@ message ErrorDetail { MergeInProgressError merge_in_progress = 37; RangeFeedRetryError rangefeed_retry = 38; IndeterminateCommitError indeterminate_commit = 39; + InvalidLeaseError invalid_lease_error = 40; } } diff --git a/pkg/server/testserver.go b/pkg/server/testserver.go index 382246545bc2..f52cd3ec6b43 100644 --- a/pkg/server/testserver.go +++ b/pkg/server/testserver.go @@ -34,6 +34,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvclient/kvcoord" "github.com/cockroachdb/cockroach/pkg/kv/kvclient/kvtenant" "github.com/cockroachdb/cockroach/pkg/kv/kvserver" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/liveness" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/protectedts" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/protectedts/ptpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/protectedts/ptprovider" @@ -63,6 +64,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/metric" "github.com/cockroachdb/cockroach/pkg/util/netutil" "github.com/cockroachdb/cockroach/pkg/util/protoutil" + "github.com/cockroachdb/cockroach/pkg/util/retry" "github.com/cockroachdb/cockroach/pkg/util/stop" "github.com/cockroachdb/cockroach/pkg/util/timeutil" "github.com/cockroachdb/errors" @@ -371,6 +373,27 @@ func (ts *TestServer) NodeLiveness() interface{} { return nil } +// HeartbeatNodeLiveness heartbeats the server's NodeLiveness record. +func (ts *TestServer) HeartbeatNodeLiveness() error { + if ts == nil { + return errors.New("no node liveness instance") + } + nl := ts.nodeLiveness + l, ok := nl.Self() + if !ok { + return errors.New("liveness not found") + } + + var err error + ctx := context.Background() + for r := retry.StartWithCtx(ctx, retry.Options{MaxRetries: 5}); r.Next(); { + if err = nl.Heartbeat(ctx, l); !errors.Is(err, liveness.ErrEpochIncremented) { + break + } + } + return err +} + // RPCContext returns the rpc context used by the TestServer. func (ts *TestServer) RPCContext() *rpc.Context { if ts != nil { @@ -1385,27 +1408,23 @@ func (ts *TestServer) ForceTableGC( return pErr.GoError() } -// ScratchRangeEx splits off a range suitable to be used as KV scratch space. -// (it doesn't overlap system spans or SQL tables). -func (ts *TestServer) ScratchRangeEx() (roachpb.RangeDescriptor, error) { - scratchKey := keys.TableDataMax - _, rngDesc, err := ts.SplitRange(scratchKey) - if err != nil { - return roachpb.RangeDescriptor{}, err - } - return rngDesc, nil -} - // ScratchRange is like ScratchRangeEx, but only returns the start key of the // new range instead of the range descriptor. func (ts *TestServer) ScratchRange() (roachpb.Key, error) { - desc, err := ts.ScratchRangeEx() + _, desc, err := ts.ScratchRangeEx() if err != nil { return nil, err } return desc.StartKey.AsRawKey(), nil } +// ScratchRangeEx splits off a range suitable to be used as KV scratch space. +// (it doesn't overlap system spans or SQL tables). +func (ts *TestServer) ScratchRangeEx() (roachpb.RangeDescriptor, roachpb.RangeDescriptor, error) { + scratchKey := keys.TableDataMax + return ts.SplitRange(scratchKey) +} + // ScratchRangeWithExpirationLease is like ScratchRangeWithExpirationLeaseEx but // returns a key for the RHS ranges, instead of both descriptors from the split. func (ts *TestServer) ScratchRangeWithExpirationLease() (roachpb.Key, error) { diff --git a/pkg/storage/enginepb/mvcc.pb.go b/pkg/storage/enginepb/mvcc.pb.go index aafef5ffdd5a..a36775f8a176 100644 --- a/pkg/storage/enginepb/mvcc.pb.go +++ b/pkg/storage/enginepb/mvcc.pb.go @@ -78,7 +78,7 @@ type MVCCMetadata struct { func (m *MVCCMetadata) Reset() { *m = MVCCMetadata{} } func (*MVCCMetadata) ProtoMessage() {} func (*MVCCMetadata) Descriptor() ([]byte, []int) { - return fileDescriptor_mvcc_69eca97822587899, []int{0} + return fileDescriptor_mvcc_33f4aa682d7df09c, []int{0} } func (m *MVCCMetadata) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -118,7 +118,7 @@ type MVCCMetadata_SequencedIntent struct { func (m *MVCCMetadata_SequencedIntent) Reset() { *m = MVCCMetadata_SequencedIntent{} } func (*MVCCMetadata_SequencedIntent) ProtoMessage() {} func (*MVCCMetadata_SequencedIntent) Descriptor() ([]byte, []int) { - return fileDescriptor_mvcc_69eca97822587899, []int{0, 0} + return fileDescriptor_mvcc_33f4aa682d7df09c, []int{0, 0} } func (m *MVCCMetadata_SequencedIntent) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -162,7 +162,7 @@ func (m *MVCCMetadataSubsetForMergeSerialization) Reset() { } func (*MVCCMetadataSubsetForMergeSerialization) ProtoMessage() {} func (*MVCCMetadataSubsetForMergeSerialization) Descriptor() ([]byte, []int) { - return fileDescriptor_mvcc_69eca97822587899, []int{1} + return fileDescriptor_mvcc_33f4aa682d7df09c, []int{1} } func (m *MVCCMetadataSubsetForMergeSerialization) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -278,7 +278,7 @@ type MVCCStats struct { // This tracks the same quantity as (key_bytes + val_bytes), but // for system-local metadata keys (which aren't counted in either // key_bytes or val_bytes). Each of the keys falling into this group - // is documented in keys/constants.go under the localPrefix constant + // is documented in keys/constants.go under the LocalPrefix constant // and is prefixed by either LocalRangeIDPrefix or LocalRangePrefix. SysBytes int64 `protobuf:"fixed64,12,opt,name=sys_bytes,json=sysBytes" json:"sys_bytes"` // sys_count is the number of meta keys tracked under sys_bytes. @@ -292,7 +292,7 @@ func (m *MVCCStats) Reset() { *m = MVCCStats{} } func (m *MVCCStats) String() string { return proto.CompactTextString(m) } func (*MVCCStats) ProtoMessage() {} func (*MVCCStats) Descriptor() ([]byte, []int) { - return fileDescriptor_mvcc_69eca97822587899, []int{2} + return fileDescriptor_mvcc_33f4aa682d7df09c, []int{2} } func (m *MVCCStats) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -346,7 +346,7 @@ func (m *MVCCStatsLegacyRepresentation) Reset() { *m = MVCCStatsLegacyRe func (m *MVCCStatsLegacyRepresentation) String() string { return proto.CompactTextString(m) } func (*MVCCStatsLegacyRepresentation) ProtoMessage() {} func (*MVCCStatsLegacyRepresentation) Descriptor() ([]byte, []int) { - return fileDescriptor_mvcc_69eca97822587899, []int{3} + return fileDescriptor_mvcc_33f4aa682d7df09c, []int{3} } func (m *MVCCStatsLegacyRepresentation) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2289,9 +2289,9 @@ var ( ErrIntOverflowMvcc = fmt.Errorf("proto: integer overflow") ) -func init() { proto.RegisterFile("storage/enginepb/mvcc.proto", fileDescriptor_mvcc_69eca97822587899) } +func init() { proto.RegisterFile("storage/enginepb/mvcc.proto", fileDescriptor_mvcc_33f4aa682d7df09c) } -var fileDescriptor_mvcc_69eca97822587899 = []byte{ +var fileDescriptor_mvcc_33f4aa682d7df09c = []byte{ // 780 bytes of a gzipped FileDescriptorProto 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xec, 0x95, 0x31, 0x4f, 0xdb, 0x4c, 0x18, 0xc7, 0xe3, 0x37, 0x01, 0x9c, 0x4b, 0x48, 0xc0, 0x2f, 0xd2, 0x1b, 0x85, 0xb7, 0x4e, 0x0a, diff --git a/pkg/storage/enginepb/mvcc.proto b/pkg/storage/enginepb/mvcc.proto index c6cc5fd9685b..f140e65c2c2e 100644 --- a/pkg/storage/enginepb/mvcc.proto +++ b/pkg/storage/enginepb/mvcc.proto @@ -200,7 +200,7 @@ message MVCCStats { // This tracks the same quantity as (key_bytes + val_bytes), but // for system-local metadata keys (which aren't counted in either // key_bytes or val_bytes). Each of the keys falling into this group - // is documented in keys/constants.go under the localPrefix constant + // is documented in keys/constants.go under the LocalPrefix constant // and is prefixed by either LocalRangeIDPrefix or LocalRangePrefix. optional sfixed64 sys_bytes = 12 [(gogoproto.nullable) = false]; // sys_count is the number of meta keys tracked under sys_bytes. diff --git a/pkg/testutils/lint/passes/errcmp/errcmp.go b/pkg/testutils/lint/passes/errcmp/errcmp.go index 6ac0fa68dfb6..fa0105793064 100644 --- a/pkg/testutils/lint/passes/errcmp/errcmp.go +++ b/pkg/testutils/lint/passes/errcmp/errcmp.go @@ -91,11 +91,11 @@ func checkErrCast(pass *analysis.Pass, texpr *ast.TypeAssertExpr) { if pass.TypesInfo.Types[texpr.X].Type == errorType { pass.Reportf(texpr.Lparen, escNl(`invalid direct cast on error object Alternatives: - if _, ok := err.(*T); ok -> if errors.HasType(err, (*T)(nil) + if _, ok := err.(*T); ok -> if errors.HasType(err, (*T)(nil)) if _, ok := err.(I); ok -> if errors.HasInterface(err, (*I)(nil)) if myErr, ok := err.(*T); ok -> if myErr := (*T)(nil); errors.As(err, &myErr) if myErr, ok := err.(I); ok -> if myErr := (I)(nil); errors.As(err, &myErr) - switch err.(type) { case *T:... -> switch { case errors.HasType(err, (*T)(nil): ... + switch err.(type) { case *T:... -> switch { case errors.HasType(err, (*T)(nil)): ... `)) } } diff --git a/pkg/testutils/serverutils/test_cluster_shim.go b/pkg/testutils/serverutils/test_cluster_shim.go index ea334f956cfd..db2a8e062d87 100644 --- a/pkg/testutils/serverutils/test_cluster_shim.go +++ b/pkg/testutils/serverutils/test_cluster_shim.go @@ -23,6 +23,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/stop" ) @@ -129,6 +130,21 @@ type TestClusterInterface interface { rangeDesc roachpb.RangeDescriptor, dest roachpb.ReplicationTarget, ) error + // MoveRangeLeaseNonCooperatively performs a non-cooperative transfer of the + // lease for a range from whoever has it to a particular store. That store + // must already have a replica of the range. If that replica already has the + // (active) lease, this method is a no-op. + // + // The transfer is non-cooperative in that the lease is made to expire by + // advancing the manual clock. The target is then instructed to acquire the + // ownerless lease. Most tests should use the cooperative version of this + // method, TransferRangeLease. + MoveRangeLeaseNonCooperatively( + rangeDesc roachpb.RangeDescriptor, + dest roachpb.ReplicationTarget, + manual *hlc.HybridManualClock, + ) error + // LookupRange returns the descriptor of the range containing key. LookupRange(key roachpb.Key) (roachpb.RangeDescriptor, error) diff --git a/pkg/testutils/serverutils/test_server_shim.go b/pkg/testutils/serverutils/test_server_shim.go index fa047022bf69..24fd40eba088 100644 --- a/pkg/testutils/serverutils/test_server_shim.go +++ b/pkg/testutils/serverutils/test_server_shim.go @@ -128,6 +128,9 @@ type TestServerInterface interface { // interface{}. NodeLiveness() interface{} + // HeartbeatNodeLiveness heartbeats the server's NodeLiveness record. + HeartbeatNodeLiveness() error + // SetDistSQLSpanResolver changes the SpanResolver used for DistSQL inside the // server's executor. The argument must be a physicalplan.SpanResolver // instance. diff --git a/pkg/testutils/testcluster/BUILD.bazel b/pkg/testutils/testcluster/BUILD.bazel index a004bd8d5ef2..8cebf296af56 100644 --- a/pkg/testutils/testcluster/BUILD.bazel +++ b/pkg/testutils/testcluster/BUILD.bazel @@ -9,8 +9,8 @@ go_library( "//pkg/base", "//pkg/gossip", "//pkg/keys", + "//pkg/kv", "//pkg/kv/kvserver", - "//pkg/kv/kvserver/liveness", "//pkg/kv/kvserver/liveness/livenesspb", "//pkg/roachpb", "//pkg/rpc", diff --git a/pkg/testutils/testcluster/testcluster.go b/pkg/testutils/testcluster/testcluster.go index dcebe198d824..96867a3dff4d 100644 --- a/pkg/testutils/testcluster/testcluster.go +++ b/pkg/testutils/testcluster/testcluster.go @@ -23,8 +23,8 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/gossip" "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/kv/kvserver" - "github.com/cockroachdb/cockroach/pkg/kv/kvserver/liveness" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/liveness/livenesspb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/rpc" @@ -771,7 +771,7 @@ func (tc *TestCluster) RemoveNonVotersOrFatal( return desc } -// TransferRangeLease is part of the TestServerInterface. +// TransferRangeLease is part of the TestClusterInterface. func (tc *TestCluster) TransferRangeLease( rangeDesc roachpb.RangeDescriptor, dest roachpb.ReplicationTarget, ) error { @@ -792,6 +792,88 @@ func (tc *TestCluster) TransferRangeLeaseOrFatal( } } +// MoveRangeLeaseNonCooperatively is part of the TestClusterInterface. +func (tc *TestCluster) MoveRangeLeaseNonCooperatively( + rangeDesc roachpb.RangeDescriptor, dest roachpb.ReplicationTarget, manual *hlc.HybridManualClock, +) error { + knobs := tc.clusterArgs.ServerArgs.Knobs.Store.(*kvserver.StoreTestingKnobs) + if !knobs.AllowLeaseRequestProposalsWhenNotLeader { + // Without this knob, we'd have to architect a Raft leadership change + // too in order to let the replica get the lease. It's easier to just + // require that callers set it. + return errors.Errorf("must set StoreTestingKnobs.AllowLeaseRequestProposalsWhenNotLeader") + } + + destServer, err := tc.findMemberServer(dest.StoreID) + if err != nil { + return err + } + destStore, err := destServer.Stores().GetStore(dest.StoreID) + if err != nil { + return err + } + + // We are going to advance the manual clock so that the current lease + // expires and then issue a request to the target in hopes that it grabs the + // lease. But it is possible that another replica grabs the lease before us + // when it's up for grabs. To handle that case, we wrap the entire operation + // in an outer retry loop. + const retryDur = testutils.DefaultSucceedsSoonDuration + return retry.ForDuration(retryDur, func() error { + // Find the current lease. + prevLease, _, err := tc.FindRangeLease(rangeDesc, nil /* hint */) + if err != nil { + return err + } + if prevLease.Replica.StoreID == dest.StoreID { + return nil + } + + // Advance the manual clock past the lease's expiration. + lhStore, err := tc.findMemberStore(prevLease.Replica.StoreID) + if err != nil { + return err + } + manual.Increment(lhStore.GetStoreConfig().LeaseExpiration()) + + // Heartbeat the destination server's liveness record so that if we are + // attempting to acquire an epoch-based lease, the server will be live. + err = destServer.HeartbeatNodeLiveness() + if err != nil { + return err + } + + // Issue a request to the target replica, which should notice that the + // old lease has expired and that it can acquire the lease. + var newLease *roachpb.Lease + ctx := context.Background() + req := &roachpb.LeaseInfoRequest{ + RequestHeader: roachpb.RequestHeader{ + Key: rangeDesc.StartKey.AsRawKey(), + }, + } + h := roachpb.Header{RangeID: rangeDesc.RangeID} + reply, pErr := kv.SendWrappedWith(ctx, destStore, h, req) + if pErr != nil { + log.Infof(ctx, "LeaseInfoRequest failed: %v", pErr) + if lErr, ok := pErr.GetDetail().(*roachpb.NotLeaseHolderError); ok && lErr.Lease != nil { + newLease = lErr.Lease + } else { + return pErr.GoError() + } + } else { + newLease = &reply.(*roachpb.LeaseInfoResponse).Lease + } + + // Is the lease in the right place? + if newLease.Replica.StoreID != dest.StoreID { + return errors.Errorf("LeaseInfoRequest succeeded, "+ + "but lease in wrong location, want %v, got %v", dest, newLease.Replica) + } + return nil + }) +} + // FindRangeLease is similar to FindRangeLeaseHolder but returns a Lease proto // without verifying if the lease is still active. Instead, it returns a time- // stamp taken off the queried node's clock. @@ -812,15 +894,9 @@ func (tc *TestCluster) FindRangeLease( // Find the server indicated by the hint and send a LeaseInfoRequest through // it. - var hintServer *server.TestServer - for _, s := range tc.Servers { - if s.GetNode().Descriptor.NodeID == hint.NodeID { - hintServer = s - break - } - } - if hintServer == nil { - return roachpb.Lease{}, hlc.ClockTimestamp{}, errors.Errorf("bad hint: %+v; no such node", hint) + hintServer, err := tc.findMemberServer(hint.StoreID) + if err != nil { + return roachpb.Lease{}, hlc.ClockTimestamp{}, errors.Wrapf(err, "bad hint: %+v; no such node", hint) } return hintServer.GetRangeLease(context.TODO(), rangeDesc.StartKey.AsRawKey()) @@ -910,20 +986,25 @@ func (tc *TestCluster) WaitForSplitAndInitialization(startKey roachpb.Key) error }) } -// findMemberStore returns the store containing a given replica. -func (tc *TestCluster) findMemberStore(storeID roachpb.StoreID) (*kvserver.Store, error) { +// findMemberServer returns the server containing a given store. +func (tc *TestCluster) findMemberServer(storeID roachpb.StoreID) (*server.TestServer, error) { for _, server := range tc.Servers { if server.Stores().HasStore(storeID) { - store, err := server.Stores().GetStore(storeID) - if err != nil { - return nil, err - } - return store, nil + return server, nil } } return nil, errors.Errorf("store not found") } +// findMemberStore returns the store with a given ID. +func (tc *TestCluster) findMemberStore(storeID roachpb.StoreID) (*kvserver.Store, error) { + server, err := tc.findMemberServer(storeID) + if err != nil { + return nil, err + } + return server.Stores().GetStore(storeID) +} + // WaitForFullReplication waits until all stores in the cluster // have no ranges with replication pending. // @@ -1263,22 +1344,6 @@ func (tc *TestCluster) GetStatusClient( return serverpb.NewStatusClient(cc) } -// HeartbeatLiveness sends a liveness heartbeat on a particular store. -func (tc *TestCluster) HeartbeatLiveness(ctx context.Context, storeIdx int) error { - nl := tc.Servers[storeIdx].NodeLiveness().(*liveness.NodeLiveness) - l, ok := nl.Self() - if !ok { - return errors.New("liveness not found") - } - var err error - for r := retry.StartWithCtx(ctx, retry.Options{MaxRetries: 5}); r.Next(); { - if err = nl.Heartbeat(ctx, l); !errors.Is(err, liveness.ErrEpochIncremented) { - break - } - } - return err -} - type testClusterFactoryImpl struct{} // TestClusterFactory can be passed to serverutils.InitTestClusterFactory diff --git a/pkg/ts/catalog/chart_catalog.go b/pkg/ts/catalog/chart_catalog.go index e81826278a8b..8f805fa50039 100644 --- a/pkg/ts/catalog/chart_catalog.go +++ b/pkg/ts/catalog/chart_catalog.go @@ -348,6 +348,7 @@ var charts = []sectionDescription{ "distsender.rpc.err.integeroverflowerrtype", "distsender.rpc.err.intentmissingerrtype", "distsender.rpc.err.internalerrtype", + "distsender.rpc.err.invalidleaseerrtype", "distsender.rpc.err.leaserejectederrtype", "distsender.rpc.err.mergeinprogresserrtype", "distsender.rpc.err.nodeunavailableerrtype", diff --git a/pkg/util/hlc/timestamp.go b/pkg/util/hlc/timestamp.go index c96f7b45c5a2..8a92f35190f2 100644 --- a/pkg/util/hlc/timestamp.go +++ b/pkg/util/hlc/timestamp.go @@ -374,6 +374,9 @@ func (t ClockTimestamp) Less(s ClockTimestamp) bool { return Timestamp(t).Less(T // LessEq returns whether the receiver is less than or equal to the parameter. func (t ClockTimestamp) LessEq(s ClockTimestamp) bool { return Timestamp(t).LessEq(Timestamp(s)) } +// Ignore unused warnings. +var _ = ClockTimestamp.LessEq + // String implements the fmt.Stringer interface. func (t ClockTimestamp) String() string { return t.ToTimestamp().String() }