From 104b96dc87af66fab860d355fa272c92d51ee4fc 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 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. Fixes #49973 Informs #9521 Release note: None --- .../concurrency/concurrency_control.go | 34 + .../concurrency/concurrency_manager.go | 64 +- pkg/kv/kvserver/concurrency/lock_table.go | 100 ++- .../concurrency/lock_table_waiter_test.go | 5 + pkg/kv/kvserver/replica_read.go | 84 ++- pkg/kv/kvserver/replica_send.go | 76 +- pkg/roachpb/errors.go | 22 + pkg/roachpb/errors.pb.go | 684 ++++++++++++------ pkg/roachpb/errors.proto | 8 + 9 files changed, 800 insertions(+), 277 deletions(-) diff --git a/pkg/kv/kvserver/concurrency/concurrency_control.go b/pkg/kv/kvserver/concurrency/concurrency_control.go index acbf453f7492..08349649c00f 100644 --- a/pkg/kv/kvserver/concurrency/concurrency_control.go +++ b/pkg/kv/kvserver/concurrency/concurrency_control.go @@ -178,6 +178,18 @@ type RequestSequencer interface { // does so, it will not return a request guard. SequenceReq(context.Context, *Guard, Request) (*Guard, Response, *Error) + // SequenceOptimisticRetryingAsPessimistic is akin to SequenceReq, but used + // for the first pessimistic retry of a previously optimistic request + // evaluation. Request.Optimistic must be false here. The first pessimistic + // retry has not called any methods in ContentionHandler, so the state of + // latches and locks is whatever it was when the optimistic request + // previously went through SequenceReq. In the current implementation, + // latches are held, but only a snapshot of the lock table has been taken + // (which will need to be snapshot again). In the future, latches will not + // be held either. + SequenceOptimisticRetryingAsPessimistic( + context.Context, *Guard, Request) (*Guard, Response, *Error) + // FinishReq marks the request as complete, releasing any protection // the request had against conflicting requests and allowing conflicting // requests that are blocked on this one to proceed. The guard should not @@ -351,6 +363,11 @@ type Request struct { // (Txn == nil), all reads and writes are considered to take place at // Timestamp. LockSpans *spanset.SpanSet + + // Set to true when the request is desiring optimistic evaluation, i.e., + // locks will be checked after evaluation using + // Guard.CheckOptimisticNoConflicts. + Optimistic bool } // Guard is returned from Manager.SequenceReq. The guard is passed back in to @@ -361,6 +378,10 @@ type Guard struct { ltg lockTableGuard } +func (g *Guard) CheckOptimisticNoConflicts(lockSpansRead *spanset.SpanSet) (ok bool) { + return g.ltg.CheckOptimisticNoConflicts(lockSpansRead) +} + // Response is a slice of responses to requests in a batch. This type is used // when the concurrency manager is able to respond to a request directly during // sequencing. @@ -460,6 +481,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 +627,12 @@ 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. + 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..ccaa81b0a90f 100644 --- a/pkg/kv/kvserver/concurrency/concurrency_manager.go +++ b/pkg/kv/kvserver/concurrency/concurrency_manager.go @@ -114,14 +114,18 @@ func (m *managerImpl) SequenceReq( var g *Guard if prev == nil { g = newGuard(req) - log.Event(ctx, "sequencing request") + if req.Optimistic { + log.Event(ctx, "optimistically sequencing request") + } else { + log.Event(ctx, "sequencing request") + } } else { g = prev g.AssertNoLatches() log.Event(ctx, "re-sequencing request") } - resp, err := m.sequenceReqWithGuard(ctx, g, req) + resp, err := m.sequenceReqWithGuard(ctx, g, req, false /* 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 +134,30 @@ func (m *managerImpl) SequenceReq( return g, nil, nil } -func (m *managerImpl) sequenceReqWithGuard( +// SequenceOptimisticRetryingAsPessimistic implements the RequestSequencer +// interface. +func (m *managerImpl) SequenceOptimisticRetryingAsPessimistic( ctx context.Context, g *Guard, req Request, +) (*Guard, Response, *Error) { + if g == nil { + panic("retry should have non-nil guard") + } + if g.Req.Optimistic { + panic("pessimistic retry has optimistic bit set to true") + } + g.AssertLatches() + log.Event(ctx, "re-sequencing request after optimistic sequencing failed") + resp, err := m.sequenceReqWithGuard(ctx, g, req, true /* holdsLatches */) + if resp != nil || err != nil { + // Ensure that we release the guard if we return a response or an error. + m.FinishReq(g) + return nil, resp, err + } + return g, nil, nil +} + +func (m *managerImpl) sequenceReqWithGuard( + 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,12 +174,18 @@ 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 + } } // Some requests don't want the wait on locks. @@ -161,9 +193,17 @@ func (m *managerImpl) sequenceReqWithGuard( 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.Optimistic { + 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() { diff --git a/pkg/kv/kvserver/concurrency/lock_table.go b/pkg/kv/kvserver/concurrency/lock_table.go index e587234ba404..4e7ce805089b 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. 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 1a219da3db11..ce94a83bd4f3 100644 --- a/pkg/kv/kvserver/replica_read.go +++ b/pkg/kv/kvserver/replica_read.go @@ -61,17 +61,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.Optimistic { + // 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 { @@ -79,6 +98,9 @@ func (r *Replica) executeReadOnlyBatch( } // Otherwise, update the timestamp cache and release the concurrency guard. + // NB: For optimistic evaluation, used for limited scans, the update to the + // timestamp cache limits itself to the spans that were read. + // TODO(sumeer): why are we updating the ts cache if pErr != nil? ec, g := endCmds{repl: r, g: g}, nil ec.done(ctx, ba, br, pErr) @@ -181,3 +203,61 @@ 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 + // baCopy.Requests = append([]roachpb.RequestUnion(nil), baCopy.Requests...) + 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..c42cc3e31518 100644 --- a/pkg/kv/kvserver/replica_send.go +++ b/pkg/kv/kvserver/replica_send.go @@ -273,6 +273,7 @@ func (r *Replica) executeBatchWithConcurrencyRetries( // Try to execute command; exit retry loop on success. var g *concurrency.Guard var latchSpans, lockSpans *spanset.SpanSet + var optimistic, retryingAsPessimistic bool defer func() { // NB: wrapped to delay g evaluation to its value when returning. if g != nil { @@ -308,10 +309,12 @@ func (r *Replica) executeBatchWithConcurrencyRetries( // 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. + // have limited the transaction's maximum timestamp. This will be done + // only in the first iteration of the for loop, which means optimistic + // can be set to true only in the first iteration. if latchSpans == nil { var err error - latchSpans, lockSpans, err = r.collectSpans(ba) + latchSpans, lockSpans, optimistic, err = r.collectSpans(ba) if err != nil { return nil, roachpb.NewError(err) } @@ -324,8 +327,15 @@ func (r *Replica) executeBatchWithConcurrencyRetries( // 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 := r.concMgr.SequenceReq + if retryingAsPessimistic { + sequenceReq = r.concMgr.SequenceOptimisticRetryingAsPessimistic + // Subsequent retries will not follow optimistic evaluation, so set + // retryingAsPessimistic to false. + retryingAsPessimistic = false + } var resp []roachpb.ResponseUnion - g, resp, pErr = r.concMgr.SequenceReq(ctx, g, concurrency.Request{ + g, resp, pErr = sequenceReq(ctx, g, concurrency.Request{ Txn: ba.Txn, Timestamp: ba.Timestamp, Priority: ba.UserPriority, @@ -334,6 +344,7 @@ func (r *Replica) executeBatchWithConcurrencyRetries( Requests: ba.Requests, LatchSpans: latchSpans, LockSpans: lockSpans, + Optimistic: optimistic, }) if pErr != nil { return nil, pErr @@ -392,6 +403,9 @@ func (r *Replica) executeBatchWithConcurrencyRetries( if pErr = r.handleMergeInProgressError(ctx, ba, pErr, t); pErr != nil { return nil, pErr } + case *roachpb.OptimisticEvalConflictsError: + optimistic = false + retryingAsPessimistic = true default: log.Fatalf(ctx, "unexpected concurrency retry error %T", t) } @@ -400,11 +414,16 @@ 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. There are two kinds of concurrency +// retry errors: +// - 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. +// - Client-side concurrency retry errors are currently limited to conflicts +// encountered in optimistic evaluation. 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 +443,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 +685,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, optimistic bool, _ 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 +723,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, false, errors.Errorf("unrecognized command %s", inner.Method()) } } @@ -714,9 +741,30 @@ 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, false, err } } - return latchSpans, lockSpans, nil + optimistic = false + 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 + optimistic = limit > 0 && limit < liveCount + } + + return latchSpans, lockSpans, optimistic, nil } 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 5accc89482ad..53f7245418c3 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 { + 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; } }