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; } }