From 2a0ea0939d0ba0196a9aba787841fa354591a8aa Mon Sep 17 00:00:00 2001 From: sumeerbhola Date: Fri, 8 Jan 2021 15:03:01 -0500 Subject: [PATCH] concurrency,kvserver: limited scans optimistically check for locks MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit This optimistic checking happens after the evaluation. The evaluation will already discover any conflicting intents, so the subsequent checking of the lock table is primarily to find conflicting unreplicated locks. This structure should be easy to extend to optimistic latching. Benchmark numbers from the new kv roachtest that does SFU to introduce false contention: name old ops/sec new ops/sec delta kv95limited-spans/enc=false/nodes=1/splt=0/seq 5.68k ± 0% 9.96k ± 1% +75.46% (p=0.000 n=8+9) name old p50 new p50 delta kv95limited-spans/enc=false/nodes=1/splt=0/seq 13.1 ± 0% 5.5 ± 0% -58.02% (p=0.000 n=8+8) name old p95 new p95 delta kv95limited-spans/enc=false/nodes=1/splt=0/seq 18.9 ± 0% 16.8 ± 0% -11.11% (p=0.001 n=8+9) name old p99 new p99 delta kv95limited-spans/enc=false/nodes=1/splt=0/seq 22.0 ± 0% 25.6 ± 2% +16.57% (p=0.000 n=8+9) Fixes #49973 Informs #9521 Release note: None --- pkg/cmd/roachtest/kv.go | 50 +- pkg/kv/kvserver/client_replica_test.go | 199 +++++ .../concurrency/concurrency_control.go | 57 +- .../concurrency/concurrency_manager.go | 80 +- pkg/kv/kvserver/concurrency/lock_table.go | 110 ++- .../concurrency/lock_table_waiter_test.go | 5 + pkg/kv/kvserver/replica_read.go | 87 ++- pkg/kv/kvserver/replica_send.go | 118 ++- pkg/roachpb/batch_generated.go | 4 + pkg/roachpb/errors.go | 22 + pkg/roachpb/errors.pb.go | 684 ++++++++++++------ pkg/roachpb/errors.proto | 8 + pkg/workload/kv/kv.go | 100 ++- 13 files changed, 1221 insertions(+), 303 deletions(-) diff --git a/pkg/cmd/roachtest/kv.go b/pkg/cmd/roachtest/kv.go index 51bad877cd83..aca539f3a7c7 100644 --- a/pkg/cmd/roachtest/kv.go +++ b/pkg/cmd/roachtest/kv.go @@ -35,13 +35,17 @@ func registerKV(r *testRegistry) { nodes int cpus int readPercent int - batchSize int - blockSize int - splits int // 0 implies default, negative implies 0 - encryption bool - sequential bool - duration time.Duration - tags []string + // If true, the reads are limited reads over the full span of the table. + // Currently this also enables SFU writes on the workload since this is + // geared towards testing optimistic locking and latching. + spanReads bool + batchSize int + blockSize int + splits int // 0 implies default, negative implies 0 + encryption bool + sequential bool + duration time.Duration + tags []string } computeNumSplits := func(opts kvOptions) int { // TODO(ajwerner): set this default to a more sane value or remove it and @@ -62,6 +66,16 @@ func registerKV(r *testRegistry) { c.Put(ctx, workload, "./workload", c.Node(nodes+1)) c.Start(ctx, t, c.Range(1, nodes), startArgs(fmt.Sprintf("--encrypt=%t", opts.encryption))) + if opts.splits < 0 { + // In addition to telling the workload to not split, disable load-based + // splitting. + db := c.Conn(ctx, 1) + defer db.Close() + if _, err := db.ExecContext(ctx, "SET CLUSTER SETTING kv.range_split.by_load_enabled = 'false'"); err != nil { + t.Fatalf("failed to disable load based splitting: %v", err) + } + } + t.Status("running workload") m := newMonitor(ctx, c, c.Range(1, nodes)) m.Go(func(ctx context.Context) error { @@ -72,7 +86,17 @@ func registerKV(r *testRegistry) { opts.duration = 10 * time.Minute } duration := " --duration=" + ifLocal("10s", opts.duration.String()) - readPercent := fmt.Sprintf(" --read-percent=%d", opts.readPercent) + var readPercent string + if opts.spanReads { + // SFU makes sense only if we repeat writes to the same key. Here + // we've arbitrarily picked a cycle-length of 1000, so 1 in 1000 + // writes will contend with the limited scan wrt locking. + readPercent = + fmt.Sprintf(" --span-percent=%d --span-limit=1 --sfu-writes=true --cycle-length=1000", + opts.readPercent) + } else { + readPercent = fmt.Sprintf(" --read-percent=%d", opts.readPercent) + } histograms := " --histograms=" + perfArtifactsDir + "/stats.json" var batchSize string if opts.batchSize > 0 { @@ -143,6 +167,10 @@ func registerKV(r *testRegistry) { {nodes: 3, cpus: 32, readPercent: 0, sequential: true}, {nodes: 3, cpus: 32, readPercent: 95, sequential: true}, + // Configs with reads, that are of limited spans, along with SFU writes. + {nodes: 1, cpus: 8, readPercent: 95, spanReads: true, splits: -1 /* no splits */, sequential: true}, + {nodes: 1, cpus: 32, readPercent: 95, spanReads: true, splits: -1 /* no splits */, sequential: true}, + // Weekly larger scale configurations. {nodes: 32, cpus: 8, readPercent: 0, tags: []string{"weekly"}, duration: time.Hour}, {nodes: 32, cpus: 8, readPercent: 95, tags: []string{"weekly"}, duration: time.Hour}, @@ -150,7 +178,11 @@ func registerKV(r *testRegistry) { opts := opts var nameParts []string - nameParts = append(nameParts, fmt.Sprintf("kv%d", opts.readPercent)) + var limitedSpanStr string + if opts.spanReads { + limitedSpanStr = "limited-spans" + } + nameParts = append(nameParts, fmt.Sprintf("kv%d%s", opts.readPercent, limitedSpanStr)) if len(opts.tags) > 0 { nameParts = append(nameParts, strings.Join(opts.tags, "/")) } diff --git a/pkg/kv/kvserver/client_replica_test.go b/pkg/kv/kvserver/client_replica_test.go index b7e9e7481365..6236bbc70c45 100644 --- a/pkg/kv/kvserver/client_replica_test.go +++ b/pkg/kv/kvserver/client_replica_test.go @@ -54,6 +54,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/retry" "github.com/cockroachdb/cockroach/pkg/util/stop" "github.com/cockroachdb/cockroach/pkg/util/syncutil" + "github.com/cockroachdb/cockroach/pkg/util/timeutil" "github.com/cockroachdb/cockroach/pkg/util/uuid" "github.com/cockroachdb/errors" "github.com/kr/pretty" @@ -3779,3 +3780,201 @@ func TestRaftSchedulerPrioritizesNodeLiveness(t *testing.T) { priorityID := store.RaftSchedulerPriorityID() require.Equal(t, livenessRangeID, priorityID) } + +func TestOptimisticEvalRetry(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + ctx := context.Background() + args := base.TestServerArgs{} + s, _, db := serverutils.StartServer(t, args) + defer s.Stopper().Stop(ctx) + + require.NoError(t, db.Txn(ctx, func(ctx context.Context, txn *kv.Txn) (err error) { + defer func() { + t.Log(err) + }() + if err := txn.Put(ctx, "a", "a"); err != nil { + return err + } + if err := txn.Put(ctx, "b", "b"); err != nil { + return err + } + return txn.Commit(ctx) + })) + time.Sleep(10 * time.Millisecond) + tup, err := db.Get(ctx, "a") + require.NoError(t, err) + require.NotNil(t, tup.Value) + tup, err = db.Get(ctx, "b") + require.NoError(t, err) + require.NotNil(t, tup.Value) + + txn1 := db.NewTxn(ctx, "locking txn") + _, err = txn1.ScanForUpdate(ctx, "a", "c", 0) + require.NoError(t, err) + + time.Sleep(10 * time.Millisecond) + readDone := make(chan error) + go func() { + readDone <- db.Txn(ctx, func(ctx context.Context, txn *kv.Txn) (err error) { + defer func() { + t.Log(err) + }() + _, err = txn.Scan(ctx, "a", "c", 1) + if err != nil { + return err + } + return txn.Commit(ctx) + }) + }() + removedLocks := false + timer := timeutil.NewTimer() + timer.Reset(time.Second * 2) + defer timer.Stop() + done := false + for !done { + select { + case err := <-readDone: + require.NoError(t, err) + require.True(t, removedLocks) + done = true + case <-timer.C: + require.NoError(t, txn1.Commit(ctx)) + removedLocks = true + } + } +} + +func TestOptimisticEvalNoContention(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + ctx := context.Background() + args := base.TestServerArgs{} + s, _, db := serverutils.StartServer(t, args) + defer s.Stopper().Stop(ctx) + + require.NoError(t, db.Txn(ctx, func(ctx context.Context, txn *kv.Txn) (err error) { + defer func() { + t.Log(err) + }() + if err := txn.Put(ctx, "a", "a"); err != nil { + return err + } + if err := txn.Put(ctx, "b", "b"); err != nil { + return err + } + return txn.Commit(ctx) + })) + time.Sleep(10 * time.Millisecond) + tup, err := db.Get(ctx, "a") + require.NoError(t, err) + require.NotNil(t, tup.Value) + tup, err = db.Get(ctx, "b") + require.NoError(t, err) + require.NotNil(t, tup.Value) + + txn1 := db.NewTxn(ctx, "locking txn") + _, err = txn1.ScanForUpdate(ctx, "b", "c", 0) + require.NoError(t, err) + + time.Sleep(10 * time.Millisecond) + readDone := make(chan error) + go func() { + readDone <- db.Txn(ctx, func(ctx context.Context, txn *kv.Txn) (err error) { + defer func() { + t.Log(err) + }() + // There is no contention when doing optimistic evaluation, since it can read a + // which is not locked. + _, err = txn.Scan(ctx, "a", "c", 1) + if err != nil { + return err + } + return txn.Commit(ctx) + }) + }() + err = <-readDone + require.NoError(t, err) + require.NoError(t, txn1.Commit(ctx)) +} + +func BenchmarkOptimisticEval(b *testing.B) { + defer log.Scope(b).Close(b) + ctx := context.Background() + args := base.TestServerArgs{} + s, _, db := serverutils.StartServer(b, args) + defer s.Stopper().Stop(ctx) + + require.NoError(b, db.Txn(ctx, func(ctx context.Context, txn *kv.Txn) (err error) { + defer func() { + b.Log(err) + }() + if err := txn.Put(ctx, "a", "a"); err != nil { + return err + } + if err := txn.Put(ctx, "b", "b"); err != nil { + return err + } + return txn.Commit(ctx) + })) + time.Sleep(10 * time.Millisecond) + tup, err := db.Get(ctx, "a") + require.NoError(b, err) + require.NotNil(b, tup.Value) + tup, err = db.Get(ctx, "b") + require.NoError(b, err) + require.NotNil(b, tup.Value) + + for _, realContention := range []bool{false, true} { + b.Run(fmt.Sprintf("real-contention=%t", realContention), + func(b *testing.B) { + lockStart := "b" + if realContention { + lockStart = "a" + } + finishWrites := make(chan struct{}) + var writers sync.WaitGroup + for i := 0; i < 1; i++ { + writers.Add(1) + go func() { + for { + txn := db.NewTxn(ctx, "locking txn") + _, err = txn.ScanForUpdate(ctx, lockStart, "c", 0) + require.NoError(b, err) + time.Sleep(5 * time.Millisecond) + // Normally, it would do a write here, but we don't bother. + require.NoError(b, txn.Commit(ctx)) + select { + case _, recv := <-finishWrites: + if !recv { + writers.Done() + return + } + default: + } + } + }() + } + time.Sleep(10 * time.Millisecond) + b.ResetTimer() + for i := 0; i < b.N; i++ { + db.Txn(ctx, func(ctx context.Context, txn *kv.Txn) (err error) { + _, err = txn.Scan(ctx, "a", "c", 1) + if err != nil { + panic(err) + } + err = txn.Commit(ctx) + if err != nil { + panic(err) + } + return err + }) + } + b.StopTimer() + close(finishWrites) + writers.Wait() + }) + } +} diff --git a/pkg/kv/kvserver/concurrency/concurrency_control.go b/pkg/kv/kvserver/concurrency/concurrency_control.go index acbf453f7492..26282d537028 100644 --- a/pkg/kv/kvserver/concurrency/concurrency_control.go +++ b/pkg/kv/kvserver/concurrency/concurrency_control.go @@ -162,13 +162,20 @@ type RequestSequencer interface { // request is guaranteed sufficient isolation for the duration of its // evaluation, until the returned request guard is released. // NOTE: this last part will not be true until replicated locks are pulled - // into the concurrency manager. + // into the concurrency manager. This is the normal behavior for a request + // marked as PessimisticEval. For OptimisticEval, it can optimize by not + // acquiring locks, and the request must call + // Guard.CheckOptimisticNoConflicts after evaluation. For + // PessimisticAfterFailedOptimisticEval, latches are already held. + // TODO(sumeer): change OptimisticEval to only queue the latches and not + // wait for them, so PessimisticAfterFailedOptimisticEval will wait for them. // // An optional existing request guard can be provided to SequenceReq. This // allows the request's position in lock wait-queues to be retained across // sequencing attempts. If provided, the guard should not be holding latches - // already. The expected usage of this parameter is that it will only be - // provided after acquiring a Guard from a ContentionHandler method. + // already for PessimisticEval. The expected usage of this parameter is that + // it will only be provided after acquiring a Guard from a ContentionHandler + // method (for non-OptimisticEval). // // If the method returns a non-nil request guard then the caller must ensure // that the guard is eventually released by passing it to FinishReq. @@ -308,6 +315,27 @@ type MetricExporter interface { // External API Type Definitions // /////////////////////////////////// +// RequestEvalKind informs the manager of the evaluation kind for the current +// evaluation attempt. Optimistic evaluation is used for requests involving +// limited scans, where the checking of locks and latches may be (partially) +// postponed until after evaluation, using Guard.CheckOptimisticNoConflicts. +// Note that intents (replicated single-key locks) will still be observed +// during evaluation. +// +// The setting can change across different calls to SequenceReq. The current +// sequence of possibilities is: +// - OptimisticEval: when optimistic evaluation succeeds. +// - OptimisticEval, PessimisticAfterFailedOptimisticEval, PessimisticEval*: +// when optimistic evaluation failed. +// - PessimisticEval+: when only pessimistic evaluation was attempted. +type RequestEvalKind int + +const ( + PessimisticEval RequestEvalKind = iota + OptimisticEval + PessimisticAfterFailedOptimisticEval +) + // Request is the input to Manager.SequenceReq. The struct contains all of the // information necessary to sequence a KV request and determine which locks and // other in-flight requests it conflicts with. @@ -351,6 +379,13 @@ type Request struct { // (Txn == nil), all reads and writes are considered to take place at // Timestamp. LockSpans *spanset.SpanSet + + // EvalKinds represents the evaluation kind for the current evaluation + // attempt of the request. + // TODO(sumeer): Move this into Guard. Confirm that the Request object + // passed to SequenceReq should not change across calls since we stash + // the first one into Guard. + EvalKind RequestEvalKind } // Guard is returned from Manager.SequenceReq. The guard is passed back in to @@ -460,6 +495,13 @@ type lockTable interface { // function. ScanAndEnqueue(Request, lockTableGuard) lockTableGuard + // ScanOptimistic takes a snapshot of the lock table for later checking for + // conflicts, and returns a guard. It is for optimistic evaluation of + // requests that will typically scan a small subset of the spans mentioned + // in the Request. After Request evaluation, CheckOptimisticNoConflicts + // must be called on the guard. + ScanOptimistic(Request) lockTableGuard + // Dequeue removes the request from its lock wait-queues. It should be // called when the request is finished, whether it evaluated or not. The // guard should not be used after being dequeued. @@ -599,6 +641,15 @@ type lockTableGuard interface { // This must be called after the waiting state has transitioned to // doneWaiting. ResolveBeforeScanning() []roachpb.LockUpdate + + // CheckOptimisticNoConflicts uses the SpanSet representing the spans that + // were actually read, to check for conflicting locks, after an optimistic + // evaluation. It returns true if there were no conflicts. See + // lockTable.ScanOptimistic for context. Note that the evaluation has + // already seen any intents (replicated single-key locks) that conflicted, + // so this checking is practically only going to find unreplicated locks + // that conflict. + CheckOptimisticNoConflicts(*spanset.SpanSet) (ok bool) } // lockTableWaiter is concerned with waiting in lock wait-queues for locks held diff --git a/pkg/kv/kvserver/concurrency/concurrency_manager.go b/pkg/kv/kvserver/concurrency/concurrency_manager.go index f6218c7db179..17556fe2c4fc 100644 --- a/pkg/kv/kvserver/concurrency/concurrency_manager.go +++ b/pkg/kv/kvserver/concurrency/concurrency_manager.go @@ -112,16 +112,39 @@ func (m *managerImpl) SequenceReq( ctx context.Context, prev *Guard, req Request, ) (*Guard, Response, *Error) { var g *Guard + holdsLatches := false if prev == nil { + switch req.EvalKind { + case PessimisticEval: + log.Event(ctx, "sequencing request") + case OptimisticEval: + log.Event(ctx, "optimistically sequencing request") + case PessimisticAfterFailedOptimisticEval: + panic("retry should have non-nil guard") + } g = newGuard(req) - log.Event(ctx, "sequencing request") } else { g = prev - g.AssertNoLatches() - log.Event(ctx, "re-sequencing request") + // TODO(sumeer): remove this hack. + g.Req.EvalKind = req.EvalKind + switch req.EvalKind { + case PessimisticEval: + g.AssertNoLatches() + log.Event(ctx, "re-sequencing request") + case OptimisticEval: + panic("optimistic eval cannot happen when re-sequencing") + case PessimisticAfterFailedOptimisticEval: + if shouldAcquireLatches(req) { + g.AssertLatches() + holdsLatches = true + } + log.Event(ctx, "re-sequencing request after optimistic sequencing failed") + } + if g.Req.EvalKind != req.EvalKind { + panic("EvalKinds are not equal") + } } - - resp, err := m.sequenceReqWithGuard(ctx, g, req) + resp, err := m.sequenceReqWithGuard(ctx, g, req, holdsLatches) if resp != nil || err != nil { // Ensure that we release the guard if we return a response or an error. m.FinishReq(g) @@ -130,8 +153,10 @@ func (m *managerImpl) SequenceReq( return g, nil, nil } +// TODO(sumeer): we are using both g.Req and req, when the former should +// suffice. Remove the req parameter. func (m *managerImpl) sequenceReqWithGuard( - ctx context.Context, g *Guard, req Request, + ctx context.Context, g *Guard, req Request, holdsLatches bool, ) (Response, *Error) { // Some requests don't need to acquire latches at all. if !shouldAcquireLatches(req) { @@ -148,22 +173,38 @@ func (m *managerImpl) sequenceReqWithGuard( } for { - // Acquire latches for the request. This synchronizes the request - // with all conflicting in-flight requests. - log.Event(ctx, "acquiring latches") - g.lg, err = m.lm.Acquire(ctx, req) - if err != nil { - return nil, err + if !holdsLatches { + // TODO(sumeer): optimistic requests could register their need for + // latches, but not actually wait until acquisition. + // https://github.com/cockroachdb/cockroach/issues/9521 + + // Acquire latches for the request. This synchronizes the request + // with all conflicting in-flight requests. + log.Event(ctx, "acquiring latches") + g.lg, err = m.lm.Acquire(ctx, req) + if err != nil { + return nil, err + } } + // For subsequent iterations. + holdsLatches = false // Some requests don't want the wait on locks. if req.LockSpans.Empty() { return nil, nil } - // Scan for conflicting locks. - log.Event(ctx, "scanning lock table for conflicting locks") - g.ltg = m.lt.ScanAndEnqueue(g.Req, g.ltg) + if req.EvalKind == OptimisticEval { + if g.ltg != nil { + panic("Optimistic locking should not have a non-nil lockTableGuard") + } + log.Event(ctx, "scanning lock table for conflicting locks") + g.ltg = m.lt.ScanOptimistic(g.Req) + } else { + // Scan for conflicting locks. + log.Event(ctx, "scanning lock table for conflicting locks") + g.ltg = m.lt.ScanAndEnqueue(g.Req, g.ltg) + } // Wait on conflicting locks, if necessary. if g.ltg.ShouldWait() { @@ -480,6 +521,15 @@ func (g *Guard) AssertNoLatches() { } } +// CheckOptimisticNoConflicts checks that the lockSpansRead do not have a +// conflicting lock. +func (g *Guard) CheckOptimisticNoConflicts(lockSpansRead *spanset.SpanSet) (ok bool) { + if g.ltg == nil { + return true + } + return g.ltg.CheckOptimisticNoConflicts(lockSpansRead) +} + func (g *Guard) moveLatchGuard() latchGuard { lg := g.lg g.lg = nil diff --git a/pkg/kv/kvserver/concurrency/lock_table.go b/pkg/kv/kvserver/concurrency/lock_table.go index e587234ba404..747c5a9cafaa 100644 --- a/pkg/kv/kvserver/concurrency/lock_table.go +++ b/pkg/kv/kvserver/concurrency/lock_table.go @@ -420,6 +420,30 @@ func (g *lockTableGuardImpl) CurState() waitingState { return g.mu.state } +func (g *lockTableGuardImpl) CheckOptimisticNoConflicts(spanSet *spanset.SpanSet) (ok bool) { + // Temporarily replace the SpanSet in the guard. + originalSpanSet := g.spans + g.spans = spanSet + defer func() { + g.spans = originalSpanSet + }() + span := stepToNextSpan(g) + for span != nil { + startKey := span.Key + tree := g.tableSnapshot[g.ss] + iter := tree.MakeIter() + ltRange := &lockState{key: startKey, endKey: span.EndKey} + for iter.FirstOverlap(ltRange); iter.Valid(); iter.NextOverlap(ltRange) { + l := iter.Cur() + if !l.isNonConflictingLock(g, g.sa) { + return false + } + } + span = stepToNextSpan(g) + } + return true +} + func (g *lockTableGuardImpl) notify() { select { case g.mu.signal <- struct{}{}: @@ -1357,6 +1381,45 @@ func (l *lockState) tryActiveWait( return true, false } +func (l *lockState) isNonConflictingLock(g *lockTableGuardImpl, sa spanset.SpanAccess) bool { + l.mu.Lock() + defer l.mu.Unlock() + + // It is possible that this lock is empty and has not yet been deleted. + if l.isEmptyLock() { + return true + } + // Lock is not empty. + lockHolderTxn, lockHolderTS := l.getLockHolder() + if lockHolderTxn == nil { + // Reservation holders are non-conflicting. + // + // When optimistic evaluation holds latches, there cannot be a conflicting + // reservation holder that is also holding latches (reservation holder + // without latches can happen due to AddDiscoveredLock). So when this + // optimistic evaluation succeeds and releases latches the reservation + // holder will need to acquire latches and scan the lock table again. When + // optimistic evaluation does not hold latches, it will check for + // conflicting latches before declaring success and a reservation holder + // that holds latches will be discovered, and the optimistic evaluation + // will retry as pessimistic. + return true + } + if g.isSameTxn(lockHolderTxn) { + // Already locked by this txn. + return true + } + // NB: We do not look at the finalizedTxnCache in this optimistic evaluation + // path. A conflict with a finalized txn will be noticed when retrying + // pessimistically. + + if sa == spanset.SpanReadOnly && g.readTS.Less(lockHolderTS) { + return true + } + // Conflicts. + return false +} + // Acquires this lock. Returns the list of guards that are done actively // waiting at this key -- these will be requests from the same transaction // that is acquiring the lock. @@ -1957,6 +2020,12 @@ func (t *treeMu) nextLockSeqNum() uint64 { return t.lockIDSeqNum } +func (t *lockTableImpl) ScanOptimistic(req Request) lockTableGuard { + g := t.newGuardForReq(req) + t.doSnapshotForGuard(g) + return g +} + // ScanAndEnqueue implements the lockTable interface. func (t *lockTableImpl) ScanAndEnqueue(req Request, guard lockTableGuard) lockTableGuard { // NOTE: there is no need to synchronize with enabledMu here. ScanAndEnqueue @@ -1967,15 +2036,7 @@ func (t *lockTableImpl) ScanAndEnqueue(req Request, guard lockTableGuard) lockTa var g *lockTableGuardImpl if guard == nil { - g = newLockTableGuardImpl() - g.seqNum = atomic.AddUint64(&t.seqNum, 1) - g.lt = t - g.txn = req.txnMeta() - g.spans = req.LockSpans - g.readTS = req.readConflictTimestamp() - g.writeTS = req.writeConflictTimestamp() - g.sa = spanset.NumSpanAccess - 1 - g.index = -1 + g = t.newGuardForReq(req) } else { g = guard.(*lockTableGuardImpl) g.key = nil @@ -1988,6 +2049,25 @@ func (t *lockTableImpl) ScanAndEnqueue(req Request, guard lockTableGuard) lockTa g.mu.Unlock() g.toResolve = g.toResolve[:0] } + t.doSnapshotForGuard(g) + g.findNextLockAfter(true /* notify */) + return g +} + +func (t *lockTableImpl) newGuardForReq(req Request) *lockTableGuardImpl { + g := newLockTableGuardImpl() + g.seqNum = atomic.AddUint64(&t.seqNum, 1) + g.lt = t + g.txn = req.txnMeta() + g.spans = req.LockSpans + g.readTS = req.readConflictTimestamp() + g.writeTS = req.writeConflictTimestamp() + g.sa = spanset.NumSpanAccess - 1 + g.index = -1 + return g +} + +func (t *lockTableImpl) doSnapshotForGuard(g *lockTableGuardImpl) { for ss := spanset.SpanScope(0); ss < spanset.NumSpanScope; ss++ { for sa := spanset.SpanAccess(0); sa < spanset.NumSpanAccess; sa++ { if len(g.spans.GetSpans(sa, ss)) > 0 { @@ -2003,8 +2083,6 @@ func (t *lockTableImpl) ScanAndEnqueue(req Request, guard lockTableGuard) lockTa } } } - g.findNextLockAfter(true /* notify */) - return g } // Dequeue implements the lockTable interface. @@ -2111,7 +2189,12 @@ func (t *lockTableImpl) AcquireLock( iter.FirstOverlap(&lockState{key: key}) if !iter.Valid() { if durability == lock.Replicated { - // Don't remember uncontended replicated locks. + // Don't remember uncontended replicated locks. The downside is that + // sometimes contention won't be noticed until when the request + // evaluates. Remembering here would be better, but our behavior when + // running into the maxLocks limit is very crude, and involves removing + // all locks from the data-structure. Treating the data-structure as a + // bounded cache with eviction would be better. tree.mu.Unlock() return nil } @@ -2128,6 +2211,9 @@ func (t *lockTableImpl) AcquireLock( // case where the lock is initially added as replicated, we drop // replicated locks from the lockTable when being upgraded from // Unreplicated to Replicated, whenever possible. + // TODO(sumeer): now that limited scans evaluate optimistically, we + // should consider removing this hack. But see the comment in the + // preceding block about maxLocks. tree.Delete(l) tree.mu.Unlock() atomic.AddInt64(&tree.numLocks, -1) diff --git a/pkg/kv/kvserver/concurrency/lock_table_waiter_test.go b/pkg/kv/kvserver/concurrency/lock_table_waiter_test.go index 2ae9ee912137..5af4771caaae 100644 --- a/pkg/kv/kvserver/concurrency/lock_table_waiter_test.go +++ b/pkg/kv/kvserver/concurrency/lock_table_waiter_test.go @@ -63,6 +63,8 @@ type mockLockTableGuard struct { toResolve []roachpb.LockUpdate } +var _ lockTableGuard = &mockLockTableGuard{} + // mockLockTableGuard implements the lockTableGuard interface. func (g *mockLockTableGuard) ShouldWait() bool { return true } func (g *mockLockTableGuard) NewStateChan() chan struct{} { return g.signal } @@ -76,6 +78,9 @@ func (g *mockLockTableGuard) CurState() waitingState { func (g *mockLockTableGuard) ResolveBeforeScanning() []roachpb.LockUpdate { return g.toResolve } +func (g *mockLockTableGuard) CheckOptimisticNoConflicts(*spanset.SpanSet) (ok bool) { + return true +} func (g *mockLockTableGuard) notify() { g.signal <- struct{}{} } // mockLockTable overrides TransactionIsFinalized, which is the only LockTable diff --git a/pkg/kv/kvserver/replica_read.go b/pkg/kv/kvserver/replica_read.go index 03984471f1ce..7b71f5f0e863 100644 --- a/pkg/kv/kvserver/replica_read.go +++ b/pkg/kv/kvserver/replica_read.go @@ -66,17 +66,36 @@ func (r *Replica) executeReadOnlyBatch( // turn means that we can bump the timestamp cache *before* evaluation // without risk of starving writes. Once we start doing that, we're free to // release latches immediately after we acquire an engine iterator as long - // as we're performing a non-locking read. + // as we're performing a non-locking read. Note that this also requires that + // the request is not being optimistically evaluated (optimistic evaluation + // does not check locks). It would also be nice, but not required for + // correctness, that the read-only engine eagerly create an iterator (that + // is later cloned) while the latches are held, so that this request does + // not "see" the effect of any later requests that happen after the latches + // are released. var result result.Result br, result, pErr = r.executeReadOnlyBatchWithServersideRefreshes(ctx, rw, rec, ba, spans) // If the request hit a server-side concurrency retry error, immediately - // proagate the error. Don't assume ownership of the concurrency guard. + // propagate the error. Don't assume ownership of the concurrency guard. if isConcurrencyRetryError(pErr) { return nil, g, pErr } + if pErr == nil && g.Req.EvalKind == concurrency.OptimisticEval { + // Gather the spans that were read. For now we ignore the latch spans, but + // when we stop waiting for latches in optimistic evaluation we will use + // these to check latches first. + _, lockSpansRead, err := r.collectSpansRead(ba, br) + if err != nil { + return nil, g, roachpb.NewError(err) + } + if ok := g.CheckOptimisticNoConflicts(lockSpansRead); !ok { + return nil, g, roachpb.NewError(roachpb.NewOptimisticEvalConflictsError()) + } + } + // Handle any local (leaseholder-only) side-effects of the request. intents := result.Local.DetachEncounteredIntents() if pErr == nil { @@ -84,6 +103,13 @@ func (r *Replica) executeReadOnlyBatch( } // Otherwise, update the timestamp cache and release the concurrency guard. + // Note: + // - The update to the timestamp cache is not gated on pErr == nil, + // since certain semantic errors (e.g. ConditionFailedError on CPut) + // require updating the timestamp cache (see updatesTSCacheOnErr). + // - For optimistic evaluation, used for limited scans, the update to the + // timestamp cache limits itself to the spans that were read, by using + // the ResumeSpans. ec, g := endCmds{repl: r, g: g}, nil ec.done(ctx, ba, br, pErr) @@ -186,3 +212,60 @@ func (r *Replica) handleReadOnlyLocalEvalResult( } return nil } + +func (r *Replica) collectSpansRead( + ba *roachpb.BatchRequest, br *roachpb.BatchResponse, +) (latchSpans, lockSpans *spanset.SpanSet, _ error) { + baCopy := *ba + baCopy.Requests = make([]roachpb.RequestUnion, len(baCopy.Requests)) + j := 0 + for i := 0; i < len(baCopy.Requests); i++ { + baReq := ba.Requests[i] + req := baReq.GetInner() + header := req.Header() + + resp := br.Responses[i].GetInner() + if resp.Header().ResumeSpan == nil { + // Fully evaluated. + baCopy.Requests[j] = baReq + j++ + continue + } + + switch t := resp.(type) { + case *roachpb.ScanResponse: + if header.Key.Equal(t.ResumeSpan.Key) { + // The request did not evaluate. Ignore it. + continue + } + header.EndKey = t.ResumeSpan.Key + case *roachpb.ReverseScanResponse: + if header.EndKey.Equal(t.ResumeSpan.EndKey) { + // The request did not evaluate. Ignore it. + continue + } + // The scan will resume at the exclusive ResumeSpan.EndKey and proceed + // towards the current header.Key. So ResumeSpan.EndKey has been read + // and becomes the inclusive start key of what was read. + header.Key = t.ResumeSpan.EndKey + default: + // Consider it fully evaluated, which is safe. + baCopy.Requests[j] = baReq + j++ + continue + } + // The ResumeSpan has changed the header. + req = req.ShallowCopy() + req.SetHeader(header) + baReq.MustSetInner(req) + baCopy.Requests[j] = baReq + j++ + } + baCopy.Requests = baCopy.Requests[:j] + + // Collect the batch's declared spans again, this time with the + // span bounds constrained to what was read. + var err error + latchSpans, lockSpans, _, err = r.collectSpans(&baCopy) + return latchSpans, lockSpans, err +} diff --git a/pkg/kv/kvserver/replica_send.go b/pkg/kv/kvserver/replica_send.go index 3d45f57a0f5b..4d294cb17338 100644 --- a/pkg/kv/kvserver/replica_send.go +++ b/pkg/kv/kvserver/replica_send.go @@ -13,6 +13,7 @@ package kvserver import ( "context" "reflect" + "sync/atomic" "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval" @@ -253,6 +254,13 @@ type batchExecutionFn func( var _ batchExecutionFn = (*Replica).executeWriteBatch var _ batchExecutionFn = (*Replica).executeReadOnlyBatch +// TODO: remove +var OptimisticEvalCount int64 +var EvalCount int64 +var OptimisticRetryEvalCount int64 +var RetryEvalCount int64 +var OptimisticPossibleButNotDoneCount int64 + // executeBatchWithConcurrencyRetries is the entry point for client (non-admin) // requests that execute against the range's state. The method coordinates the // execution of requests that may require multiple retries due to interactions @@ -273,6 +281,7 @@ func (r *Replica) executeBatchWithConcurrencyRetries( // Try to execute command; exit retry loop on success. var g *concurrency.Guard var latchSpans, lockSpans *spanset.SpanSet + requestEvalKind := concurrency.PessimisticEval defer func() { // NB: wrapped to delay g evaluation to its value when returning. if g != nil { @@ -280,6 +289,12 @@ func (r *Replica) executeBatchWithConcurrencyRetries( } }() for { + evalCount := atomic.AddInt64(&EvalCount, 1) + if evalCount%500 == 0 { + log.Infof(ctx, "EvalCount: %d, Optimistic: %d, Retry: %d, OptimisticRetry: %d", + evalCount, atomic.LoadInt64(&OptimisticEvalCount), + atomic.LoadInt64(&RetryEvalCount), atomic.LoadInt64(&OptimisticRetryEvalCount)) + } // Exit loop if context has been canceled or timed out. if err := ctx.Err(); err != nil { return nil, roachpb.NewError(errors.Wrap(err, "aborted during Replica.Send")) @@ -306,24 +321,34 @@ func (r *Replica) executeBatchWithConcurrencyRetries( // 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. + // 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. This will be done only in + // the first iteration of the for loop, which means requestEvalKind can be + // set to OptimisticEval only in the first iteration. if latchSpans == nil { var err error - latchSpans, lockSpans, err = r.collectSpans(ba) + latchSpans, lockSpans, requestEvalKind, err = r.collectSpans(ba) if err != nil { return nil, roachpb.NewError(err) } // Handle load-based splitting. r.recordBatchForLoadBasedSplitting(ctx, ba, latchSpans) + } else { + atomic.AddInt64(&RetryEvalCount, 1) + } + if requestEvalKind == concurrency.OptimisticEval { + atomic.AddInt64(&OptimisticEvalCount, 1) } - // 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 - // returns a request guard that must be eventually released. + // SequenceReq may acquire latches, if not already held, to prevent + // overlapping requests from executing until this request completes. After + // latching, if not doing optimistic evaluation, it will wait on any + // conflicting locks to ensure that the request has full isolation during + // evaluation. This returns a request guard that must be eventually + // released. For optimistic evaluation, Guard.CheckOptimisticNoConflicts + // must be called immediately after successful evaluation. var resp []roachpb.ResponseUnion g, resp, pErr = r.concMgr.SequenceReq(ctx, g, concurrency.Request{ Txn: ba.Txn, @@ -334,6 +359,7 @@ func (r *Replica) executeBatchWithConcurrencyRetries( Requests: ba.Requests, LatchSpans: latchSpans, LockSpans: lockSpans, + EvalKind: requestEvalKind, }) if pErr != nil { return nil, pErr @@ -365,6 +391,14 @@ func (r *Replica) executeBatchWithConcurrencyRetries( if filter := r.store.cfg.TestingKnobs.TestingConcurrencyRetryFilter; filter != nil { filter(ctx, *ba, pErr) } + + // Typically, retries are marked PessimisticEval. The one exception is a + // pessimistic retry immediately after an optimistic eval which failed + // when checking for conflicts, which is handled below. Note that an + // optimistic eval failure for any other reason will also retry as + // PessimisticEval. + requestEvalKind = concurrency.PessimisticEval + switch t := pErr.GetDetail().(type) { case *roachpb.WriteIntentError: // Drop latches, but retain lock wait-queues. @@ -392,6 +426,12 @@ func (r *Replica) executeBatchWithConcurrencyRetries( if pErr = r.handleMergeInProgressError(ctx, ba, pErr, t); pErr != nil { return nil, pErr } + case *roachpb.OptimisticEvalConflictsError: + // We are deliberately not dropping latches. The next iteration will + // pessimistically check for locks while holding these latches, and will + // find them again and queue up, and then release latches. + requestEvalKind = concurrency.PessimisticAfterFailedOptimisticEval + atomic.AddInt64(&OptimisticRetryEvalCount, 1) default: log.Fatalf(ctx, "unexpected concurrency retry error %T", t) } @@ -400,11 +440,13 @@ func (r *Replica) executeBatchWithConcurrencyRetries( } // isConcurrencyRetryError returns whether or not the provided error is a -// "server-side concurrency retry error" that will be captured and retried by -// executeBatchWithConcurrencyRetries. Server-side concurrency retry errors are -// handled by dropping a request's latches, waiting for and/or ensuring that the -// condition which caused the error is handled, re-sequencing through the -// concurrency manager, and executing the request again. +// "concurrency retry error" that will be captured and retried by +// executeBatchWithConcurrencyRetries. Most concurrency retry errors are +// handled by dropping a request's latches, waiting for and/or ensuring that +// the condition which caused the error is handled, re-sequencing through the +// concurrency manager, and executing the request again. The one exception is +// OptimisticEvalConflictsError, where there is no need to drop latches, and +// the request can immediately proceed to retrying pessimistically. func isConcurrencyRetryError(pErr *roachpb.Error) bool { switch pErr.GetDetail().(type) { case *roachpb.WriteIntentError: @@ -424,11 +466,14 @@ func isConcurrencyRetryError(pErr *roachpb.Error) bool { // retrying. 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 + // evaluated against is in the process of being merged into its left-hand // neighbor. The request cannot proceed until the range merge completes, // either successfully or unsuccessfully, so it waits before retrying. // If the merge does complete successfully, the retry will be rejected // with an error that will propagate back to the client. + case *roachpb.OptimisticEvalConflictsError: + // Optimistic evaluation encountered a conflict. The request will + // immediately retry pessimistically. default: return false } @@ -663,8 +708,14 @@ func (r *Replica) checkBatchRequest(ba *roachpb.BatchRequest, isReadOnly bool) e func (r *Replica) collectSpans( ba *roachpb.BatchRequest, -) (latchSpans, lockSpans *spanset.SpanSet, _ error) { +) (latchSpans, lockSpans *spanset.SpanSet, requestEvalKind concurrency.RequestEvalKind, _ error) { latchSpans, lockSpans = new(spanset.SpanSet), new(spanset.SpanSet) + isReadOnly := ba.IsReadOnly() + r.mu.RLock() + desc := r.descRLocked() + liveCount := r.mu.state.Stats.LiveCount + r.mu.RUnlock() + // TODO(bdarnell): need to make this less global when local // latches are used more heavily. For example, a split will // have a large read-only span but also a write (see #10084). @@ -695,14 +746,13 @@ func (r *Replica) collectSpans( // than the request timestamp, and may have to retry at a higher timestamp. // This is still safe as we're only ever writing at timestamps higher than the // timestamp any write latch would be declared at. - desc := r.Desc() batcheval.DeclareKeysForBatch(desc, ba.Header, latchSpans) for _, union := range ba.Requests { inner := union.GetInner() if cmd, ok := batcheval.LookupCommand(inner.Method()); ok { cmd.DeclareKeys(desc, ba.Header, inner, latchSpans, lockSpans) } else { - return nil, nil, errors.Errorf("unrecognized command %s", inner.Method()) + return nil, nil, 0, errors.Errorf("unrecognized command %s", inner.Method()) } } @@ -714,9 +764,39 @@ func (r *Replica) collectSpans( // If any command gave us spans that are invalid, bail out early // (before passing them to the spanlatch manager, which may panic). if err := s.Validate(); err != nil { - return nil, nil, err + return nil, nil, 0, err + } + } + + requestEvalKind = concurrency.PessimisticEval + if isReadOnly { + // Evaluate batches optimistically if they have a key limit which is less + // than the number of live keys on the Range. Ignoring write latches and + // locks can be beneficial because it can help avoid waiting on writes to + // keys that the batch will never actually need to read due to the + // overestimate of its key bounds. Only after it is clear exactly what + // spans were read do we verify whether there were any conflicts with + // concurrent writes. + // + // This case is not uncommon; for example, a Scan which requests the entire + // range but has a limit of 1 result. We want to avoid allowing overly broad + // spans from backing up the latch manager, or encountering too much contention + // in the lock table. + // + // The heuristic is limit < k * liveCount, where k <= 1. The use of k=1 + // below is an un-tuned setting. + limit := ba.Header.MaxSpanRequestKeys + const k = 1 + if limit > 0 && limit < k*liveCount { + requestEvalKind = concurrency.OptimisticEval + } else { + opbndCount := atomic.AddInt64(&OptimisticPossibleButNotDoneCount, 1) + if opbndCount%500 == 0 { + log.Infof(context.TODO(), "OptimisticPossibleBND: %d, limit: %d, liveCount: %d", + opbndCount, limit, liveCount) + } } } - return latchSpans, lockSpans, nil + return latchSpans, lockSpans, requestEvalKind, nil } diff --git a/pkg/roachpb/batch_generated.go b/pkg/roachpb/batch_generated.go index 99556c74f790..590746bafb19 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_OptimisticEvalConflicts: + return t.OptimisticEvalConflicts default: return nil } @@ -328,6 +330,8 @@ func (ru *ErrorDetail) MustSetInner(r error) { union = &ErrorDetail_RangefeedRetry{t} case *IndeterminateCommitError: union = &ErrorDetail_IndeterminateCommit{t} + case *OptimisticEvalConflictsError: + union = &ErrorDetail_OptimisticEvalConflicts{t} default: panic(fmt.Sprintf("unsupported type %T for %T", r, ru)) } diff --git a/pkg/roachpb/errors.go b/pkg/roachpb/errors.go index 4aee815c3e65..97f042a76bf5 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 + OptimisticEvalConflictsErrType ErrorDetailType = 40 // When adding new error types, don't forget to update NumErrors below. // CommunicationErrType indicates a gRPC error; this is not an ErrorDetail. @@ -1238,3 +1239,24 @@ func (e *IndeterminateCommitError) Type() ErrorDetailType { } var _ ErrorDetailInterface = &IndeterminateCommitError{} + +// NewOptimisticEvalConflictsError initializes a new +// OptimisticEvalConflictsError. +func NewOptimisticEvalConflictsError() *OptimisticEvalConflictsError { + return &OptimisticEvalConflictsError{} +} + +func (e *OptimisticEvalConflictsError) Error() string { + return e.message(nil) +} + +func (e *OptimisticEvalConflictsError) message(pErr *Error) string { + return "optimistic eval encountered conflict" +} + +// Type is part of the ErrorDetailInterface. +func (e *OptimisticEvalConflictsError) Type() ErrorDetailType { + return OptimisticEvalConflictsErrType +} + +var _ ErrorDetailInterface = &OptimisticEvalConflictsError{} diff --git a/pkg/roachpb/errors.pb.go b/pkg/roachpb/errors.pb.go index 27fb90b85f44..cecb60eb9caa 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_b3c9ed361eb5091b, []int{0} + return fileDescriptor_errors_25a35fbc562ffb0a, []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_b3c9ed361eb5091b, []int{1} + return fileDescriptor_errors_25a35fbc562ffb0a, []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_b3c9ed361eb5091b, []int{2} + return fileDescriptor_errors_25a35fbc562ffb0a, []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_b3c9ed361eb5091b, []int{9, 0} + return fileDescriptor_errors_25a35fbc562ffb0a, []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_b3c9ed361eb5091b, []int{27, 0} + return fileDescriptor_errors_25a35fbc562ffb0a, []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_b3c9ed361eb5091b, []int{0} + return fileDescriptor_errors_25a35fbc562ffb0a, []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_b3c9ed361eb5091b, []int{1} + return fileDescriptor_errors_25a35fbc562ffb0a, []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_b3c9ed361eb5091b, []int{2} + return fileDescriptor_errors_25a35fbc562ffb0a, []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_b3c9ed361eb5091b, []int{3} + return fileDescriptor_errors_25a35fbc562ffb0a, []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_b3c9ed361eb5091b, []int{4} + return fileDescriptor_errors_25a35fbc562ffb0a, []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_b3c9ed361eb5091b, []int{5} + return fileDescriptor_errors_25a35fbc562ffb0a, []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_b3c9ed361eb5091b, []int{6} + return fileDescriptor_errors_25a35fbc562ffb0a, []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_b3c9ed361eb5091b, []int{7} + return fileDescriptor_errors_25a35fbc562ffb0a, []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_b3c9ed361eb5091b, []int{8} + return fileDescriptor_errors_25a35fbc562ffb0a, []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_b3c9ed361eb5091b, []int{9} + return fileDescriptor_errors_25a35fbc562ffb0a, []int{9} } func (m *TransactionStatusError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -763,7 +763,7 @@ func (m *WriteIntentError) Reset() { *m = WriteIntentError{} } func (m *WriteIntentError) String() string { return proto.CompactTextString(m) } func (*WriteIntentError) ProtoMessage() {} func (*WriteIntentError) Descriptor() ([]byte, []int) { - return fileDescriptor_errors_b3c9ed361eb5091b, []int{10} + return fileDescriptor_errors_25a35fbc562ffb0a, []int{10} } func (m *WriteIntentError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -808,7 +808,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_b3c9ed361eb5091b, []int{11} + return fileDescriptor_errors_25a35fbc562ffb0a, []int{11} } func (m *WriteTooOldError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -844,7 +844,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_b3c9ed361eb5091b, []int{12} + return fileDescriptor_errors_25a35fbc562ffb0a, []int{12} } func (m *OpRequiresTxnError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -881,7 +881,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_b3c9ed361eb5091b, []int{13} + return fileDescriptor_errors_25a35fbc562ffb0a, []int{13} } func (m *ConditionFailedError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -918,7 +918,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_b3c9ed361eb5091b, []int{14} + return fileDescriptor_errors_25a35fbc562ffb0a, []int{14} } func (m *LeaseRejectedError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -956,7 +956,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_b3c9ed361eb5091b, []int{15} + return fileDescriptor_errors_25a35fbc562ffb0a, []int{15} } func (m *AmbiguousResultError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -990,7 +990,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_b3c9ed361eb5091b, []int{16} + return fileDescriptor_errors_25a35fbc562ffb0a, []int{16} } func (m *RaftGroupDeletedError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1028,7 +1028,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_b3c9ed361eb5091b, []int{17} + return fileDescriptor_errors_25a35fbc562ffb0a, []int{17} } func (m *ReplicaCorruptionError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1065,7 +1065,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_b3c9ed361eb5091b, []int{18} + return fileDescriptor_errors_25a35fbc562ffb0a, []int{18} } func (m *ReplicaTooOldError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1100,7 +1100,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_b3c9ed361eb5091b, []int{19} + return fileDescriptor_errors_25a35fbc562ffb0a, []int{19} } func (m *StoreNotFoundError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1145,7 +1145,7 @@ type UnhandledRetryableError struct { func (m *UnhandledRetryableError) Reset() { *m = UnhandledRetryableError{} } func (*UnhandledRetryableError) ProtoMessage() {} func (*UnhandledRetryableError) Descriptor() ([]byte, []int) { - return fileDescriptor_errors_b3c9ed361eb5091b, []int{20} + return fileDescriptor_errors_25a35fbc562ffb0a, []int{20} } func (m *UnhandledRetryableError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1197,7 +1197,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_b3c9ed361eb5091b, []int{21} + return fileDescriptor_errors_25a35fbc562ffb0a, []int{21} } func (m *TransactionRetryWithProtoRefreshError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1235,7 +1235,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_b3c9ed361eb5091b, []int{22} + return fileDescriptor_errors_25a35fbc562ffb0a, []int{22} } func (m *TxnAlreadyEncounteredErrorError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1272,7 +1272,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_b3c9ed361eb5091b, []int{23} + return fileDescriptor_errors_25a35fbc562ffb0a, []int{23} } func (m *IntegerOverflowError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1308,7 +1308,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_b3c9ed361eb5091b, []int{24} + return fileDescriptor_errors_25a35fbc562ffb0a, []int{24} } func (m *BatchTimestampBeforeGCError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1347,7 +1347,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_b3c9ed361eb5091b, []int{25} + return fileDescriptor_errors_25a35fbc562ffb0a, []int{25} } func (m *IntentMissingError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1384,7 +1384,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_b3c9ed361eb5091b, []int{26} + return fileDescriptor_errors_25a35fbc562ffb0a, []int{26} } func (m *MergeInProgressError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1419,7 +1419,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_b3c9ed361eb5091b, []int{27} + return fileDescriptor_errors_25a35fbc562ffb0a, []int{27} } func (m *RangeFeedRetryError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1458,7 +1458,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_b3c9ed361eb5091b, []int{28} + return fileDescriptor_errors_25a35fbc562ffb0a, []int{28} } func (m *IndeterminateCommitError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1483,6 +1483,42 @@ func (m *IndeterminateCommitError) XXX_DiscardUnknown() { var xxx_messageInfo_IndeterminateCommitError proto.InternalMessageInfo +// OptimisticEvalConflictsError indicates that the optimistic evaluation of a +// BatchRequest failed when checking locks or latches after evaluation. This +// is never sent over-the-wire and is a proto to fit the existing plumbing +// that passes a roachpb.Error. +type OptimisticEvalConflictsError struct { +} + +func (m *OptimisticEvalConflictsError) Reset() { *m = OptimisticEvalConflictsError{} } +func (m *OptimisticEvalConflictsError) String() string { return proto.CompactTextString(m) } +func (*OptimisticEvalConflictsError) ProtoMessage() {} +func (*OptimisticEvalConflictsError) Descriptor() ([]byte, []int) { + return fileDescriptor_errors_25a35fbc562ffb0a, []int{29} +} +func (m *OptimisticEvalConflictsError) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *OptimisticEvalConflictsError) 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 *OptimisticEvalConflictsError) XXX_Merge(src proto.Message) { + xxx_messageInfo_OptimisticEvalConflictsError.Merge(dst, src) +} +func (m *OptimisticEvalConflictsError) XXX_Size() int { + return m.Size() +} +func (m *OptimisticEvalConflictsError) XXX_DiscardUnknown() { + xxx_messageInfo_OptimisticEvalConflictsError.DiscardUnknown(m) +} + +var xxx_messageInfo_OptimisticEvalConflictsError 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 +1550,7 @@ type ErrorDetail struct { // *ErrorDetail_MergeInProgress // *ErrorDetail_RangefeedRetry // *ErrorDetail_IndeterminateCommit + // *ErrorDetail_OptimisticEvalConflicts Value isErrorDetail_Value `protobuf_oneof:"value"` } @@ -1521,7 +1558,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_b3c9ed361eb5091b, []int{29} + return fileDescriptor_errors_25a35fbc562ffb0a, []int{30} } func (m *ErrorDetail) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1636,6 +1673,9 @@ type ErrorDetail_RangefeedRetry struct { type ErrorDetail_IndeterminateCommit struct { IndeterminateCommit *IndeterminateCommitError `protobuf:"bytes,39,opt,name=indeterminate_commit,json=indeterminateCommit,oneof"` } +type ErrorDetail_OptimisticEvalConflicts struct { + OptimisticEvalConflicts *OptimisticEvalConflictsError `protobuf:"bytes,40,opt,name=optimistic_eval_conflicts,json=optimisticEvalConflicts,oneof"` +} func (*ErrorDetail_NotLeaseHolder) isErrorDetail_Value() {} func (*ErrorDetail_RangeNotFound) isErrorDetail_Value() {} @@ -1665,6 +1705,7 @@ func (*ErrorDetail_IntentMissing) isErrorDetail_Value() {} func (*ErrorDetail_MergeInProgress) isErrorDetail_Value() {} func (*ErrorDetail_RangefeedRetry) isErrorDetail_Value() {} func (*ErrorDetail_IndeterminateCommit) isErrorDetail_Value() {} +func (*ErrorDetail_OptimisticEvalConflicts) isErrorDetail_Value() {} func (m *ErrorDetail) GetValue() isErrorDetail_Value { if m != nil { @@ -1869,6 +1910,13 @@ func (m *ErrorDetail) GetIndeterminateCommit() *IndeterminateCommitError { return nil } +func (m *ErrorDetail) GetOptimisticEvalConflicts() *OptimisticEvalConflictsError { + if x, ok := m.GetValue().(*ErrorDetail_OptimisticEvalConflicts); ok { + return x.OptimisticEvalConflicts + } + 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 +1948,7 @@ func (*ErrorDetail) XXX_OneofFuncs() (func(msg proto.Message, b *proto.Buffer) e (*ErrorDetail_MergeInProgress)(nil), (*ErrorDetail_RangefeedRetry)(nil), (*ErrorDetail_IndeterminateCommit)(nil), + (*ErrorDetail_OptimisticEvalConflicts)(nil), } } @@ -2047,6 +2096,11 @@ func _ErrorDetail_OneofMarshaler(msg proto.Message, b *proto.Buffer) error { if err := b.EncodeMessage(x.IndeterminateCommit); err != nil { return err } + case *ErrorDetail_OptimisticEvalConflicts: + _ = b.EncodeVarint(40<<3 | proto.WireBytes) + if err := b.EncodeMessage(x.OptimisticEvalConflicts); err != nil { + return err + } case nil: default: return fmt.Errorf("ErrorDetail.Value has unexpected type %T", x) @@ -2281,6 +2335,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.optimistic_eval_conflicts + if wire != proto.WireBytes { + return true, proto.ErrInternalBadWireType + } + msg := new(OptimisticEvalConflictsError) + err := b.DecodeMessage(msg) + m.Value = &ErrorDetail_OptimisticEvalConflicts{msg} + return true, err default: return false, nil } @@ -2430,6 +2492,11 @@ func _ErrorDetail_OneofSizer(msg proto.Message) (n int) { n += 2 // tag and wire n += proto.SizeVarint(uint64(s)) n += s + case *ErrorDetail_OptimisticEvalConflicts: + s := proto.Size(x.OptimisticEvalConflicts) + 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 +2515,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_b3c9ed361eb5091b, []int{30} + return fileDescriptor_errors_25a35fbc562ffb0a, []int{31} } func (m *ErrPosition) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2510,7 +2577,7 @@ type Error struct { func (m *Error) Reset() { *m = Error{} } func (*Error) ProtoMessage() {} func (*Error) Descriptor() ([]byte, []int) { - return fileDescriptor_errors_b3c9ed361eb5091b, []int{31} + return fileDescriptor_errors_25a35fbc562ffb0a, []int{32} } func (m *Error) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2565,6 +2632,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((*OptimisticEvalConflictsError)(nil), "cockroach.kv.kvpb.OptimisticEvalConflictsError") proto.RegisterType((*ErrorDetail)(nil), "cockroach.kv.kvpb.ErrorDetail") proto.RegisterType((*ErrPosition)(nil), "cockroach.kv.kvpb.ErrPosition") proto.RegisterType((*Error)(nil), "cockroach.kv.kvpb.Error") @@ -3419,6 +3487,24 @@ func (m *IndeterminateCommitError) MarshalTo(dAtA []byte) (int, error) { return i, nil } +func (m *OptimisticEvalConflictsError) 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 *OptimisticEvalConflictsError) 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 +3950,22 @@ func (m *ErrorDetail_IndeterminateCommit) MarshalTo(dAtA []byte) (int, error) { } return i, nil } +func (m *ErrorDetail_OptimisticEvalConflicts) MarshalTo(dAtA []byte) (int, error) { + i := 0 + if m.OptimisticEvalConflicts != nil { + dAtA[i] = 0xc2 + i++ + dAtA[i] = 0x2 + i++ + i = encodeVarintErrors(dAtA, i, uint64(m.OptimisticEvalConflicts.Size())) + n52, err := m.OptimisticEvalConflicts.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 +4013,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 +4025,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 } @@ -4347,6 +4449,15 @@ func (m *IndeterminateCommitError) Size() (n int) { return n } +func (m *OptimisticEvalConflictsError) 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 +4806,18 @@ func (m *ErrorDetail_IndeterminateCommit) Size() (n int) { } return n } +func (m *ErrorDetail_OptimisticEvalConflicts) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.OptimisticEvalConflicts != nil { + l = m.OptimisticEvalConflicts.Size() + n += 2 + l + sovErrors(uint64(l)) + } + return n +} func (m *ErrPosition) Size() (n int) { if m == nil { return 0 @@ -7554,6 +7677,56 @@ func (m *IndeterminateCommitError) Unmarshal(dAtA []byte) error { } return nil } +func (m *OptimisticEvalConflictsError) 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: OptimisticEvalConflictsError: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: OptimisticEvalConflictsError: 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 +8652,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 OptimisticEvalConflicts", 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 := &OptimisticEvalConflictsError{} + if err := v.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.Value = &ErrorDetail_OptimisticEvalConflicts{v} + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipErrors(dAtA[iNdEx:]) @@ -8947,195 +9152,198 @@ var ( ErrIntOverflowErrors = fmt.Errorf("proto: integer overflow") ) -func init() { proto.RegisterFile("roachpb/errors.proto", fileDescriptor_errors_b3c9ed361eb5091b) } - -var fileDescriptor_errors_b3c9ed361eb5091b = []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_25a35fbc562ffb0a) } + +var fileDescriptor_errors_25a35fbc562ffb0a = []byte{ + // 3039 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x9c, 0x59, 0xcb, 0x6f, 0x1b, 0xd7, + 0xd5, 0x27, 0x29, 0x4a, 0xa2, 0x8e, 0x1e, 0x1e, 0x5f, 0xcb, 0xd2, 0x48, 0xb6, 0x29, 0x65, 0x6c, + 0x27, 0xb6, 0x83, 0x50, 0x1f, 0x9c, 0x2f, 0xc0, 0x97, 0x7c, 0xce, 0x82, 0x8f, 0x91, 0x38, 0x12, + 0x1f, 0xea, 0x90, 0x8a, 0xed, 0x18, 0xc5, 0xcd, 0x88, 0x73, 0x45, 0x4d, 0x45, 0xce, 0x30, 0x77, + 0x86, 0xb2, 0x04, 0x74, 0x51, 0xb4, 0x9b, 0xa0, 0x05, 0x8a, 0xee, 0xda, 0x65, 0x81, 0xa0, 0x8b, + 0x02, 0x45, 0xd1, 0x5d, 0x77, 0x5d, 0x67, 0x99, 0x65, 0x50, 0x14, 0x46, 0xeb, 0x74, 0xd7, 0xff, + 0x20, 0xab, 0xe2, 0x3e, 0x86, 0x1c, 0x8a, 0x43, 0x59, 0xc9, 0x6e, 0x78, 0x5e, 0xf7, 0xdc, 0xd7, + 0xef, 0xfc, 0xce, 0x25, 0x2c, 0x53, 0xcf, 0x6a, 0x1d, 0xf7, 0x0e, 0xb7, 0x08, 0xa5, 0x1e, 0xf5, + 0x73, 0x3d, 0xea, 0x05, 0x1e, 0xba, 0xde, 0xf2, 0x5a, 0x27, 0x5c, 0x93, 0x3b, 0x39, 0xcd, 0x9d, + 0x9c, 0xf6, 0x0e, 0xd7, 0x6f, 0x0a, 0x83, 0x0b, 0x96, 0xeb, 0x28, 0xf4, 0xb7, 0xad, 0xc0, 0x92, + 0xb2, 0x95, 0x50, 0xd6, 0x25, 0x81, 0x15, 0x91, 0xab, 0xfd, 0xc0, 0xe9, 0x6c, 0x1d, 0x77, 0x5a, + 0x5b, 0x81, 0xd3, 0x25, 0x7e, 0x60, 0x75, 0x7b, 0x52, 0xb3, 0xdc, 0xf6, 0xda, 0x1e, 0xff, 0xdc, + 0x62, 0x5f, 0x42, 0xaa, 0xfd, 0x25, 0x05, 0x37, 0x6a, 0x5e, 0x50, 0x21, 0x96, 0x4f, 0xca, 0x5e, + 0xc7, 0x26, 0x54, 0x67, 0x43, 0xa3, 0x12, 0xcc, 0x52, 0xd2, 0xeb, 0x38, 0x2d, 0x4b, 0x4d, 0x6e, + 0x26, 0x1f, 0xcc, 0x3f, 0xbe, 0x97, 0x1b, 0xe6, 0x2b, 0xc7, 0xce, 0x99, 0xc2, 0xa2, 0x44, 0xfc, + 0x16, 0x75, 0x7a, 0x81, 0x47, 0x0b, 0xe9, 0xaf, 0x5e, 0x6d, 0x24, 0xcc, 0xd0, 0x15, 0xed, 0xc0, + 0x42, 0x87, 0x45, 0xc6, 0xc7, 0x3c, 0xb4, 0x9a, 0xba, 0x7a, 0x28, 0x73, 0xbe, 0x33, 0xcc, 0x09, + 0x7d, 0x00, 0x19, 0x6a, 0xb9, 0x6d, 0x82, 0x1d, 0x5b, 0x9d, 0xda, 0x4c, 0x3e, 0x98, 0x2a, 0xac, + 0xb3, 0x91, 0x5e, 0xbf, 0xda, 0x98, 0x35, 0x99, 0xdc, 0x28, 0x7d, 0x37, 0xfc, 0x34, 0x67, 0xb9, + 0xad, 0x61, 0xa3, 0x1c, 0x4c, 0xf3, 0x28, 0x6a, 0x9a, 0x0f, 0xac, 0xc6, 0x0c, 0xcc, 0x67, 0x6e, + 0x0a, 0x33, 0x74, 0x17, 0xa0, 0xd5, 0xf7, 0x03, 0xaf, 0x8b, 0xbb, 0x7e, 0x5b, 0x9d, 0xde, 0x4c, + 0x3e, 0x98, 0x93, 0x53, 0x9a, 0x13, 0xf2, 0xaa, 0xdf, 0xd6, 0x56, 0x60, 0xb9, 0xe6, 0xd9, 0xe4, + 0xc0, 0xb5, 0x4e, 0x2d, 0xa7, 0x63, 0x1d, 0x76, 0x08, 0x5f, 0x32, 0x6d, 0x0d, 0x56, 0x0f, 0x5c, + 0xbf, 0xdf, 0xeb, 0x79, 0x34, 0x20, 0xb6, 0x49, 0x3e, 0xef, 0x13, 0x3f, 0x10, 0xaa, 0x9f, 0x27, + 0x01, 0xf1, 0xe4, 0x6a, 0x5e, 0xb0, 0xed, 0xf5, 0x5d, 0x5b, 0x2c, 0x72, 0x74, 0x56, 0xc9, 0xab, + 0xcf, 0xea, 0x03, 0xc8, 0xf8, 0x81, 0x47, 0xb9, 0x5b, 0x6a, 0xd4, 0xad, 0xc1, 0xe4, 0xc2, 0x4d, + 0x7e, 0x9a, 0xb3, 0xdc, 0xd6, 0xb0, 0xb5, 0xdf, 0x4e, 0xc1, 0x4d, 0x1e, 0x6b, 0x8f, 0x9c, 0x57, + 0x1d, 0xbf, 0x6b, 0x05, 0xad, 0x63, 0x91, 0xc7, 0xfb, 0x70, 0x9d, 0x8a, 0x74, 0xb1, 0x1f, 0x58, + 0x34, 0xc0, 0x27, 0xe4, 0x9c, 0x27, 0xb4, 0x50, 0x98, 0xfd, 0xee, 0xd5, 0xc6, 0xd4, 0x1e, 0x39, + 0x37, 0xaf, 0x49, 0x8b, 0x06, 0x33, 0xd8, 0x23, 0xe7, 0x68, 0x0b, 0x42, 0x11, 0x26, 0xae, 0xcd, + 0x5d, 0x52, 0xa3, 0x2e, 0x8b, 0x52, 0xaf, 0xbb, 0x36, 0x73, 0x38, 0x86, 0x5b, 0x36, 0xe9, 0x51, + 0xd2, 0xb2, 0x02, 0x62, 0xe3, 0xae, 0xcc, 0x80, 0xd8, 0x98, 0xcf, 0x8b, 0x6f, 0xeb, 0xfc, 0x63, + 0x2d, 0xee, 0x6c, 0x30, 0xfd, 0xd8, 0x21, 0x5b, 0x1b, 0x06, 0xab, 0x0e, 0x62, 0x71, 0x53, 0xf4, + 0x19, 0xac, 0x47, 0x46, 0xf2, 0xfb, 0xed, 0x36, 0xf1, 0x83, 0xc1, 0x40, 0xe9, 0xab, 0x0e, 0x64, + 0xaa, 0xc3, 0x28, 0x8d, 0x30, 0x88, 0x18, 0xa1, 0x02, 0x33, 0x3c, 0x98, 0xaf, 0x4e, 0x6f, 0x4e, + 0x3d, 0x98, 0x7f, 0x7c, 0x7b, 0x52, 0x34, 0xc3, 0x3d, 0xf2, 0x0a, 0x2b, 0x72, 0x7b, 0x96, 0x84, + 0x8f, 0xe1, 0x06, 0x84, 0xba, 0x56, 0xc7, 0x94, 0x31, 0xb4, 0xff, 0xa4, 0x40, 0x33, 0x89, 0x65, + 0x3f, 0x75, 0x82, 0x63, 0xc7, 0x3d, 0x70, 0x5b, 0x84, 0x06, 0x96, 0xe3, 0x06, 0xe7, 0xdc, 0xf2, + 0xd4, 0xea, 0x88, 0x6d, 0xda, 0x85, 0x25, 0x4a, 0x2c, 0x1b, 0x0f, 0x6e, 0xb6, 0xbc, 0x9a, 0x77, + 0x22, 0x83, 0xb3, 0xeb, 0x9f, 0x3b, 0xee, 0xb4, 0x72, 0xcd, 0xd0, 0x48, 0x2e, 0xd7, 0x22, 0x73, + 0x1d, 0x08, 0x91, 0x09, 0x88, 0x9c, 0x39, 0x7e, 0xe0, 0xb8, 0xed, 0x48, 0xbc, 0xd4, 0xd5, 0xe3, + 0x5d, 0x0f, 0xdd, 0x87, 0x31, 0x0b, 0xb0, 0xd8, 0xb5, 0xce, 0x22, 0xe1, 0xa6, 0xae, 0x10, 0xce, + 0x5c, 0xe8, 0x5a, 0x67, 0xc3, 0x18, 0x2f, 0xe0, 0x86, 0x77, 0xe8, 0x13, 0x7a, 0x4a, 0x22, 0xf3, + 0xf4, 0xd5, 0x34, 0x5f, 0xe5, 0x38, 0xe0, 0xa8, 0x4b, 0xeb, 0x8b, 0xf9, 0x21, 0xef, 0xa2, 0xc2, + 0xff, 0x28, 0xfd, 0xbb, 0xdf, 0x6f, 0x24, 0x34, 0x1b, 0x56, 0x9b, 0xd4, 0x72, 0x7d, 0xab, 0x15, + 0x38, 0x9e, 0x9b, 0x3f, 0xe4, 0xd7, 0x55, 0xac, 0xb0, 0x01, 0x33, 0x94, 0x58, 0xbe, 0xe7, 0xf2, + 0x95, 0x5d, 0x7a, 0xfc, 0x6e, 0x6e, 0x0c, 0xa4, 0x73, 0xe3, 0xbe, 0x26, 0x77, 0x91, 0xe3, 0xca, + 0x00, 0xda, 0x0b, 0x58, 0x8e, 0x58, 0xee, 0xf7, 0x7d, 0x79, 0xd7, 0x8a, 0x00, 0xbd, 0xbe, 0x7f, + 0x4c, 0x08, 0x0e, 0xce, 0x5c, 0xb9, 0x81, 0xd9, 0x98, 0x79, 0x45, 0x9c, 0x43, 0x08, 0x12, 0x7e, + 0xcd, 0x33, 0x57, 0xfb, 0x45, 0x12, 0x6e, 0x46, 0x0c, 0x4c, 0x12, 0xd0, 0x73, 0x11, 0x7e, 0xe7, + 0xc2, 0x0c, 0x1e, 0x5e, 0x3e, 0x03, 0xee, 0x19, 0x97, 0x3f, 0x7a, 0x0b, 0xe6, 0xc8, 0x59, 0x40, + 0x2d, 0x8e, 0x84, 0xa9, 0x08, 0x12, 0x66, 0xb8, 0x98, 0x01, 0xe1, 0xdf, 0x92, 0xb0, 0x12, 0x89, + 0xd5, 0x08, 0xac, 0xa0, 0xef, 0x8b, 0x34, 0x56, 0x60, 0x8a, 0xf9, 0x25, 0x23, 0x7e, 0x4c, 0x80, + 0x6a, 0x83, 0xf4, 0x52, 0x3c, 0xbd, 0xff, 0xb9, 0x3c, 0xbd, 0x48, 0xc8, 0x5c, 0xec, 0x2a, 0x3f, + 0x81, 0x19, 0x21, 0x47, 0x08, 0x96, 0x4c, 0x3d, 0xdf, 0xa8, 0xd7, 0xf0, 0x41, 0x6d, 0xaf, 0x56, + 0x7f, 0x5a, 0x53, 0x12, 0x48, 0x85, 0x65, 0x29, 0x6b, 0x3e, 0xab, 0xe1, 0x62, 0xbd, 0x5a, 0x35, + 0x9a, 0x4d, 0xbd, 0xa4, 0xa4, 0xb4, 0x74, 0x26, 0xa9, 0x24, 0xb5, 0x06, 0x28, 0x4f, 0xa9, 0x13, + 0x10, 0x76, 0xcd, 0x5c, 0x01, 0xd5, 0xe8, 0x43, 0x98, 0x75, 0xf8, 0x4f, 0x5f, 0x4d, 0xf2, 0x43, + 0xb7, 0x16, 0xb3, 0x39, 0xc2, 0x21, 0xac, 0x76, 0xd2, 0x7e, 0x37, 0x9d, 0x49, 0x29, 0x53, 0xda, + 0x1f, 0x92, 0x32, 0x6a, 0xd3, 0xf3, 0xea, 0x1d, 0x79, 0xb0, 0xf2, 0x30, 0xf7, 0x83, 0x6e, 0xed, + 0xd0, 0x0b, 0xd5, 0x40, 0xb1, 0x5a, 0x41, 0xdf, 0xea, 0xfc, 0xb0, 0xfb, 0x7a, 0x4d, 0x38, 0x0f, + 0xc4, 0xda, 0x32, 0xa0, 0x7a, 0x8f, 0x55, 0x29, 0x87, 0x12, 0xbf, 0x79, 0xe6, 0x8a, 0x4a, 0xd5, + 0x80, 0xe5, 0xa2, 0xe7, 0xda, 0x0e, 0x5b, 0xfd, 0x6d, 0xcb, 0xe9, 0x84, 0x37, 0xe3, 0xff, 0x61, + 0x41, 0x8e, 0x7e, 0x6a, 0x75, 0xfa, 0x44, 0xce, 0x21, 0xae, 0xa0, 0x7e, 0xc2, 0xf4, 0xe6, 0xbc, + 0xb0, 0xe6, 0x3f, 0xb4, 0x3f, 0x27, 0x01, 0x89, 0x3a, 0x4b, 0x7e, 0x42, 0x5a, 0x83, 0xdb, 0x96, + 0x85, 0xd9, 0x2e, 0xf1, 0x7d, 0xab, 0x4d, 0x46, 0x0e, 0x4a, 0x28, 0x44, 0x4f, 0x60, 0x4e, 0x56, + 0x10, 0x62, 0xcb, 0xa9, 0x4e, 0xac, 0xe0, 0xe1, 0x7a, 0x0d, 0x1c, 0xd0, 0x47, 0x90, 0x09, 0x21, + 0x4a, 0x02, 0xd1, 0x9b, 0x9c, 0x07, 0xf6, 0xda, 0xe7, 0xb0, 0x9c, 0xef, 0x1e, 0x3a, 0xed, 0xbe, + 0xd7, 0xf7, 0x4d, 0xe2, 0xf7, 0x3b, 0xc1, 0xd5, 0x32, 0xfe, 0x10, 0xe6, 0x5f, 0x52, 0xab, 0xd7, + 0x23, 0x36, 0x26, 0x94, 0xc6, 0xe4, 0x1c, 0x9e, 0x71, 0x1e, 0xce, 0x04, 0x69, 0xac, 0x53, 0xaa, + 0xad, 0xb2, 0xe2, 0x7c, 0x14, 0xec, 0x50, 0xaf, 0xdf, 0x2b, 0x91, 0x0e, 0x09, 0x57, 0x49, 0xc3, + 0xb0, 0x22, 0xc9, 0x51, 0xd1, 0xa3, 0xb4, 0xdf, 0x63, 0x3b, 0x23, 0xb2, 0x61, 0x57, 0x94, 0x7d, + 0xe0, 0x8b, 0x57, 0x2d, 0xc3, 0xc5, 0x55, 0xbf, 0x8d, 0x34, 0x98, 0xeb, 0x51, 0xaf, 0x45, 0x7c, + 0x5f, 0x2e, 0x61, 0x66, 0x00, 0x26, 0xa1, 0x58, 0x6b, 0x00, 0x92, 0x03, 0x44, 0x4f, 0xec, 0xc7, + 0x00, 0x92, 0xc5, 0x85, 0xec, 0x64, 0xba, 0x90, 0x95, 0x75, 0x6c, 0x4e, 0xda, 0x73, 0xa2, 0x31, + 0xfc, 0xc1, 0x56, 0x5f, 0x7c, 0xda, 0xda, 0x1e, 0x20, 0x4e, 0x40, 0xc6, 0x08, 0xcf, 0x80, 0xb9, + 0x24, 0xaf, 0xce, 0x5c, 0x1a, 0x8c, 0x59, 0x1d, 0x5b, 0xae, 0xdd, 0x61, 0x60, 0x1b, 0xd0, 0xf3, + 0x01, 0xe9, 0x42, 0x8f, 0x21, 0xdd, 0xd3, 0x29, 0x8d, 0x39, 0x8f, 0x23, 0x4b, 0x2d, 0x67, 0xcd, + 0x6d, 0x65, 0x19, 0xf8, 0x77, 0x12, 0xee, 0x5f, 0x44, 0x42, 0x56, 0x80, 0xf7, 0x19, 0x2f, 0x36, + 0xc9, 0x11, 0x25, 0x21, 0x64, 0x4f, 0x02, 0xb3, 0x17, 0x30, 0x13, 0x9c, 0xb9, 0x21, 0x0b, 0x5b, + 0x28, 0x94, 0x98, 0xea, 0xef, 0xaf, 0x36, 0xde, 0x6f, 0x3b, 0xc1, 0x71, 0xff, 0x30, 0xd7, 0xf2, + 0xba, 0x5b, 0x83, 0x7c, 0xec, 0xc3, 0xe1, 0xf7, 0x56, 0xef, 0xa4, 0xbd, 0xc5, 0x89, 0x7a, 0xbf, + 0xef, 0xd8, 0xb9, 0x83, 0x03, 0xa3, 0xf4, 0xfa, 0xd5, 0xc6, 0x74, 0xf3, 0xcc, 0x35, 0x4a, 0xe6, + 0x74, 0x70, 0xe6, 0x1a, 0x36, 0xda, 0x86, 0xf9, 0x60, 0x98, 0x9d, 0x3c, 0xc1, 0x57, 0x2b, 0x14, + 0x51, 0x47, 0x6d, 0x1b, 0x36, 0x9a, 0x67, 0x6e, 0xbe, 0xc3, 0xca, 0xff, 0xb9, 0xee, 0xb6, 0xbc, + 0x3e, 0xe3, 0x14, 0xf2, 0x70, 0x89, 0xf9, 0xdd, 0x05, 0xe8, 0x51, 0x72, 0x8a, 0xf9, 0xa9, 0x19, + 0x99, 0xe6, 0x1c, 0x93, 0x8b, 0x63, 0xf8, 0xab, 0x24, 0x2c, 0x33, 0xd8, 0x6b, 0x13, 0x5a, 0x3f, + 0x25, 0xf4, 0xa8, 0xe3, 0xbd, 0x14, 0xde, 0x6b, 0x30, 0x15, 0x43, 0x17, 0x99, 0x0c, 0x3d, 0x84, + 0xc5, 0x56, 0x9f, 0x52, 0xe2, 0x06, 0x12, 0x35, 0x04, 0x5b, 0x15, 0xb1, 0x17, 0xa4, 0x8a, 0x43, + 0x04, 0x7a, 0x0f, 0xae, 0x39, 0x6e, 0x8b, 0x92, 0xee, 0xd0, 0x78, 0x2a, 0x62, 0xbc, 0x34, 0x50, + 0x0a, 0x44, 0xf9, 0x32, 0x09, 0xb7, 0x0a, 0x8c, 0xf2, 0x0d, 0x61, 0x8e, 0x1c, 0x79, 0x94, 0xec, + 0x14, 0x07, 0x78, 0xdb, 0xfc, 0x41, 0x78, 0x3b, 0x64, 0x22, 0x2c, 0xc4, 0x31, 0x3b, 0x04, 0x5e, + 0xc7, 0xfe, 0x3e, 0x40, 0x3b, 0xf4, 0xd2, 0xba, 0x80, 0x44, 0xa5, 0xa8, 0x3a, 0xbe, 0xef, 0xb8, + 0x6d, 0x91, 0xdb, 0x13, 0x58, 0x78, 0x49, 0x3d, 0xb7, 0x8d, 0x45, 0xdd, 0x90, 0xe9, 0x4d, 0x2e, + 0x33, 0xe6, 0x3c, 0x37, 0x17, 0x3f, 0xc2, 0xe5, 0x4e, 0x8d, 0x2f, 0x37, 0x6b, 0x4c, 0xaa, 0x84, + 0x32, 0xce, 0xb9, 0x4f, 0xbd, 0x36, 0x25, 0xbe, 0xa8, 0x9c, 0xda, 0xaf, 0x53, 0x70, 0x83, 0x93, + 0xd1, 0x6d, 0x22, 0xef, 0x8f, 0x48, 0x64, 0xef, 0x02, 0x57, 0x78, 0x2f, 0xe6, 0xf6, 0xc4, 0xf8, + 0xc5, 0x57, 0xe2, 0x3f, 0x26, 0x07, 0xa5, 0x78, 0x1d, 0x56, 0x64, 0xd9, 0x35, 0xf5, 0xfd, 0x8a, + 0x51, 0xcc, 0x63, 0x53, 0xaf, 0xd6, 0x3f, 0xd1, 0x4b, 0x4a, 0x02, 0xad, 0x00, 0x0a, 0x75, 0xf9, + 0xda, 0x8e, 0x8e, 0x1b, 0xfb, 0x15, 0xa3, 0xa9, 0x24, 0xd1, 0x2a, 0xdc, 0x18, 0x91, 0x57, 0x75, + 0x73, 0x87, 0x55, 0xea, 0x48, 0x0d, 0x37, 0xf3, 0xdb, 0x4d, 0xdc, 0xa8, 0xe5, 0xf7, 0x1b, 0xe5, + 0x7a, 0x53, 0x99, 0x42, 0x59, 0x58, 0x97, 0x9a, 0x4a, 0x7d, 0xc7, 0x28, 0xe6, 0x2b, 0xb8, 0xbe, + 0xdf, 0xc0, 0x55, 0xa3, 0xd1, 0x30, 0x6a, 0x3b, 0x4a, 0x3a, 0xe2, 0xd9, 0xa8, 0xd4, 0x9f, 0xe2, + 0x62, 0xbd, 0xd6, 0x38, 0xa8, 0xea, 0xa6, 0x32, 0xad, 0x59, 0xa0, 0x1a, 0xae, 0x4d, 0x02, 0x42, + 0xbb, 0x8e, 0x6b, 0x05, 0xa4, 0xe8, 0x75, 0xbb, 0x8e, 0x84, 0x78, 0x1d, 0xe6, 0xfd, 0xc0, 0x6a, + 0x73, 0x5e, 0xfc, 0x3d, 0x09, 0x1a, 0x48, 0x47, 0xc6, 0xd0, 0xb2, 0x70, 0xbb, 0xde, 0x0b, 0x9c, + 0x2e, 0x2b, 0x28, 0x2d, 0xfd, 0xd4, 0xea, 0x14, 0x3d, 0xf7, 0xa8, 0xe3, 0xb4, 0x02, 0xb9, 0x27, + 0x7f, 0xbd, 0x09, 0xf3, 0xfc, 0xab, 0x44, 0x02, 0xcb, 0xe9, 0x20, 0x13, 0x14, 0xd7, 0x0b, 0xf0, + 0x48, 0xb7, 0x2c, 0xc6, 0x7e, 0x3b, 0x66, 0x57, 0x62, 0x3a, 0xf6, 0x72, 0xc2, 0x5c, 0x72, 0x47, + 0xc4, 0xa8, 0x0e, 0xd7, 0x44, 0x7b, 0xc9, 0x22, 0x1f, 0x31, 0x14, 0x96, 0xe7, 0xf8, 0xfe, 0xa4, + 0x8d, 0x1e, 0x41, 0xeb, 0x32, 0x6b, 0x1a, 0xa2, 0x52, 0xf4, 0x0c, 0x90, 0x08, 0x78, 0x42, 0xce, + 0x07, 0x0d, 0x9c, 0x84, 0xa6, 0x07, 0x93, 0x62, 0x5e, 0xec, 0x36, 0xcb, 0x09, 0x53, 0xa1, 0x17, + 0x14, 0xe8, 0x67, 0x49, 0xd8, 0xe4, 0xbd, 0xcd, 0x4b, 0xde, 0x02, 0xe1, 0xfe, 0xb0, 0x07, 0xe2, + 0xd7, 0x84, 0x35, 0x41, 0xb2, 0x71, 0xfb, 0x20, 0x6e, 0xa0, 0x37, 0x36, 0x4f, 0xe5, 0x84, 0x79, + 0x87, 0x5e, 0x66, 0x85, 0x7e, 0x0c, 0x37, 0x22, 0xb8, 0x89, 0x2d, 0xc1, 0xed, 0xf9, 0x23, 0xc0, + 0xfc, 0xe3, 0x47, 0x57, 0x6a, 0x04, 0xc2, 0x91, 0x50, 0x30, 0xa6, 0x42, 0x4d, 0x50, 0xa2, 0xe1, + 0x19, 0x97, 0x57, 0x67, 0x78, 0xec, 0x77, 0x2e, 0x8f, 0x3d, 0x68, 0x1d, 0xca, 0x09, 0xf3, 0x5a, + 0x30, 0x2a, 0x47, 0x4f, 0xe1, 0x7a, 0x34, 0x2a, 0x65, 0x97, 0x54, 0x9d, 0x9d, 0xb8, 0x21, 0xb1, + 0x3d, 0x03, 0xdb, 0x90, 0xe0, 0x82, 0x02, 0x7d, 0x0a, 0xd1, 0x49, 0x60, 0x9f, 0x13, 0x71, 0x35, + 0xc3, 0x23, 0x3f, 0xbc, 0x32, 0x69, 0x2f, 0x27, 0xcc, 0x68, 0x7e, 0x42, 0x83, 0xca, 0x0c, 0x00, + 0x9d, 0x80, 0x84, 0x00, 0x38, 0xc7, 0xa3, 0xde, 0x8d, 0x89, 0x7a, 0x91, 0x9d, 0x97, 0x13, 0x0c, + 0x0c, 0x07, 0x32, 0x64, 0xc0, 0xa2, 0x88, 0x14, 0x78, 0x1e, 0x66, 0x38, 0x0d, 0x97, 0x87, 0x8a, + 0x10, 0x9c, 0x41, 0x28, 0x21, 0x63, 0x97, 0xc5, 0xeb, 0x61, 0x2a, 0xf9, 0x30, 0xbf, 0xfb, 0xf3, + 0x13, 0x2f, 0xcb, 0x38, 0x71, 0x66, 0x97, 0xc5, 0x8b, 0x4a, 0xd9, 0x86, 0xb7, 0x42, 0x26, 0x8d, + 0x8f, 0x38, 0x95, 0x56, 0x17, 0x26, 0x6e, 0x78, 0x1c, 0xe9, 0x66, 0x1b, 0xde, 0x1a, 0x95, 0xa3, + 0x1a, 0x2c, 0x09, 0x8c, 0xa0, 0x92, 0x4a, 0xab, 0x8b, 0x13, 0xb3, 0x1c, 0xa7, 0xdc, 0x2c, 0xcb, + 0x4e, 0x54, 0xca, 0xb2, 0x74, 0x3d, 0x9b, 0xe0, 0xfe, 0xf0, 0x35, 0x4b, 0x5d, 0x9a, 0x98, 0x65, + 0xdc, 0xbb, 0x17, 0xcb, 0xd2, 0x1d, 0x95, 0x0b, 0xa0, 0x38, 0x0a, 0x70, 0x9b, 0xb1, 0x59, 0x6c, + 0x0b, 0x3a, 0xab, 0x2a, 0x97, 0x00, 0x45, 0x0c, 0xf3, 0x15, 0x40, 0x31, 0xaa, 0x60, 0xe7, 0x32, + 0xa4, 0xa5, 0xad, 0x01, 0x1d, 0x56, 0xaf, 0x4f, 0x3c, 0x97, 0xf1, 0xd4, 0x99, 0x9d, 0x4b, 0x7a, + 0x51, 0xc3, 0xf1, 0x52, 0xc6, 0x0e, 0xcf, 0x13, 0x9a, 0x8c, 0x97, 0x63, 0x94, 0x99, 0xe3, 0x65, + 0x54, 0xca, 0x16, 0xd7, 0x0a, 0xdb, 0x08, 0x4c, 0x79, 0x1f, 0xa1, 0xae, 0x4f, 0x5c, 0xdc, 0xb8, + 0x8e, 0x83, 0x2d, 0xae, 0x35, 0x2a, 0x67, 0x69, 0x0a, 0x12, 0x3d, 0x84, 0xf5, 0x5b, 0x13, 0xd3, + 0x1c, 0x27, 0xe1, 0x2c, 0x4d, 0x3f, 0x2a, 0x45, 0xbf, 0x4c, 0xc2, 0xbd, 0x31, 0x14, 0xe1, 0x48, + 0x8c, 0xf9, 0x23, 0x31, 0xa6, 0x82, 0x0d, 0xab, 0xb7, 0xf9, 0x30, 0xff, 0x77, 0x05, 0x60, 0x89, + 0x25, 0xd2, 0xe5, 0x84, 0xb9, 0x19, 0xbc, 0xc1, 0x90, 0xad, 0x99, 0x23, 0x68, 0x26, 0xf6, 0x24, + 0xcf, 0x54, 0x37, 0x26, 0xae, 0x59, 0x1c, 0x23, 0x65, 0x6b, 0xe6, 0x8c, 0xca, 0x19, 0xb8, 0xf7, + 0x87, 0x6f, 0xb3, 0x58, 0x76, 0x89, 0xea, 0xe6, 0x44, 0x70, 0x9f, 0xf0, 0x92, 0xcb, 0xc0, 0xbd, + 0x3f, 0xa6, 0x42, 0x2f, 0x40, 0x19, 0x34, 0xe5, 0xf8, 0x90, 0x33, 0x51, 0x55, 0xe3, 0xb1, 0x73, + 0x31, 0xb1, 0x2f, 0x21, 0xae, 0x1c, 0xe3, 0x47, 0x35, 0xe8, 0x25, 0xdc, 0x61, 0x6d, 0x86, 0x25, + 0x28, 0x3c, 0x26, 0x43, 0x0e, 0x2f, 0x19, 0xfb, 0x5d, 0x3e, 0xd2, 0xe3, 0xb8, 0x6d, 0xb9, 0x9c, + 0xf9, 0x97, 0x13, 0xe6, 0x7a, 0x30, 0xd1, 0x84, 0x61, 0x8d, 0x40, 0x68, 0x56, 0xeb, 0x19, 0x7f, + 0x55, 0xef, 0x4d, 0x3c, 0x67, 0xe3, 0x3c, 0x97, 0x9d, 0x33, 0x27, 0x2a, 0x45, 0x07, 0x70, 0xbd, + 0xcb, 0xf8, 0x29, 0x76, 0x5c, 0x76, 0xb0, 0x38, 0x43, 0x55, 0xef, 0x4f, 0xdc, 0xdb, 0x38, 0x2e, + 0xcb, 0xd6, 0xa7, 0x3b, 0x2a, 0x47, 0x3f, 0x92, 0x34, 0xe7, 0x88, 0xf0, 0x9d, 0x65, 0x15, 0xf0, + 0xed, 0x89, 0xcc, 0x29, 0x86, 0xcf, 0x32, 0xe6, 0x34, 0x08, 0x20, 0xaa, 0xdf, 0x67, 0xb0, 0xec, + 0x44, 0x09, 0x22, 0x6e, 0x71, 0x86, 0xa8, 0xbe, 0xc3, 0xe3, 0xbe, 0x1b, 0x3b, 0xff, 0x78, 0x3e, + 0x59, 0x4e, 0x98, 0x37, 0x9c, 0x71, 0x1d, 0xea, 0xc2, 0x9a, 0x37, 0xe0, 0x87, 0x98, 0x9c, 0x5a, + 0x1d, 0xdc, 0x0a, 0x19, 0xa2, 0xfa, 0x80, 0x0f, 0xb3, 0x15, 0x5b, 0x78, 0x26, 0x73, 0xca, 0x72, + 0xc2, 0x5c, 0xf5, 0xe2, 0xf5, 0x85, 0x59, 0x98, 0xe6, 0x4d, 0xd5, 0x6e, 0x3a, 0x73, 0x4d, 0x51, + 0x76, 0xd3, 0x99, 0x1b, 0xca, 0xf2, 0x6e, 0x3a, 0xb3, 0xac, 0xdc, 0xdc, 0x4d, 0x67, 0x6e, 0x2a, + 0x2b, 0xbb, 0xe9, 0xcc, 0x8a, 0xb2, 0xba, 0x9b, 0xce, 0xac, 0x2a, 0xea, 0x6e, 0x3a, 0xa3, 0x2a, + 0x6b, 0xbb, 0xe9, 0xcc, 0x9a, 0xb2, 0xbe, 0x9b, 0xce, 0xdc, 0x51, 0xb2, 0xbb, 0xe9, 0x4c, 0x56, + 0xd9, 0xd8, 0x4d, 0x67, 0xde, 0x52, 0x34, 0xed, 0x21, 0x27, 0xae, 0xfb, 0x9e, 0xcf, 0xcb, 0x12, + 0x5a, 0x87, 0x69, 0x36, 0xbf, 0x33, 0xf9, 0x44, 0x20, 0x98, 0xb0, 0x10, 0x69, 0x5f, 0x4c, 0xc3, + 0x74, 0xf8, 0x0f, 0xc3, 0x85, 0x87, 0x93, 0x35, 0xd9, 0xf7, 0x5f, 0x8f, 0xbc, 0xe2, 0x0b, 0x83, + 0xe1, 0x6b, 0xca, 0x4f, 0x47, 0x19, 0x19, 0x25, 0xfc, 0xcf, 0x09, 0xce, 0x37, 0x97, 0x62, 0x0f, + 0xe1, 0x08, 0x0a, 0x71, 0xe3, 0xc2, 0x3d, 0x39, 0xce, 0xed, 0xe1, 0x38, 0xe3, 0x56, 0x23, 0x84, + 0x4d, 0xca, 0x50, 0x11, 0x16, 0xfb, 0x2e, 0x39, 0xeb, 0x79, 0x3e, 0xb1, 0x39, 0x1d, 0x48, 0x5f, + 0xa5, 0x15, 0x30, 0x17, 0x06, 0x4e, 0x8c, 0x04, 0x6c, 0xc1, 0xbc, 0x47, 0x9d, 0xb6, 0xe3, 0x62, + 0x56, 0x22, 0x39, 0x99, 0x9c, 0x2e, 0x2c, 0xb1, 0x9c, 0xbe, 0x7b, 0xb5, 0x31, 0xc3, 0xca, 0xa9, + 0x51, 0x32, 0x41, 0x98, 0xb0, 0x5f, 0x68, 0x1f, 0x66, 0x6c, 0xde, 0x11, 0x48, 0x72, 0x98, 0x9d, + 0xf4, 0xa2, 0x21, 0xfa, 0x86, 0x82, 0x2a, 0xe7, 0xa7, 0x0c, 0xe7, 0x27, 0x34, 0xa6, 0x8c, 0x83, + 0xfe, 0x37, 0xdc, 0xa0, 0xd9, 0xcb, 0x02, 0x86, 0xfb, 0x29, 0xb7, 0x0e, 0xf5, 0x61, 0xca, 0xf5, + 0x5e, 0x4a, 0xc2, 0xf7, 0x86, 0xbe, 0xb7, 0x24, 0xe7, 0xf3, 0xe4, 0xea, 0xef, 0x1e, 0x2c, 0x40, + 0xb1, 0xe3, 0xb5, 0x4e, 0x86, 0xff, 0x03, 0xb0, 0xf1, 0xd0, 0x1e, 0x2c, 0x32, 0x7c, 0xb3, 0x07, + 0xd8, 0x26, 0xb8, 0xe1, 0x66, 0x24, 0x81, 0xf0, 0x3f, 0xd2, 0x9c, 0x2e, 0x0c, 0xa3, 0xef, 0x3b, + 0x0b, 0x24, 0x22, 0x13, 0xef, 0x3c, 0xe2, 0x55, 0xf6, 0xd1, 0x3f, 0x52, 0xa0, 0x4e, 0x7a, 0xb9, + 0x67, 0x9d, 0x62, 0xbe, 0x50, 0x37, 0x9b, 0x78, 0xec, 0x05, 0xf9, 0x3e, 0xbc, 0x35, 0xa2, 0xe1, + 0x3f, 0xf4, 0x12, 0x36, 0xf5, 0x62, 0xdd, 0x2c, 0xe1, 0xed, 0xfa, 0x41, 0xad, 0xa4, 0x24, 0x59, + 0x2b, 0x3a, 0x62, 0x56, 0xac, 0x18, 0x7a, 0x8d, 0xfd, 0xda, 0xd5, 0x8b, 0xac, 0x55, 0xdd, 0x80, + 0x5b, 0x23, 0xfa, 0xfd, 0x83, 0x46, 0x59, 0x37, 0xc3, 0x68, 0x4a, 0x1a, 0xdd, 0x82, 0xd5, 0xf1, + 0x71, 0x70, 0x63, 0x3f, 0x5f, 0x53, 0xa6, 0x51, 0x1e, 0x3e, 0x1e, 0x55, 0x56, 0x4c, 0x3d, 0x5f, + 0x7a, 0x3e, 0x7c, 0xd0, 0xc6, 0x75, 0x13, 0x9b, 0xf5, 0x4a, 0x45, 0x2f, 0xe1, 0x42, 0xbe, 0xb8, + 0x87, 0xf7, 0xeb, 0x8d, 0x86, 0x51, 0xa8, 0xe8, 0xbc, 0xff, 0xce, 0x3f, 0x57, 0x66, 0xd0, 0x3b, + 0x70, 0x77, 0x24, 0x44, 0x4d, 0x7f, 0x8a, 0x2b, 0x7a, 0xbe, 0xa1, 0xe3, 0x7d, 0x53, 0xff, 0x44, + 0xaf, 0x35, 0x1b, 0xb8, 0xf9, 0xac, 0xa6, 0x64, 0xd0, 0x43, 0xb8, 0x3f, 0x62, 0xd8, 0x34, 0xaa, + 0x7a, 0xa3, 0x99, 0xaf, 0xee, 0xe3, 0x62, 0xbe, 0x58, 0xd6, 0xe5, 0x94, 0xf4, 0x92, 0x32, 0xbb, + 0x9e, 0xfe, 0xe2, 0xcb, 0x6c, 0x42, 0x63, 0xcb, 0x9b, 0x7a, 0xf4, 0xa7, 0xd1, 0xbf, 0x02, 0x22, + 0x7f, 0x2b, 0x88, 0x36, 0xbc, 0x69, 0x3e, 0x1f, 0x5f, 0x5c, 0xde, 0xf3, 0x33, 0xcd, 0x53, 0xd3, + 0x68, 0xea, 0xb8, 0x59, 0xaf, 0xe3, 0x7a, 0x85, 0x2d, 0x27, 0x7f, 0x24, 0x60, 0x8a, 0x86, 0x6e, + 0x1a, 0xf9, 0x8a, 0xf1, 0x69, 0xbe, 0x50, 0xd1, 0x95, 0x29, 0x74, 0x07, 0xd6, 0x84, 0x3c, 0xdf, + 0x78, 0x5e, 0x2b, 0x4a, 0xb7, 0xed, 0xbc, 0x51, 0x39, 0x30, 0x75, 0x65, 0x1a, 0x69, 0x90, 0x15, + 0x6a, 0xb1, 0x30, 0xb8, 0xa4, 0xe7, 0x4b, 0x15, 0xa3, 0xa6, 0x63, 0xfd, 0x59, 0x51, 0xd7, 0x4b, + 0x7a, 0x49, 0x99, 0x11, 0x49, 0x3f, 0xfa, 0x08, 0xd0, 0x38, 0x0a, 0xa0, 0x0c, 0xa4, 0x6b, 0xf5, + 0x9a, 0xae, 0x24, 0xd0, 0x3c, 0xcc, 0xb2, 0x85, 0xac, 0x6f, 0x6f, 0x2b, 0x49, 0xb4, 0x08, 0x73, + 0x46, 0xb5, 0xaa, 0x97, 0x8c, 0x7c, 0x53, 0x57, 0x52, 0x85, 0x87, 0x5f, 0xfd, 0x2b, 0x9b, 0xf8, + 0xea, 0x75, 0x36, 0xf9, 0xf5, 0xeb, 0x6c, 0xf2, 0x9b, 0xd7, 0xd9, 0xe4, 0x3f, 0x5f, 0x67, 0x93, + 0xbf, 0xf9, 0x36, 0x9b, 0xf8, 0xfa, 0xdb, 0x6c, 0xe2, 0x9b, 0x6f, 0xb3, 0x89, 0x4f, 0x67, 0x25, + 0x2e, 0xfc, 0x37, 0x00, 0x00, 0xff, 0xff, 0xf0, 0xba, 0x6e, 0x7c, 0xf9, 0x1f, 0x00, 0x00, } diff --git a/pkg/roachpb/errors.proto b/pkg/roachpb/errors.proto index f25f249e6370..068c8bcaf29a 100644 --- a/pkg/roachpb/errors.proto +++ b/pkg/roachpb/errors.proto @@ -472,6 +472,13 @@ message IndeterminateCommitError { optional roachpb.Transaction staging_txn = 1 [(gogoproto.nullable) = false]; } +// OptimisticEvalConflictsError indicates that the optimistic evaluation of a +// BatchRequest failed when checking locks or latches after evaluation. This +// is never sent over-the-wire and is a proto to fit the existing plumbing +// that passes a roachpb.Error. +message OptimisticEvalConflictsError { +} + // 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 +516,7 @@ message ErrorDetail { MergeInProgressError merge_in_progress = 37; RangeFeedRetryError rangefeed_retry = 38; IndeterminateCommitError indeterminate_commit = 39; + OptimisticEvalConflictsError optimistic_eval_conflicts = 40; } } diff --git a/pkg/workload/kv/kv.go b/pkg/workload/kv/kv.go index 40eac4a4378e..20216a13689c 100644 --- a/pkg/workload/kv/kv.go +++ b/pkg/workload/kv/kv.go @@ -22,11 +22,14 @@ import ( "strconv" "strings" "sync/atomic" + "time" + "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" "github.com/cockroachdb/cockroach/pkg/util/timeutil" "github.com/cockroachdb/cockroach/pkg/workload" "github.com/cockroachdb/cockroach/pkg/workload/histogram" "github.com/cockroachdb/errors" + "github.com/jackc/pgx" "github.com/spf13/pflag" ) @@ -66,6 +69,8 @@ type kv struct { cycleLength int64 readPercent int spanPercent int + spanLimit int + writesUseSelectForUpdate bool seed int64 writeSeq string sequential bool @@ -114,6 +119,10 @@ var kvMeta = workload.Meta{ `Percent (0-100) of operations that are reads of existing keys.`) g.flags.IntVar(&g.spanPercent, `span-percent`, 0, `Percent (0-100) of operations that are spanning queries of all ranges.`) + g.flags.IntVar(&g.spanLimit, `span-limit`, 0, + `LIMIT count for each spanning query, or 0 for no limit`) + g.flags.BoolVar(&g.writesUseSelectForUpdate, `sfu-writes`, false, + `Use SFU and transactional writes with a sleep after SFU.`) g.flags.Int64Var(&g.seed, `seed`, 1, `Key hash seed.`) g.flags.BoolVar(&g.zipfian, `zipfian`, false, `Pick keys in a zipfian distribution instead of randomly.`) @@ -301,8 +310,32 @@ func (w *kv) Ops( } writeStmtStr := buf.String() + // Select for update statement + var sfuStmtStr string + if w.writesUseSelectForUpdate { + if w.shards != 0 { + return workload.QueryLoad{}, fmt.Errorf("select for update in kv requires shard=0") + } + buf.Reset() + buf.WriteString(`SELECT k, v FROM kv WHERE k IN (`) + for i := 0; i < w.batchSize; i++ { + if i > 0 { + buf.WriteString(", ") + } + fmt.Fprintf(&buf, `$%d`, i+1) + } + buf.WriteString(`) FOR UPDATE`) + sfuStmtStr = buf.String() + } + // Span statement - spanStmtStr := "SELECT count(v) FROM kv" + buf.Reset() + buf.WriteString(`SELECT count(v) FROM [SELECT v FROM kv`) + if w.spanLimit > 0 { + fmt.Fprintf(&buf, ` ORDER BY k LIMIT %d`, w.spanLimit) + } + buf.WriteString(`]`) + spanStmtStr := buf.String() ql := workload.QueryLoad{SQLDatabase: sqlDatabase} seq := &sequence{config: w, val: int64(writeSeq)} @@ -315,10 +348,14 @@ func (w *kv) Ops( } op.readStmt = op.sr.Define(readStmtStr) op.writeStmt = op.sr.Define(writeStmtStr) + if len(sfuStmtStr) > 0 { + op.sfuStmt = op.sr.Define(sfuStmtStr) + } op.spanStmt = op.sr.Define(spanStmtStr) if err := op.sr.Init(ctx, "kv", mcp, w.connFlags); err != nil { return workload.QueryLoad{}, err } + op.mcp = mcp if w.sequential { op.g = newSequentialGenerator(seq) } else if w.zipfian { @@ -336,9 +373,11 @@ type kvOp struct { config *kv hists *histogram.Histograms sr workload.SQLRunner + mcp *workload.MultiConnPool readStmt workload.StmtHandle writeStmt workload.StmtHandle spanStmt workload.StmtHandle + sfuStmt workload.StmtHandle g keyGenerator numEmptyResults *int64 // accessed atomically } @@ -377,19 +416,70 @@ func (o *kvOp) run(ctx context.Context) error { return err } const argCount = 2 - args := make([]interface{}, argCount*o.config.batchSize) + writeArgs := make([]interface{}, argCount*o.config.batchSize) + var sfuArgs []interface{} + if o.config.writesUseSelectForUpdate { + sfuArgs = make([]interface{}, o.config.batchSize) + } for i := 0; i < o.config.batchSize; i++ { j := i * argCount - args[j+0] = o.g.writeKey() - args[j+1] = randomBlock(o.config, o.g.rand()) + writeArgs[j+0] = o.g.writeKey() + if sfuArgs != nil { + sfuArgs[i] = writeArgs[j] + } + writeArgs[j+1] = randomBlock(o.config, o.g.rand()) } start := timeutil.Now() - _, err := o.writeStmt.Exec(ctx, args...) + var err error + if o.config.writesUseSelectForUpdate { + var tx *pgx.Tx + if tx, err = o.mcp.Get().Begin(); err != nil { + return err + } + defer tx.Rollback() + rows, err := o.sfuStmt.QueryTx(ctx, tx, sfuArgs...) + if err != nil { + return err + } + rows.Close() + if err = rows.Err(); err != nil { + return err + } + // Simulate a transaction that does other work between the SFU and write. + // TODO(sumeer): this should be configurable. + time.Sleep(10 * time.Millisecond) + if _, err = o.writeStmt.ExecTx(ctx, tx, writeArgs...); err != nil { + // Multiple write transactions can contend and encounter + // a serialization failure. We swallow such an error. + return o.tryHandleWriteErr("write-write-err", start, err) + } + if err = tx.Commit(); err != nil { + return o.tryHandleWriteErr("write-commit-err", start, err) + } + } else { + _, err = o.writeStmt.Exec(ctx, writeArgs...) + } elapsed := timeutil.Since(start) o.hists.Get(`write`).Record(elapsed) return err } +func (o *kvOp) tryHandleWriteErr(name string, start time.Time, err error) error { + // If the error not an instance of pgx.PgError, then it is unexpected. + pgErr := pgx.PgError{} + if !errors.As(err, &pgErr) { + return err + } + // Transaction retry errors are acceptable. Allow the transaction + // to rollback. + if pgcode.MakeCode(pgErr.Code) == pgcode.SerializationFailure { + elapsed := timeutil.Since(start) + o.hists.Get(name).Record(elapsed) + return nil + } + return err +} + func (o *kvOp) close(context.Context) { if empty := atomic.LoadInt64(o.numEmptyResults); empty != 0 { fmt.Printf("Number of reads that didn't return any results: %d.\n", empty)