diff --git a/pkg/kv/kvclient/kvcoord/txn_interceptor_heartbeater.go b/pkg/kv/kvclient/kvcoord/txn_interceptor_heartbeater.go index abf7ed3e1a64..7f157adc6a99 100644 --- a/pkg/kv/kvclient/kvcoord/txn_interceptor_heartbeater.go +++ b/pkg/kv/kvclient/kvcoord/txn_interceptor_heartbeater.go @@ -187,14 +187,9 @@ func (h *txnHeartbeater) SendLocked( ba.Txn.Key = anchor } - // Start the heartbeat loop if it has not already started and this batch - // is not intending to commit/abort the transaction. + // Start the heartbeat loop if it has not already started. if !h.mu.loopStarted { - if !hasET { - if err := h.startHeartbeatLoopLocked(ctx); err != nil { - return nil, roachpb.NewError(err) - } - } + h.startHeartbeatLoopLocked(ctx) } } @@ -204,6 +199,10 @@ func (h *txnHeartbeater) SendLocked( // Set the EndTxn request's TxnHeartbeating flag. Set to true if // a hearbeat loop was started which indicates that transaction has // a transaction record. + // + // TODO(erikgrinaker): In v21.2 we always heartbeat the txn record, so + // this field is never used. However, we still need to set it when + // interacting with v21.1 nodes. We can remove this field in v22.1. et.TxnHeartbeating = h.mu.loopStarted // Preemptively stop the heartbeat loop in case of transaction abort. @@ -282,7 +281,7 @@ func (h *txnHeartbeater) closeLocked() { } // startHeartbeatLoopLocked starts a heartbeat loop in a different goroutine. -func (h *txnHeartbeater) startHeartbeatLoopLocked(ctx context.Context) error { +func (h *txnHeartbeater) startHeartbeatLoopLocked(ctx context.Context) { if h.mu.loopStarted { log.Fatal(ctx, "attempting to start a second heartbeat loop") } @@ -292,16 +291,31 @@ func (h *txnHeartbeater) startHeartbeatLoopLocked(ctx context.Context) error { // (it's zero). h.AmbientContext.AddLogTag("txn-hb", h.mu.txn.Short()) + const taskName = "[async] kv.TxnCoordSender: heartbeat loop" + // Create a new context so that the heartbeat loop doesn't inherit the // caller's cancelation. - // We want the loop to run in a span linked to the current one, though, so we - // put our span in the new context and expect RunAsyncTask to fork it - // immediately. - hbCtx := h.AnnotateCtx(context.Background()) - hbCtx = tracing.ContextWithSpan(hbCtx, tracing.SpanFromContext(ctx)) - hbCtx, h.mu.loopCancel = context.WithCancel(hbCtx) - - return h.stopper.RunAsyncTask(hbCtx, "kv.TxnCoordSender: heartbeat loop", h.heartbeatLoop) + hbCtx, hbCancel := context.WithCancel(h.AnnotateCtx(context.Background())) + + // Delay spawning the loop goroutine until the first loopInterval passes, to + // avoid the associated cost for small write transactions. In benchmarks, + // this gave a 3% throughput increase for point writes at high concurrency. + timer := time.AfterFunc(h.loopInterval, func() { + // We want the loop to run in a span linked to the current one, so we put + // our span in the context and fork it. + var span *tracing.Span + hbCtx = tracing.ContextWithSpan(hbCtx, tracing.SpanFromContext(ctx)) + hbCtx, span = tracing.ForkSpan(hbCtx, taskName) + defer span.Finish() + + // Only errors on quiesce, which is safe to ignore. + _ = h.stopper.RunTask(hbCtx, taskName, h.heartbeatLoop) + }) + + h.mu.loopCancel = func() { + timer.Stop() + hbCancel() + } } func (h *txnHeartbeater) cancelHeartbeatLoopLocked() { @@ -333,6 +347,11 @@ func (h *txnHeartbeater) heartbeatLoop(ctx context.Context) { defer ticker.Stop() } + // Loop is only spawned after loopInterval, so heartbeat immediately. + if !h.heartbeat(ctx) { + return + } + // Loop with ticker for periodic heartbeats. for { select { @@ -448,7 +467,7 @@ func (h *txnHeartbeater) heartbeat(ctx context.Context) bool { return true } -// abortTxnAsyncLocked sends an EndTxn(commmit=false) asynchronously. +// abortTxnAsyncLocked sends an EndTxn(commit=false) asynchronously. // The purpose of the async cleanup is to resolve transaction intents as soon // as possible when a transaction coordinator observes an ABORTED transaction. func (h *txnHeartbeater) abortTxnAsyncLocked(ctx context.Context) { diff --git a/pkg/kv/kvclient/kvcoord/txn_interceptor_heartbeater_test.go b/pkg/kv/kvclient/kvcoord/txn_interceptor_heartbeater_test.go index eae8e9748804..e3b0c54d92ab 100644 --- a/pkg/kv/kvclient/kvcoord/txn_interceptor_heartbeater_test.go +++ b/pkg/kv/kvclient/kvcoord/txn_interceptor_heartbeater_test.go @@ -216,9 +216,9 @@ func TestTxnHeartbeaterLoopStartedOnFirstLock(t *testing.T) { }) } -// TestTxnHeartbeaterLoopNotStartedFor1PC tests that the txnHeartbeater does -// not start a heartbeat loop if it detects a 1PC transaction. -func TestTxnHeartbeaterLoopNotStartedFor1PC(t *testing.T) { +// TestTxnHeartbeaterLoopStartedFor1PC tests that the txnHeartbeater +// starts a heartbeat loop if it detects a 1PC transaction. +func TestTxnHeartbeaterLoopStartedFor1PC(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) ctx := context.Background() @@ -238,7 +238,7 @@ func TestTxnHeartbeaterLoopNotStartedFor1PC(t *testing.T) { require.IsType(t, &roachpb.EndTxnRequest{}, ba.Requests[1].GetInner()) etReq := ba.Requests[1].GetInner().(*roachpb.EndTxnRequest) - require.False(t, etReq.TxnHeartbeating) + require.True(t, etReq.TxnHeartbeating) br := ba.CreateReply() br.Txn = ba.Txn @@ -250,8 +250,9 @@ func TestTxnHeartbeaterLoopNotStartedFor1PC(t *testing.T) { require.NotNil(t, br) th.mu.Lock() - require.False(t, th.mu.loopStarted) - require.False(t, th.heartbeatLoopRunningLocked()) + require.True(t, th.mu.loopStarted) + require.True(t, th.heartbeatLoopRunningLocked()) + th.closeLocked() th.mu.Unlock() } diff --git a/pkg/kv/kvserver/batcheval/transaction.go b/pkg/kv/kvserver/batcheval/transaction.go index 8ef18a9937c5..15c5706bdeb5 100644 --- a/pkg/kv/kvserver/batcheval/transaction.go +++ b/pkg/kv/kvserver/batcheval/transaction.go @@ -15,11 +15,9 @@ import ( "context" "fmt" - "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/storage" "github.com/cockroachdb/cockroach/pkg/storage/enginepb" - "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/errors" ) @@ -195,17 +193,3 @@ func SynthesizeTxnFromMeta( } return synth.AsTransaction() } - -// HasTxnRecord returns whether the provided transaction has a transaction -// record. The provided reader must come from the leaseholder of the transaction -// record's Range. -func HasTxnRecord( - ctx context.Context, reader storage.Reader, txn *roachpb.Transaction, -) (bool, error) { - key := keys.TransactionKey(txn.Key, txn.ID) - val, _, err := storage.MVCCGet(ctx, reader, key, hlc.Timestamp{}, storage.MVCCGetOptions{}) - if err != nil { - return false, err - } - return val != nil, nil -} diff --git a/pkg/kv/kvserver/replica_test.go b/pkg/kv/kvserver/replica_test.go index abdd6fd035a3..95cd47df620f 100644 --- a/pkg/kv/kvserver/replica_test.go +++ b/pkg/kv/kvserver/replica_test.go @@ -4728,7 +4728,7 @@ func TestRPCRetryProtectionInTxn(t *testing.T) { t.Fatalf("expected error, got nil") } require.Regexp(t, - `TransactionAbortedError\(ABORT_REASON_ALREADY_COMMITTED_OR_ROLLED_BACK_POSSIBLE_REPLAY\)`, + `TransactionAbortedError\(ABORT_REASON_RECORD_ALREADY_WRITTEN_POSSIBLE_REPLAY\)`, pErr) }) } @@ -4910,7 +4910,7 @@ func TestBatchRetryCantCommitIntents(t *testing.T) { // Heartbeat should fail with a TransactionAbortedError. _, pErr = tc.SendWrappedWith(hbH, &hb) - expErr := "TransactionAbortedError(ABORT_REASON_ALREADY_COMMITTED_OR_ROLLED_BACK_POSSIBLE_REPLAY)" + expErr := "TransactionAbortedError(ABORT_REASON_RECORD_ALREADY_WRITTEN_POSSIBLE_REPLAY)" if !testutils.IsPError(pErr, regexp.QuoteMeta(expErr)) { t.Errorf("expected %s; got %v", expErr, pErr) } @@ -11308,7 +11308,6 @@ func TestTxnRecordLifecycleTransitions(t *testing.T) { run: func(txn *roachpb.Transaction, _ hlc.Timestamp) error { et, etH := endTxnArgs(txn, true /* commit */) et.InFlightWrites = inFlightWrites - et.TxnHeartbeating = true return sendWrappedWithErr(etH, &et) }, expTxn: txnWithStagingStatusAndInFlightWrites, @@ -11321,7 +11320,6 @@ func TestTxnRecordLifecycleTransitions(t *testing.T) { }, run: func(txn *roachpb.Transaction, _ hlc.Timestamp) error { et, etH := endTxnArgs(txn, false /* commit */) - et.TxnHeartbeating = true return sendWrappedWithErr(etH, &et) }, // The transaction record will be eagerly GC-ed. @@ -11335,7 +11333,6 @@ func TestTxnRecordLifecycleTransitions(t *testing.T) { }, run: func(txn *roachpb.Transaction, _ hlc.Timestamp) error { et, etH := endTxnArgs(txn, true /* commit */) - et.TxnHeartbeating = true return sendWrappedWithErr(etH, &et) }, // The transaction record will be eagerly GC-ed. @@ -11349,7 +11346,6 @@ func TestTxnRecordLifecycleTransitions(t *testing.T) { }, run: func(txn *roachpb.Transaction, _ hlc.Timestamp) error { et, etH := endTxnArgs(txn, false /* commit */) - et.TxnHeartbeating = true return sendWrappedWithErr(etH, &et) }, expTxn: txnWithStatus(roachpb.ABORTED), @@ -11363,7 +11359,6 @@ func TestTxnRecordLifecycleTransitions(t *testing.T) { }, run: func(txn *roachpb.Transaction, _ hlc.Timestamp) error { et, etH := endTxnArgs(txn, true /* commit */) - et.TxnHeartbeating = true return sendWrappedWithErr(etH, &et) }, expTxn: txnWithStatus(roachpb.COMMITTED), @@ -11686,7 +11681,7 @@ func TestTxnRecordLifecycleTransitions(t *testing.T) { hb, hbH := heartbeatArgs(txn, now) return sendWrappedWithErr(hbH, &hb) }, - expError: "TransactionAbortedError(ABORT_REASON_ALREADY_COMMITTED_OR_ROLLED_BACK_POSSIBLE_REPLAY)", + expError: "TransactionAbortedError(ABORT_REASON_RECORD_ALREADY_WRITTEN_POSSIBLE_REPLAY)", expTxn: noTxnRecord, }, { @@ -11705,7 +11700,7 @@ func TestTxnRecordLifecycleTransitions(t *testing.T) { hb, hbH := heartbeatArgs(clone, now) return sendWrappedWithErr(hbH, &hb) }, - expError: "TransactionAbortedError(ABORT_REASON_ALREADY_COMMITTED_OR_ROLLED_BACK_POSSIBLE_REPLAY)", + expError: "TransactionAbortedError(ABORT_REASON_RECORD_ALREADY_WRITTEN_POSSIBLE_REPLAY)", expTxn: noTxnRecord, }, { @@ -11720,7 +11715,7 @@ func TestTxnRecordLifecycleTransitions(t *testing.T) { et.InFlightWrites = inFlightWrites return sendWrappedWithErr(etH, &et) }, - expError: "TransactionAbortedError(ABORT_REASON_ALREADY_COMMITTED_OR_ROLLED_BACK_POSSIBLE_REPLAY)", + expError: "TransactionAbortedError(ABORT_REASON_RECORD_ALREADY_WRITTEN_POSSIBLE_REPLAY)", expTxn: noTxnRecord, }, { @@ -11748,7 +11743,7 @@ func TestTxnRecordLifecycleTransitions(t *testing.T) { et, etH := endTxnArgs(txn, true /* commit */) return sendWrappedWithErr(etH, &et) }, - expError: "TransactionAbortedError(ABORT_REASON_ALREADY_COMMITTED_OR_ROLLED_BACK_POSSIBLE_REPLAY)", + expError: "TransactionAbortedError(ABORT_REASON_RECORD_ALREADY_WRITTEN_POSSIBLE_REPLAY)", expTxn: noTxnRecord, }, { @@ -11786,7 +11781,7 @@ func TestTxnRecordLifecycleTransitions(t *testing.T) { hb, hbH := heartbeatArgs(txn, now) return sendWrappedWithErr(hbH, &hb) }, - expError: "TransactionAbortedError(ABORT_REASON_ALREADY_COMMITTED_OR_ROLLED_BACK_POSSIBLE_REPLAY)", + expError: "TransactionAbortedError(ABORT_REASON_RECORD_ALREADY_WRITTEN_POSSIBLE_REPLAY)", expTxn: noTxnRecord, }, { @@ -11801,7 +11796,7 @@ func TestTxnRecordLifecycleTransitions(t *testing.T) { et.InFlightWrites = inFlightWrites return sendWrappedWithErr(etH, &et) }, - expError: "TransactionAbortedError(ABORT_REASON_ALREADY_COMMITTED_OR_ROLLED_BACK_POSSIBLE_REPLAY)", + expError: "TransactionAbortedError(ABORT_REASON_RECORD_ALREADY_WRITTEN_POSSIBLE_REPLAY)", expTxn: noTxnRecord, }, { @@ -11829,7 +11824,7 @@ func TestTxnRecordLifecycleTransitions(t *testing.T) { et, etH := endTxnArgs(txn, true /* commit */) return sendWrappedWithErr(etH, &et) }, - expError: "TransactionAbortedError(ABORT_REASON_ALREADY_COMMITTED_OR_ROLLED_BACK_POSSIBLE_REPLAY)", + expError: "TransactionAbortedError(ABORT_REASON_RECORD_ALREADY_WRITTEN_POSSIBLE_REPLAY)", expTxn: noTxnRecord, }, { @@ -12149,7 +12144,6 @@ func TestTxnRecordLifecycleTransitions(t *testing.T) { run: func(txn *roachpb.Transaction, _ hlc.Timestamp) error { et, etH := endTxnArgs(txn, true /* commit */) et.Sequence = 1 // qualify for 1PC - et.TxnHeartbeating = true return sendWrappedWithErr(etH, &et) }, expTxn: noTxnRecord, @@ -12166,7 +12160,6 @@ func TestTxnRecordLifecycleTransitions(t *testing.T) { run: func(txn *roachpb.Transaction, _ hlc.Timestamp) error { et, etH := endTxnArgs(txn, true /* commit */) et.Sequence = 1 // qualify for 1PC - et.TxnHeartbeating = true et.Require1PC = true return sendWrappedWithErr(etH, &et) }, @@ -12653,7 +12646,7 @@ func TestRollbackMissingTxnRecordNoError(t *testing.T) { // a retryable TransactionAbortedError, but if there's actually a sort of // replay at work and a client is still waiting for the error, the error would // be transformed into something more ambiguous on the way. - expErr := "TransactionAbortedError(ABORT_REASON_ALREADY_COMMITTED_OR_ROLLED_BACK_POSSIBLE_REPLAY)" + expErr := "TransactionAbortedError(ABORT_REASON_RECORD_ALREADY_WRITTEN_POSSIBLE_REPLAY)" if !testutils.IsPError(pErr, regexp.QuoteMeta(expErr)) { t.Errorf("expected %s; got %v", expErr, pErr) } diff --git a/pkg/kv/kvserver/replica_tscache.go b/pkg/kv/kvserver/replica_tscache.go index d1a7e6b34e85..975a7564db4f 100644 --- a/pkg/kv/kvserver/replica_tscache.go +++ b/pkg/kv/kvserver/replica_tscache.go @@ -108,18 +108,21 @@ func (r *Replica) updateTimestampCache( start, end := header.Key, header.EndKey switch t := args.(type) { case *roachpb.EndTxnRequest: - // EndTxn requests that finalize their transaction record a - // tombstone in the timestamp cache to ensure replays and concurrent - // requests aren't able to recreate the transaction record. + // EndTxn requests record a tombstone in the timestamp cache to ensure + // replays and concurrent requests aren't able to recreate the transaction + // record. // - // It inserts the timestamp of the final batch in the transaction. - // This timestamp must necessarily be equal to or greater than the - // transaction's MinTimestamp, which is consulted in - // CanCreateTxnRecord. - if br.Txn.Status.IsFinalized() { - key := transactionTombstoneMarker(start, txnID) - addToTSCache(key, nil, ts, txnID) - } + // It inserts the timestamp of the final batch in the transaction. This + // timestamp must necessarily be equal to or greater than the + // transaction's MinTimestamp, which is consulted in CanCreateTxnRecord. + key := transactionTombstoneMarker(start, txnID) + addToTSCache(key, nil, ts, txnID) + case *roachpb.HeartbeatTxnRequest: + // HeartbeatTxn requests record a tombstone entry when the record is + // initially written. This is used when considering potential 1PC + // evaluation, avoiding checking for a transaction record on disk. + key := transactionTombstoneMarker(start, txnID) + addToTSCache(key, nil, ts, txnID) case *roachpb.RecoverTxnRequest: // A successful RecoverTxn request may or may not have finalized the // transaction that it was trying to recover. If so, then we record @@ -375,6 +378,9 @@ func (r *Replica) applyTimestampCache( // never needs to explicitly create the transaction record for contending // transactions. // +// In addition, it is used when considering 1PC evaluation, to avoid checking +// for a transaction record on disk. +// // This is detailed in the transaction record state machine below: // // +----------------------------------------------------+ @@ -388,12 +394,13 @@ func (r *Replica) applyTimestampCache( // | v -> t = forward v by timestamp t | // +----------------------------------------------------+ // -// PushTxn(TIMESTAMP) HeartbeatTxn -// then: v1 -> push.ts then: update record -// +------+ +------+ -// PushTxn(ABORT) | | HeartbeatTxn | | PushTxn(TIMESTAMP) -// then: v2 -> txn.ts | v if: v2 < txn.orig | v then: update record -// +-----------------+ then: txn.ts -> v1 +--------------------+ +// HeartbeatTxn +// PushTxn(TIMESTAMP) then: update record +// then: v1 -> push.ts v2 -> txn.ts +// +------+ HeartbeatTxn +------+ +// PushTxn(ABORT) | | if: v2 < txn.orig | | PushTxn(TIMESTAMP) +// then: v2 -> txn.ts | v then: txn.ts -> v1 | v then: update record +// +-----------------+ v2 -> txn.ts +--------------------+ // +----| | else: fail | |----+ // | | |------------------------->| | | // | | no txn record | | txn record written | | @@ -500,22 +507,20 @@ func (r *Replica) CanCreateTxnRecord( minCommitTS, _ = r.store.tsCache.GetMax(pushKey, nil /* end */) // Also look in the timestamp cache to see if there is a tombstone entry for - // this transaction, which would indicate this transaction has already been - // finalized or was already aborted by a concurrent transaction. If there is - // an entry, then we return a retriable error: if this is a re-evaluation, - // then the error will be transformed into an ambiguous one higher up. - // Otherwise, if the client is still waiting for a result, then this cannot - // be a "replay" of any sort. - tombstoneTimestamp, tombstomeTxnID := r.store.tsCache.GetMax(tombstoneKey, nil /* end */) + // this transaction, which indicates that this transaction has already written + // a transaction record. If there is an entry, then we return a retriable + // error: if this is a re-evaluation, then the error will be transformed into + // an ambiguous one higher up. Otherwise, if the client is still waiting for + // a result, then this cannot be a "replay" of any sort. + tombstoneTimestamp, tombstoneTxnID := r.store.tsCache.GetMax(tombstoneKey, nil /* end */) // Compare against the minimum timestamp that the transaction could have // written intents at. if txnMinTS.LessEq(tombstoneTimestamp) { - switch tombstomeTxnID { + switch tombstoneTxnID { case txnID: - // If we find our own transaction ID then an EndTxn request sent by - // our coordinator has already been processed. We might be a replay (e.g. - // a DistSender retry), or we raced with an asynchronous abort. Either - // way, return an error. + // If we find our own transaction ID then a transaction record has already + // been written. We might be a replay (e.g. a DistSender retry), or we + // raced with an asynchronous abort. Either way, return an error. // // TODO(andrei): We could keep a bit more info in the tscache to return a // different error for COMMITTED transactions. If the EndTxn(commit) was @@ -525,7 +530,7 @@ func (r *Replica) CanCreateTxnRecord( // still have trouble reconstructing the result, but at least it could // provide a non-ambiguous error to the application. return false, hlc.Timestamp{}, - roachpb.ABORT_REASON_ALREADY_COMMITTED_OR_ROLLED_BACK_POSSIBLE_REPLAY + roachpb.ABORT_REASON_RECORD_ALREADY_WRITTEN_POSSIBLE_REPLAY case uuid.Nil: lease, _ /* nextLease */ := r.GetLease() // Recognize the case where a lease started recently. Lease transfers bump @@ -547,16 +552,17 @@ func (r *Replica) CanCreateTxnRecord( } // transactionTombstoneMarker returns the key used as a marker indicating that a -// particular txn was finalized (i.e. by an EndTransaction, RecoverTxn or -// PushTxn(Abort)). It is used as a marker in the timestamp cache serving as a -// guard against creating a transaction record after the transaction record has -// been cleaned up (i.e. by a BeginTxn being evaluated out of order or arriving -// after another txn Push(Abort)'ed the txn). +// particular txn has written a transaction record (which may or may not still +// exist). It serves as a guard against recreating a transaction record after it +// has been cleaned up (i.e. by a BeginTxn being evaluated out of order or +// arriving after another txn Push(Abort)'ed the txn). It is also used to check +// for existing txn records when considering 1PC evaluation without hitting +// disk. func transactionTombstoneMarker(key roachpb.Key, txnID uuid.UUID) roachpb.Key { return append(keys.TransactionKey(key, txnID), []byte("-tmbs")...) } -// transactionPushMarker returns the key used by the marker indicating that a +// transactionPushMarker returns the key used as a marker indicating that a // particular txn was pushed before writing its transaction record. It is used // as a marker in the timestamp cache indicating that the transaction was pushed // in case the push happens before there's a transaction record. diff --git a/pkg/kv/kvserver/replica_write.go b/pkg/kv/kvserver/replica_write.go index a4bbacc2f64f..e40fd58cb793 100644 --- a/pkg/kv/kvserver/replica_write.go +++ b/pkg/kv/kvserver/replica_write.go @@ -391,9 +391,9 @@ support contract. Otherwise, please open an issue at: // executed as 1PC. func (r *Replica) canAttempt1PCEvaluation( ctx context.Context, ba *roachpb.BatchRequest, latchSpans *spanset.SpanSet, -) (bool, *roachpb.Error) { +) bool { if !isOnePhaseCommit(ba) { - return false, nil + return false } if ba.Timestamp != ba.Txn.WriteTimestamp { @@ -401,42 +401,25 @@ func (r *Replica) canAttempt1PCEvaluation( ba.Timestamp, ba.Txn.WriteTimestamp) } - // Check whether the txn record has already been created. If so, we can't - // perform a 1PC evaluation because we need to clean up the record during - // evaluation. - // - // We only perform this check if the transaction's EndTxn indicates that it - // has started its heartbeat loop. If not, the transaction cannot have an - // existing record. However, we perform it unconditionally under race to - // catch bugs. - arg, _ := ba.GetArg(roachpb.EndTxn) - etArg := arg.(*roachpb.EndTxnRequest) - if etArg.TxnHeartbeating || util.RaceEnabled { - if ok, err := batcheval.HasTxnRecord(ctx, r.store.Engine(), ba.Txn); err != nil { - return false, roachpb.NewError(err) - } else if ok { - if !etArg.TxnHeartbeating { - log.Fatalf(ctx, "non-heartbeating txn with txn record before EndTxn: %v", ba.Txn) - } - return false, nil - } - } - // The EndTxn checks whether the txn record can be created, but we're // eliding the EndTxn. So, we'll do the check instead. - ok, minCommitTS, reason := r.CanCreateTxnRecord(ctx, ba.Txn.ID, ba.Txn.Key, ba.Txn.MinTimestamp) + // + // Note that the returned reason does not distinguish between an existing + // record (which should fall back to non-1PC EndTxn evaluation) and a + // finalized record (which should return an error), so we ignore it here and + // let EndTxn return an error as appropriate. This lets us avoid a disk read + // to check for an existing record. + ok, minCommitTS, _ := r.CanCreateTxnRecord(ctx, ba.Txn.ID, ba.Txn.Key, ba.Txn.MinTimestamp) if !ok { - newTxn := ba.Txn.Clone() - newTxn.Status = roachpb.ABORTED - return false, roachpb.NewErrorWithTxn(roachpb.NewTransactionAbortedError(reason), newTxn) + return false } if ba.Timestamp.Less(minCommitTS) { ba.Txn.WriteTimestamp = minCommitTS // We can only evaluate at the new timestamp if we manage to bump the read // timestamp. - return maybeBumpReadTimestampToWriteTimestamp(ctx, ba, latchSpans), nil + return maybeBumpReadTimestampToWriteTimestamp(ctx, ba, latchSpans) } - return true, nil + return true } // evaluateWriteBatch evaluates the supplied batch. @@ -464,11 +447,7 @@ func (r *Replica) evaluateWriteBatch( // Attempt 1PC execution, if applicable. If not transactional or there are // indications that the batch's txn will require retry, execute as normal. - ok, pErr := r.canAttempt1PCEvaluation(ctx, ba, latchSpans) - if pErr != nil { - return nil, enginepb.MVCCStats{}, nil, result.Result{}, pErr - } - if ok { + if r.canAttempt1PCEvaluation(ctx, ba, latchSpans) { res := r.evaluate1PC(ctx, idKey, ba, latchSpans, lockSpans) switch res.success { case onePCSucceeded: @@ -613,11 +592,6 @@ func (r *Replica) evaluate1PC( // have acquired unreplicated locks, so inform the concurrency manager that // it is finalized and than any unreplicated locks that it has acquired can // be released. - // - // TODO(nvanbenschoten): once we can rely on EndTxn.TxnHeartbeating being - // correct in v21.1, we can gate these notifications on TxnHeartbeating - // because we know that a transaction hasn't acquired any unreplicated - // locks if it hasn't started heartbeating. res.Local.UpdatedTxns = []*roachpb.Transaction{clonedTxn} res.Local.ResolvedLocks = make([]roachpb.LockUpdate, len(etArg.LockSpans)) for i, sp := range etArg.LockSpans { diff --git a/pkg/roachpb/api.go b/pkg/roachpb/api.go index 9f183e99b1a7..1ab5f3495193 100644 --- a/pkg/roachpb/api.go +++ b/pkg/roachpb/api.go @@ -1211,9 +1211,12 @@ func (*AdminMergeRequest) flags() int { return isAdmin | isAlone } func (*AdminTransferLeaseRequest) flags() int { return isAdmin | isAlone } func (*AdminChangeReplicasRequest) flags() int { return isAdmin | isAlone } func (*AdminRelocateRangeRequest) flags() int { return isAdmin | isAlone } -func (*HeartbeatTxnRequest) flags() int { return isWrite | isTxn } func (*GCRequest) flags() int { return isWrite | isRange } +// HeartbeatTxn updates the timestamp cache with transaction records, +// to avoid checking for them on disk when considering 1PC evaluation. +func (*HeartbeatTxnRequest) flags() int { return isWrite | isTxn | updatesTSCache } + // PushTxnRequest updates different marker keys in the timestamp cache when // pushing a transaction's timestamp and when aborting a transaction. func (*PushTxnRequest) flags() int { diff --git a/pkg/roachpb/api.pb.go b/pkg/roachpb/api.pb.go index f26799871e45..5ae49e2cb942 100644 --- a/pkg/roachpb/api.pb.go +++ b/pkg/roachpb/api.pb.go @@ -1745,6 +1745,10 @@ type EndTxnRequest struct { // // WARNING: this flag was introduced in the v20.2 release. v20.1 nodes may not // set this properly so it should not be relied upon for correctness. + // + // TODO(erikgrinaker): This flag is no longer used in v21.2 as we now always + // heartbeat the txn record, but we still need to set it when interacting with + // v21.1 nodes. It should be removed in v22.1. TxnHeartbeating bool `protobuf:"varint,10,opt,name=txn_heartbeating,json=txnHeartbeating,proto3" json:"txn_heartbeating,omitempty"` } diff --git a/pkg/roachpb/api.proto b/pkg/roachpb/api.proto index bc4c9e5fd546..a63fbd2f8f2e 100644 --- a/pkg/roachpb/api.proto +++ b/pkg/roachpb/api.proto @@ -670,6 +670,10 @@ message EndTxnRequest { // // WARNING: this flag was introduced in the v20.2 release. v20.1 nodes may not // set this properly so it should not be relied upon for correctness. + // + // TODO(erikgrinaker): This flag is no longer used in v21.2 as we now always + // heartbeat the txn record, but we still need to set it when interacting with + // v21.1 nodes. It should be removed in v22.1. bool txn_heartbeating = 10; reserved 7, 8; } diff --git a/pkg/roachpb/errors.pb.go b/pkg/roachpb/errors.pb.go index a6c5c0925740..bf2ded8e1d55 100644 --- a/pkg/roachpb/errors.pb.go +++ b/pkg/roachpb/errors.pb.go @@ -73,12 +73,13 @@ const ( ABORT_REASON_ABORT_SPAN TransactionAbortedReason = 5 // A request attempting to create a transaction record encountered a write // timestamp cache entry for the txn key, and the entry identifies this - // transaction. This means that the transaction definitely committed or rolled - // back before. So, this request is either a delayed replay of some sort, or - // it raced with an async abort and lost. If a client gets this - // TransactionAbortedError (without it being wrapped in an ambiguous error), - // it must be the latter case, and the transaction can be retried. - ABORT_REASON_ALREADY_COMMITTED_OR_ROLLED_BACK_POSSIBLE_REPLAY TransactionAbortedReason = 6 + // transaction. This means that the transaction already wrote a transaction + // record previously, which may or may not have been removed since. So, this + // request is either a delayed replay of some sort, or it raced with an async + // abort and lost. If a client gets this TransactionAbortedError (without it + // being wrapped in an ambiguous error), it must be the latter case, and the + // transaction can be retried. + ABORT_REASON_RECORD_ALREADY_WRITTEN_POSSIBLE_REPLAY TransactionAbortedReason = 6 // A request attempting to create a transaction record is not allowed to // proceed by the timestamp cache because it cannot be verified that the // respective transaction record did not previously exist. As opposed to the @@ -108,20 +109,20 @@ var TransactionAbortedReason_name = map[int32]string{ 3: "ABORT_REASON_CLIENT_REJECT", 4: "ABORT_REASON_PUSHER_ABORTED", 5: "ABORT_REASON_ABORT_SPAN", - 6: "ABORT_REASON_ALREADY_COMMITTED_OR_ROLLED_BACK_POSSIBLE_REPLAY", + 6: "ABORT_REASON_RECORD_ALREADY_WRITTEN_POSSIBLE_REPLAY", 8: "ABORT_REASON_NEW_LEASE_PREVENTS_TXN", 7: "ABORT_REASON_TIMESTAMP_CACHE_REJECTED", } var TransactionAbortedReason_value = map[string]int32{ - "ABORT_REASON_UNKNOWN": 0, - "ABORT_REASON_ABORTED_RECORD_FOUND": 1, - "ABORT_REASON_CLIENT_REJECT": 3, - "ABORT_REASON_PUSHER_ABORTED": 4, - "ABORT_REASON_ABORT_SPAN": 5, - "ABORT_REASON_ALREADY_COMMITTED_OR_ROLLED_BACK_POSSIBLE_REPLAY": 6, - "ABORT_REASON_NEW_LEASE_PREVENTS_TXN": 8, - "ABORT_REASON_TIMESTAMP_CACHE_REJECTED": 7, + "ABORT_REASON_UNKNOWN": 0, + "ABORT_REASON_ABORTED_RECORD_FOUND": 1, + "ABORT_REASON_CLIENT_REJECT": 3, + "ABORT_REASON_PUSHER_ABORTED": 4, + "ABORT_REASON_ABORT_SPAN": 5, + "ABORT_REASON_RECORD_ALREADY_WRITTEN_POSSIBLE_REPLAY": 6, + "ABORT_REASON_NEW_LEASE_PREVENTS_TXN": 8, + "ABORT_REASON_TIMESTAMP_CACHE_REJECTED": 7, } func (x TransactionAbortedReason) Enum() *TransactionAbortedReason { @@ -2198,203 +2199,203 @@ func init() { func init() { proto.RegisterFile("roachpb/errors.proto", fileDescriptor_123941c6716fd549) } var fileDescriptor_123941c6716fd549 = []byte{ - // 3125 bytes of a gzipped FileDescriptorProto + // 3121 bytes of a gzipped FileDescriptorProto 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x9c, 0x59, 0xcb, 0x6f, 0x1b, 0xd7, 0xb9, 0x27, 0x25, 0x4a, 0xa2, 0x3e, 0x3d, 0x3c, 0x3a, 0x92, 0xa5, 0x91, 0x6c, 0x53, 0xca, 0xd8, - 0x4e, 0x6c, 0x07, 0x91, 0x2e, 0x9c, 0x1b, 0xe0, 0x26, 0xd7, 0x59, 0xf0, 0x31, 0x12, 0x29, 0xf1, - 0xa1, 0x3b, 0xa4, 0xfc, 0x88, 0x11, 0x9c, 0x8c, 0x38, 0x47, 0xd4, 0x5c, 0x0f, 0x67, 0x98, 0x33, + 0x4e, 0x6c, 0x07, 0x91, 0x2e, 0x9c, 0x1b, 0xdc, 0x9b, 0x5c, 0xdf, 0x05, 0x1f, 0x23, 0x91, 0x12, + 0x1f, 0xea, 0x90, 0xf2, 0x23, 0x46, 0x70, 0x32, 0xe2, 0x1c, 0x51, 0x53, 0x0f, 0x67, 0x98, 0x33, 0x43, 0x59, 0x02, 0xba, 0x28, 0xda, 0x4d, 0xd0, 0x02, 0x7d, 0xac, 0xda, 0x65, 0x81, 0xa0, 0x8b, - 0x02, 0x45, 0xd1, 0xbf, 0xa0, 0xeb, 0x2c, 0xb3, 0x0c, 0x8a, 0xc2, 0x68, 0xed, 0xfe, 0x15, 0x59, - 0x15, 0xe7, 0x31, 0xe4, 0x90, 0x1c, 0xca, 0x4a, 0x76, 0x9c, 0xef, 0x75, 0xbe, 0xf3, 0xfa, 0x7d, - 0xbf, 0xef, 0x10, 0x56, 0xa8, 0x67, 0x36, 0x4f, 0x3b, 0xc7, 0x3b, 0x84, 0x52, 0x8f, 0xfa, 0xdb, - 0x1d, 0xea, 0x05, 0x1e, 0x5a, 0x6a, 0x7a, 0xcd, 0x17, 0x5c, 0xb3, 0xfd, 0xe2, 0x6c, 0xfb, 0xc5, - 0x59, 0xe7, 0x78, 0xe3, 0xba, 0x30, 0x18, 0xb2, 0xdc, 0x40, 0xa1, 0xbf, 0x65, 0x06, 0xa6, 0x94, - 0xad, 0x86, 0xb2, 0x36, 0x09, 0xcc, 0x88, 0x5c, 0xed, 0x06, 0xb6, 0xb3, 0x73, 0xea, 0x34, 0x77, - 0x02, 0xbb, 0x4d, 0xfc, 0xc0, 0x6c, 0x77, 0xa4, 0x66, 0xa5, 0xe5, 0xb5, 0x3c, 0xfe, 0x73, 0x87, - 0xfd, 0x12, 0x52, 0xed, 0xaf, 0x13, 0xb0, 0x5c, 0xf5, 0x82, 0x32, 0x31, 0x7d, 0x52, 0xf4, 0x1c, - 0x8b, 0x50, 0x9d, 0x0d, 0x8d, 0x0a, 0x30, 0x43, 0x49, 0xc7, 0xb1, 0x9b, 0xa6, 0x9a, 0xdc, 0x4a, - 0xde, 0x9b, 0x7b, 0x78, 0x67, 0xbb, 0x9f, 0xaf, 0x1c, 0x7b, 0xdb, 0x10, 0x16, 0x05, 0xe2, 0x37, - 0xa9, 0xdd, 0x09, 0x3c, 0x9a, 0x4b, 0x7d, 0xf3, 0x6a, 0x33, 0x61, 0x84, 0xae, 0x68, 0x0f, 0xe6, - 0x1d, 0x16, 0x19, 0x9f, 0xf2, 0xd0, 0xea, 0xc4, 0xd5, 0x43, 0x19, 0x73, 0x4e, 0x3f, 0x27, 0xb4, - 0x0d, 0x53, 0xfc, 0x53, 0x4d, 0xf1, 0x08, 0x6a, 0x4c, 0x04, 0x3e, 0x05, 0x43, 0x98, 0xa1, 0x8f, - 0x20, 0x4d, 0x4d, 0xb7, 0x45, 0xb0, 0x6d, 0xa9, 0x93, 0x5b, 0xc9, 0x7b, 0x93, 0xb9, 0x0d, 0x96, - 0xd9, 0xeb, 0x57, 0x9b, 0x33, 0x06, 0x93, 0x97, 0x0a, 0xdf, 0xf7, 0x7f, 0x1a, 0x33, 0xdc, 0xb6, - 0x64, 0xa1, 0xdb, 0x00, 0xcd, 0xae, 0x1f, 0x78, 0x6d, 0xdc, 0xf6, 0x5b, 0xea, 0xd4, 0x56, 0xf2, - 0xde, 0xac, 0x9c, 0xd2, 0xac, 0x90, 0x57, 0xfc, 0x96, 0xb6, 0x0a, 0x2b, 0x55, 0xcf, 0x22, 0x47, - 0xae, 0x79, 0x66, 0xda, 0x8e, 0x79, 0xec, 0x10, 0xbe, 0x64, 0xda, 0x3a, 0xac, 0x1d, 0xb9, 0x7e, - 0xb7, 0xd3, 0xf1, 0x68, 0x40, 0x2c, 0x83, 0x7c, 0xd9, 0x25, 0x7e, 0x20, 0x54, 0x3f, 0x4b, 0x02, - 0xe2, 0x83, 0x55, 0xbd, 0x60, 0xd7, 0xeb, 0xba, 0x96, 0x58, 0xe4, 0x68, 0x96, 0xc9, 0xab, 0x67, - 0xf9, 0x11, 0xa4, 0xfd, 0xc0, 0xa3, 0xdc, 0x6d, 0x62, 0xd0, 0xad, 0xce, 0xe4, 0xc2, 0x4d, 0xfe, - 0x34, 0x66, 0xb8, 0x6d, 0xc9, 0xd2, 0x7e, 0x37, 0x09, 0xd7, 0x79, 0xac, 0x03, 0x72, 0x51, 0xb1, - 0xfd, 0xb6, 0x19, 0x34, 0x4f, 0x45, 0x1e, 0x1f, 0xc2, 0x12, 0x15, 0xe9, 0x62, 0x3f, 0x30, 0x69, - 0x80, 0x5f, 0x90, 0x0b, 0x9e, 0xd0, 0x7c, 0x6e, 0xe6, 0xfb, 0x57, 0x9b, 0x93, 0x07, 0xe4, 0xc2, - 0xb8, 0x26, 0x2d, 0xea, 0xcc, 0xe0, 0x80, 0x5c, 0xa0, 0x1d, 0x08, 0x45, 0x98, 0xb8, 0x16, 0x77, - 0x99, 0x18, 0x74, 0x59, 0x90, 0x7a, 0xdd, 0xb5, 0x98, 0xc3, 0x29, 0xdc, 0xb0, 0x48, 0x87, 0x92, - 0xa6, 0x19, 0x10, 0x0b, 0xb7, 0x65, 0x06, 0xc4, 0xc2, 0x7c, 0x5e, 0x7c, 0x9b, 0xe6, 0x1e, 0x6a, - 0x71, 0x67, 0x83, 0xe9, 0x47, 0x0e, 0xd9, 0x7a, 0x3f, 0x58, 0xa5, 0x17, 0x8b, 0x9b, 0xa2, 0x2f, - 0x60, 0x23, 0x32, 0x92, 0xdf, 0x6d, 0xb5, 0x88, 0x1f, 0xf4, 0x06, 0x4a, 0x5d, 0x75, 0x20, 0x43, - 0xed, 0x47, 0xa9, 0x87, 0x41, 0xc4, 0x08, 0x65, 0x98, 0xe6, 0xc1, 0x7c, 0x75, 0x6a, 0x6b, 0xf2, - 0xde, 0xdc, 0xc3, 0x9b, 0xe3, 0xa2, 0x95, 0xdc, 0x13, 0x2f, 0xb7, 0x2a, 0xb7, 0x67, 0x51, 0xf8, - 0x94, 0xdc, 0x80, 0x50, 0xd7, 0x74, 0x0c, 0x19, 0x43, 0x7b, 0x33, 0x09, 0x9a, 0x41, 0x4c, 0xeb, - 0x89, 0x1d, 0x9c, 0xda, 0xee, 0x91, 0xdb, 0x24, 0x34, 0x30, 0x6d, 0x37, 0xb8, 0xe0, 0x96, 0x67, - 0xa6, 0x23, 0xb6, 0x69, 0x1f, 0x16, 0x29, 0x31, 0x2d, 0xdc, 0xbb, 0xd9, 0xf2, 0x6a, 0xde, 0x8a, - 0x0c, 0xce, 0xae, 0xff, 0xf6, 0xa9, 0xd3, 0xdc, 0x6e, 0x84, 0x46, 0x72, 0xb9, 0x16, 0x98, 0x6b, - 0x4f, 0x88, 0x0c, 0x40, 0xe4, 0xdc, 0xf6, 0x03, 0xdb, 0x6d, 0x45, 0xe2, 0x4d, 0x5c, 0x3d, 0xde, - 0x52, 0xe8, 0xde, 0x8f, 0xf9, 0x1c, 0xd6, 0x1c, 0xaf, 0x69, 0x3a, 0xb8, 0xdb, 0x9f, 0x01, 0x76, - 0xec, 0xb6, 0x1d, 0xf0, 0xab, 0x74, 0xc5, 0xc0, 0xd7, 0x79, 0x8c, 0xc8, 0x22, 0x94, 0x59, 0x04, - 0xf4, 0x39, 0xa8, 0x2d, 0xc7, 0x3b, 0x8e, 0x8d, 0x3e, 0x79, 0xf5, 0xe8, 0xab, 0x22, 0xc8, 0x48, - 0xf8, 0xe7, 0xb0, 0xec, 0x1d, 0xfb, 0x84, 0x9e, 0x91, 0xc8, 0xfa, 0xfa, 0x6a, 0x8a, 0xef, 0x6e, - 0x1c, 0x60, 0xd5, 0xa4, 0xf5, 0xf0, 0x00, 0xc8, 0x1b, 0x56, 0xf8, 0x9f, 0xa4, 0x7e, 0xff, 0x87, - 0xcd, 0x84, 0x66, 0xc1, 0x5a, 0x83, 0x9a, 0xae, 0x6f, 0x36, 0x03, 0xdb, 0x73, 0xb3, 0xc7, 0x1c, - 0x26, 0xc4, 0xce, 0x96, 0x60, 0x9a, 0x12, 0xd3, 0xf7, 0x5c, 0xbe, 0xa3, 0x8b, 0x0f, 0xdf, 0xdf, - 0x1e, 0x29, 0x0e, 0xdb, 0xa3, 0xbe, 0x06, 0x77, 0x91, 0xe3, 0xca, 0x00, 0xda, 0x73, 0x58, 0x89, - 0x58, 0x1e, 0x76, 0x7d, 0x79, 0xc7, 0xf3, 0x00, 0x9d, 0xae, 0x7f, 0x4a, 0x08, 0x0e, 0xce, 0x5d, - 0x79, 0x70, 0x32, 0x31, 0xf3, 0x8a, 0x38, 0x87, 0xd0, 0x27, 0xfc, 0x1a, 0xe7, 0xae, 0xf6, 0xf3, - 0x24, 0x5c, 0x8f, 0x18, 0x18, 0x24, 0xa0, 0x17, 0x22, 0xfc, 0xde, 0xd0, 0x0c, 0xee, 0x5f, 0x3e, - 0x03, 0xee, 0x19, 0x97, 0x3f, 0x7a, 0x07, 0x66, 0xc9, 0x79, 0x40, 0x4d, 0x8e, 0xc0, 0x13, 0x11, - 0x04, 0x4e, 0x73, 0x31, 0x03, 0xe0, 0xbf, 0x25, 0x61, 0x35, 0x12, 0xab, 0x1e, 0x98, 0x41, 0xd7, - 0x17, 0x69, 0xac, 0xc2, 0x24, 0xf3, 0x4b, 0x46, 0xfc, 0x98, 0x00, 0x55, 0x7b, 0xe9, 0x4d, 0xf0, - 0xf4, 0xfe, 0xeb, 0xf2, 0xf4, 0x22, 0x21, 0xb7, 0x63, 0x57, 0xf9, 0x11, 0x4c, 0x0b, 0x39, 0x42, - 0xb0, 0x68, 0xe8, 0xd9, 0x7a, 0xad, 0x8a, 0x8f, 0xaa, 0x07, 0xd5, 0xda, 0x93, 0xaa, 0x92, 0x40, - 0x2a, 0xac, 0x48, 0x59, 0xe3, 0x69, 0x15, 0xe7, 0x6b, 0x95, 0x4a, 0xa9, 0xd1, 0xd0, 0x0b, 0xca, - 0x84, 0x96, 0x4a, 0x27, 0x95, 0xa4, 0xf6, 0xdb, 0x24, 0x28, 0x4f, 0xa8, 0x1d, 0x10, 0x76, 0xbf, - 0x5d, 0x51, 0x23, 0xd0, 0xc7, 0x30, 0x63, 0xf3, 0x4f, 0x5f, 0x4d, 0xf2, 0x53, 0xb7, 0x1e, 0xb3, - 0x3b, 0xc2, 0x21, 0x2c, 0xb3, 0xd2, 0x1e, 0x3d, 0x82, 0x45, 0x51, 0x66, 0x7d, 0x06, 0xb8, 0x6e, - 0x93, 0xc8, 0x9a, 0x77, 0x9d, 0x99, 0x7d, 0xff, 0x6a, 0x73, 0x81, 0xd7, 0xc6, 0xba, 0x54, 0x1a, - 0x0b, 0x4e, 0xf4, 0x73, 0x3f, 0x95, 0x9e, 0x50, 0x26, 0xb5, 0x3f, 0x86, 0x39, 0x35, 0x3c, 0xaf, - 0xe6, 0xc8, 0x73, 0x99, 0x85, 0xd9, 0x1f, 0x05, 0x36, 0x7d, 0x2f, 0x54, 0x05, 0xc5, 0x6c, 0x06, - 0x5d, 0xd3, 0xf9, 0x71, 0x30, 0x73, 0x4d, 0x38, 0xf7, 0xc4, 0xda, 0x0a, 0xa0, 0x5a, 0x87, 0x15, - 0x57, 0x9b, 0x12, 0xbf, 0x71, 0xee, 0x8a, 0x02, 0x5b, 0x87, 0x95, 0xbc, 0xe7, 0x5a, 0x36, 0xdb, - 0xbc, 0x5d, 0xd3, 0x76, 0xc2, 0x8b, 0xf5, 0xbf, 0x30, 0x2f, 0x47, 0x3f, 0x33, 0x9d, 0x2e, 0x91, - 0x73, 0x88, 0xa3, 0x0f, 0x8f, 0x99, 0xde, 0x98, 0x13, 0xd6, 0xfc, 0x43, 0xfb, 0x4b, 0x12, 0x90, - 0x60, 0x15, 0xe4, 0xff, 0x49, 0xb3, 0x77, 0x59, 0x33, 0x30, 0xd3, 0x26, 0xbe, 0x6f, 0xb6, 0xc8, - 0xc0, 0x39, 0x0b, 0x85, 0xe8, 0x11, 0xcc, 0xca, 0xc2, 0x47, 0x2c, 0x39, 0xd5, 0xb1, 0x7c, 0x25, - 0x5c, 0xaf, 0x9e, 0x03, 0xfa, 0x04, 0xd2, 0x21, 0xb2, 0x4a, 0x5c, 0x7b, 0x9b, 0x73, 0xcf, 0x5e, - 0xfb, 0x12, 0x56, 0xb2, 0xed, 0x63, 0xbb, 0xd5, 0xf5, 0xba, 0xbe, 0x41, 0xfc, 0xae, 0x13, 0x5c, - 0x2d, 0xe3, 0x8f, 0x61, 0xee, 0x25, 0x35, 0x3b, 0x1d, 0x62, 0x61, 0x42, 0x69, 0x4c, 0xce, 0xe1, - 0x15, 0xe1, 0xe1, 0x0c, 0x90, 0xc6, 0x3a, 0xa5, 0xda, 0x1a, 0xe3, 0x14, 0x27, 0xc1, 0x1e, 0xf5, - 0xba, 0x9d, 0x02, 0x71, 0x48, 0xb8, 0x4a, 0x1a, 0x86, 0x55, 0xc9, 0xe9, 0xf2, 0x1e, 0xa5, 0xdd, - 0x0e, 0xdb, 0x19, 0x91, 0x0d, 0xbb, 0xe1, 0xec, 0x07, 0x1e, 0xbe, 0xa9, 0x69, 0x2e, 0xae, 0xf8, - 0x2d, 0xa4, 0xc1, 0x6c, 0x87, 0x7a, 0x4d, 0xe2, 0xfb, 0x72, 0x09, 0xd3, 0x3d, 0x2c, 0x0a, 0xc5, - 0x5a, 0x1d, 0x90, 0x1c, 0x20, 0x7a, 0x62, 0x3f, 0x05, 0x90, 0xe4, 0x33, 0x24, 0x55, 0x53, 0xb9, - 0x8c, 0x2c, 0xbf, 0xb3, 0xd2, 0x9e, 0xf3, 0xa3, 0xfe, 0x07, 0x5b, 0x7d, 0xf1, 0xd3, 0xd2, 0x0e, - 0x00, 0x71, 0xde, 0x34, 0xc2, 0xd3, 0x7a, 0x84, 0x2b, 0x79, 0x75, 0xc2, 0x55, 0x67, 0x84, 0xf0, - 0xd4, 0x74, 0x2d, 0x87, 0x61, 0x75, 0x40, 0x2f, 0x7a, 0x5c, 0x11, 0x3d, 0x84, 0x54, 0x47, 0xa7, - 0x34, 0xe6, 0x3c, 0x0e, 0x2c, 0xb5, 0x9c, 0x35, 0xb7, 0x95, 0x55, 0xe4, 0xdf, 0x49, 0xb8, 0x3b, - 0x0c, 0xa4, 0x8c, 0x37, 0x1c, 0x32, 0x3a, 0x6f, 0x90, 0x13, 0x4a, 0x42, 0xc4, 0x1f, 0x87, 0x85, - 0xcf, 0x61, 0x3a, 0x38, 0x77, 0x43, 0xf2, 0x38, 0x9f, 0x2b, 0x30, 0xd5, 0xdf, 0x5f, 0x6d, 0x7e, - 0xd8, 0xb2, 0x83, 0xd3, 0xee, 0xf1, 0x76, 0xd3, 0x6b, 0xef, 0xf4, 0xf2, 0xb1, 0x8e, 0xfb, 0xbf, - 0x77, 0x3a, 0x2f, 0x5a, 0x3b, 0xbc, 0xbf, 0xe8, 0x76, 0x6d, 0x6b, 0xfb, 0xe8, 0xa8, 0x54, 0x78, - 0xfd, 0x6a, 0x73, 0xaa, 0x71, 0xee, 0x96, 0x0a, 0xc6, 0x54, 0x70, 0xee, 0x96, 0x2c, 0xb4, 0x0b, - 0x73, 0x41, 0x3f, 0x3b, 0x79, 0x82, 0xaf, 0x56, 0x67, 0xa2, 0x8e, 0xda, 0x2e, 0x6c, 0x36, 0xce, - 0xdd, 0xac, 0xc3, 0x58, 0xcb, 0x85, 0xee, 0x36, 0xbd, 0x2e, 0xa3, 0x42, 0xf2, 0x70, 0x89, 0xf9, - 0xdd, 0x06, 0xe8, 0x50, 0x72, 0x86, 0xf9, 0xa9, 0x19, 0x98, 0xe6, 0x2c, 0x93, 0x8b, 0x63, 0xf8, - 0xcb, 0x24, 0xac, 0x30, 0xd0, 0x6c, 0x11, 0x5a, 0x3b, 0x23, 0xf4, 0xc4, 0xf1, 0x5e, 0x0a, 0xef, - 0x75, 0x98, 0x8c, 0x61, 0xb9, 0x4c, 0x86, 0xee, 0xc3, 0x42, 0xb3, 0x4b, 0x29, 0x71, 0x03, 0x89, - 0x1a, 0x82, 0x64, 0x8b, 0xd8, 0xf3, 0x52, 0xc5, 0x21, 0x02, 0x7d, 0x00, 0xd7, 0x6c, 0xb7, 0x49, - 0x49, 0xbb, 0x6f, 0x3c, 0x19, 0x31, 0x5e, 0xec, 0x29, 0x05, 0xa2, 0x7c, 0x9d, 0x84, 0x1b, 0x39, - 0xc6, 0x54, 0xfb, 0x30, 0x47, 0x4e, 0x3c, 0x4a, 0xf6, 0xf2, 0x3d, 0xbc, 0x6d, 0xfc, 0x28, 0xbc, - 0xed, 0x93, 0x30, 0x16, 0xe2, 0x94, 0x1d, 0x02, 0xcf, 0xb1, 0x7e, 0x08, 0xd0, 0xf6, 0xbd, 0xb4, - 0x36, 0x20, 0x51, 0x67, 0x2a, 0xb6, 0xef, 0xdb, 0x6e, 0x4b, 0xe4, 0xf6, 0x08, 0xe6, 0x5f, 0x52, - 0xcf, 0x6d, 0x61, 0x51, 0x75, 0x64, 0x7a, 0xe3, 0x8b, 0x94, 0x31, 0xc7, 0xcd, 0xc5, 0x47, 0xb8, - 0xdc, 0x13, 0xa3, 0xcb, 0xcd, 0xfa, 0xa9, 0x0a, 0xa1, 0x8c, 0x2a, 0x1f, 0x52, 0xaf, 0x45, 0x89, - 0x2f, 0x0a, 0xaf, 0xf6, 0xab, 0x09, 0x58, 0xe6, 0x1c, 0x7a, 0x97, 0xc8, 0xfb, 0x23, 0x12, 0x39, - 0x18, 0xa2, 0x1a, 0x1f, 0xc4, 0xdc, 0x9e, 0x18, 0xbf, 0xf8, 0x42, 0xfe, 0xa7, 0x64, 0xaf, 0x92, - 0x6f, 0xc0, 0xaa, 0xac, 0xda, 0x86, 0x7e, 0x58, 0x2e, 0xe5, 0xb3, 0xd8, 0xd0, 0x2b, 0xb5, 0xc7, - 0x7a, 0x41, 0x49, 0xa0, 0x55, 0x40, 0xa1, 0x2e, 0x5b, 0xdd, 0xd3, 0x71, 0xfd, 0xb0, 0x5c, 0x6a, - 0x28, 0x49, 0xb4, 0x06, 0xcb, 0x03, 0xf2, 0x8a, 0x6e, 0xec, 0xb1, 0x42, 0x1f, 0xa1, 0x00, 0x46, - 0x76, 0xb7, 0x81, 0xeb, 0xd5, 0xec, 0x61, 0xbd, 0x58, 0x6b, 0x28, 0x93, 0x28, 0x03, 0x1b, 0x52, - 0x53, 0xae, 0xed, 0x95, 0xf2, 0xd9, 0x32, 0xae, 0x1d, 0xd6, 0x71, 0xa5, 0x54, 0xaf, 0x97, 0xaa, - 0x7b, 0x4a, 0x2a, 0xe2, 0x59, 0x2f, 0xd7, 0x9e, 0xe0, 0x7c, 0xad, 0x5a, 0x3f, 0xaa, 0xe8, 0x86, - 0x32, 0xa5, 0x99, 0xa0, 0x96, 0x5c, 0x8b, 0x04, 0x84, 0xb6, 0x6d, 0xd7, 0x0c, 0x48, 0xde, 0x6b, - 0xb7, 0x6d, 0x09, 0xf1, 0x3a, 0xcc, 0xf9, 0x81, 0xd9, 0xe2, 0x74, 0xfe, 0x07, 0xf2, 0x3b, 0x90, - 0x8e, 0x8c, 0xe0, 0x2d, 0xc3, 0x52, 0xc9, 0x3d, 0x33, 0x1d, 0xdb, 0xe2, 0x15, 0x46, 0x6c, 0x44, - 0x06, 0x6e, 0xd6, 0x3a, 0x81, 0xdd, 0x66, 0x55, 0xa6, 0xa9, 0x9f, 0x99, 0x4e, 0xde, 0x73, 0x4f, - 0x1c, 0xbb, 0x19, 0xc8, 0x8d, 0xfa, 0xf5, 0x2a, 0xcc, 0xf1, 0x5f, 0x05, 0x12, 0x98, 0xb6, 0x83, - 0x0c, 0x50, 0x5c, 0x2f, 0xc0, 0x03, 0x9d, 0xbf, 0x48, 0xe8, 0xdd, 0x98, 0xad, 0x8a, 0x79, 0x7d, - 0x28, 0x26, 0x8c, 0x45, 0x77, 0x40, 0x8c, 0x6a, 0x70, 0x4d, 0xb4, 0xca, 0x2c, 0xf2, 0x09, 0x83, - 0x66, 0x79, 0xb8, 0xef, 0x8e, 0xdb, 0xfd, 0x01, 0x08, 0x2f, 0xb2, 0x06, 0x28, 0x2a, 0x45, 0x4f, - 0x01, 0x89, 0x80, 0x2f, 0xc8, 0x45, 0xaf, 0x19, 0x95, 0x78, 0x75, 0x6f, 0x5c, 0xcc, 0xe1, 0xce, - 0xb9, 0x98, 0x30, 0x14, 0x3a, 0xa4, 0x40, 0x3f, 0x4d, 0xc2, 0x16, 0xef, 0xd3, 0x5e, 0xf2, 0x76, - 0x6e, 0xa0, 0x5f, 0xb1, 0x65, 0x43, 0x27, 0x9b, 0xd0, 0x8f, 0xe2, 0x06, 0x7a, 0x6b, 0x23, 0x58, - 0x4c, 0x18, 0xb7, 0xe8, 0x65, 0x56, 0xe8, 0x73, 0x58, 0x8e, 0x80, 0x29, 0x36, 0x45, 0xbf, 0x20, - 0xbb, 0xb0, 0x07, 0x57, 0x6a, 0x2e, 0xc2, 0x91, 0x50, 0x30, 0xa2, 0x42, 0x0d, 0x50, 0xa2, 0xe1, - 0x59, 0x7f, 0xa0, 0x4e, 0xf3, 0xd8, 0xef, 0x5d, 0x1e, 0xbb, 0xd7, 0x8e, 0x14, 0x13, 0xc6, 0xb5, - 0x60, 0x50, 0x8e, 0x9e, 0xc0, 0x52, 0x34, 0x2a, 0x65, 0x37, 0x57, 0x9d, 0x19, 0xbb, 0x21, 0xb1, - 0x7d, 0x08, 0xdb, 0x90, 0x60, 0x48, 0x81, 0x3e, 0x83, 0xe8, 0x24, 0xb0, 0xcf, 0xc9, 0xbd, 0x9a, - 0xe6, 0x91, 0xef, 0x5f, 0xb9, 0x11, 0x28, 0x26, 0x8c, 0x68, 0x7e, 0x42, 0x83, 0x8a, 0x0c, 0x15, - 0xed, 0x80, 0x84, 0xa8, 0x38, 0xcb, 0xa3, 0xde, 0x8e, 0x89, 0x3a, 0x4c, 0xf8, 0x8b, 0x09, 0x86, - 0x90, 0x3d, 0x19, 0x2a, 0xc1, 0x82, 0x88, 0x14, 0x78, 0x1e, 0x66, 0xe0, 0x0d, 0x97, 0x87, 0x8a, - 0xb0, 0x9e, 0x5e, 0x28, 0x21, 0x63, 0x97, 0xc5, 0xeb, 0x60, 0x2a, 0x49, 0x32, 0x07, 0x84, 0xb9, - 0xb1, 0x97, 0x65, 0x94, 0x4d, 0xb3, 0xcb, 0xe2, 0x45, 0xa5, 0x6c, 0xc3, 0x9b, 0x21, 0xbd, 0xc6, - 0x27, 0x9c, 0x5f, 0xab, 0xf3, 0x63, 0x37, 0x3c, 0x8e, 0x89, 0xb3, 0x0d, 0x6f, 0x0e, 0xca, 0x51, - 0x35, 0x6c, 0x5b, 0xa8, 0xe4, 0xd7, 0xea, 0xc2, 0xd8, 0x2c, 0x47, 0x79, 0x38, 0xcb, 0xd2, 0x89, - 0x4a, 0x59, 0x96, 0xae, 0x67, 0x11, 0xdc, 0xed, 0xbf, 0xcc, 0xa9, 0x8b, 0x63, 0xb3, 0x8c, 0x7b, - 0xc3, 0x63, 0x59, 0xba, 0x83, 0x72, 0x01, 0x14, 0x27, 0x01, 0x6e, 0x31, 0x8a, 0x8b, 0x2d, 0xc1, - 0x71, 0x55, 0xe5, 0x12, 0xa0, 0x88, 0xa1, 0xc3, 0x02, 0x28, 0x06, 0x15, 0xec, 0x5c, 0x86, 0x5c, - 0xb5, 0xd9, 0xe3, 0xc8, 0xea, 0xd2, 0xd8, 0x73, 0x19, 0xcf, 0xa7, 0xd9, 0xb9, 0xa4, 0xc3, 0x1a, - 0x8e, 0x97, 0x32, 0x76, 0x78, 0x9e, 0xd0, 0x78, 0xbc, 0x1c, 0xe1, 0xd1, 0x1c, 0x2f, 0xa3, 0x52, - 0xb6, 0xb8, 0x66, 0xd8, 0x5b, 0x60, 0xca, 0x9b, 0x0b, 0x75, 0x63, 0xec, 0xe2, 0xc6, 0xb5, 0x21, - 0x6c, 0x71, 0xcd, 0x41, 0x39, 0x4b, 0x53, 0x30, 0xeb, 0x3e, 0xac, 0xdf, 0x18, 0x9b, 0xe6, 0x28, - 0x33, 0x67, 0x69, 0xfa, 0x51, 0x29, 0xfa, 0x45, 0x12, 0xee, 0x8c, 0xa0, 0x08, 0x47, 0x62, 0xcc, - 0x1f, 0xbc, 0x31, 0x15, 0x14, 0x59, 0xbd, 0xc9, 0x87, 0xf9, 0x9f, 0x2b, 0x00, 0x4b, 0x2c, 0xbb, - 0x2e, 0x26, 0x8c, 0xad, 0xe0, 0x2d, 0x86, 0x6c, 0xcd, 0x6c, 0xc1, 0x3d, 0xb1, 0x27, 0xc9, 0xa7, - 0xba, 0x39, 0x76, 0xcd, 0xe2, 0x68, 0x2a, 0x5b, 0x33, 0x7b, 0x50, 0xce, 0xc0, 0xbd, 0xdb, 0x7f, - 0x67, 0xc6, 0xb2, 0x75, 0x54, 0xb7, 0xc6, 0x82, 0xfb, 0x98, 0x57, 0x69, 0x06, 0xee, 0xdd, 0x11, - 0x15, 0x7a, 0x0e, 0x4a, 0xaf, 0x53, 0xc7, 0xc7, 0x9c, 0x9e, 0xaa, 0x1a, 0x8f, 0xbd, 0x1d, 0x13, - 0xfb, 0x12, 0x36, 0xcb, 0x31, 0x7e, 0x50, 0x83, 0x5e, 0xc2, 0x2d, 0xd6, 0x7b, 0x98, 0x82, 0xd7, - 0x63, 0xd2, 0x27, 0xf6, 0x92, 0xc6, 0xdf, 0xe6, 0x23, 0x3d, 0x8c, 0xdb, 0x96, 0xcb, 0xdb, 0x81, - 0x62, 0xc2, 0xd8, 0x08, 0xc6, 0x9a, 0x30, 0xac, 0x11, 0x08, 0xcd, 0x6a, 0x3d, 0x23, 0xb5, 0xea, - 0x9d, 0xb1, 0xe7, 0x6c, 0x94, 0xfc, 0xb2, 0x73, 0x66, 0x47, 0xa5, 0xe8, 0x08, 0x96, 0xda, 0x8c, - 0xb4, 0x62, 0xdb, 0x65, 0x07, 0x8b, 0xd3, 0x56, 0xf5, 0xee, 0xd8, 0xbd, 0x8d, 0x23, 0xb8, 0x6c, - 0x7d, 0xda, 0x83, 0x72, 0xf4, 0x7f, 0x92, 0xe6, 0x9c, 0x10, 0xbe, 0xb3, 0xac, 0x02, 0xbe, 0x3b, - 0x96, 0x39, 0xc5, 0x90, 0x5c, 0xc6, 0x9c, 0x7a, 0x01, 0x44, 0xf5, 0xfb, 0x02, 0x56, 0xec, 0x28, - 0x6b, 0xc4, 0x4d, 0x4e, 0x1b, 0xd5, 0xf7, 0x78, 0xdc, 0xf7, 0x63, 0xe7, 0x1f, 0x4f, 0x32, 0x8b, - 0x09, 0x63, 0xd9, 0x1e, 0xd5, 0xa1, 0xc7, 0xb0, 0x6c, 0x0b, 0xd2, 0x28, 0x39, 0x9f, 0xd8, 0xca, - 0x7b, 0x23, 0x7f, 0xf6, 0xf4, 0x07, 0x18, 0xa2, 0x98, 0x0c, 0xc3, 0xec, 0x61, 0x21, 0x6a, 0xc3, - 0xba, 0xd7, 0xe3, 0x9d, 0x98, 0x9c, 0x99, 0x0e, 0x6e, 0x86, 0xcc, 0x53, 0xbd, 0xcf, 0xa3, 0xef, - 0xc4, 0x16, 0xb4, 0xf1, 0x5c, 0xb5, 0x98, 0x30, 0xd6, 0xbc, 0x78, 0x7d, 0x6e, 0x06, 0xa6, 0x78, - 0x07, 0xb7, 0x9f, 0x4a, 0x5f, 0x53, 0x94, 0xfd, 0x54, 0x7a, 0x59, 0x59, 0xd9, 0x4f, 0xa5, 0x57, - 0x94, 0xeb, 0xfb, 0xa9, 0xf4, 0x75, 0x65, 0x75, 0x3f, 0x95, 0x5e, 0x55, 0xd6, 0xf6, 0x53, 0xe9, - 0x35, 0x45, 0xdd, 0x4f, 0xa5, 0x55, 0x65, 0x7d, 0x3f, 0x95, 0x5e, 0x57, 0x36, 0xf6, 0x53, 0xe9, - 0x5b, 0x4a, 0x66, 0x3f, 0x95, 0xce, 0x28, 0x9b, 0xfb, 0xa9, 0xf4, 0x3b, 0x8a, 0xa6, 0xdd, 0xe7, - 0x84, 0xf8, 0xd0, 0xf3, 0x79, 0xb9, 0x43, 0x1b, 0x30, 0xc5, 0xd6, 0xed, 0x5c, 0xbe, 0x47, 0x08, - 0xda, 0x2d, 0x44, 0xda, 0x57, 0x53, 0x30, 0x15, 0xfe, 0x0b, 0x33, 0xf4, 0x4a, 0xb3, 0x2e, 0x1f, - 0x19, 0x96, 0x22, 0xff, 0x74, 0x08, 0x83, 0xfe, 0xd3, 0xcd, 0x4f, 0x06, 0x99, 0x1e, 0x25, 0xfc, - 0x0f, 0x1c, 0xce, 0x63, 0x17, 0x63, 0x0f, 0xf7, 0x00, 0xba, 0x71, 0xe3, 0xdc, 0x1d, 0x39, 0xce, - 0xcd, 0xfe, 0x38, 0xa3, 0x56, 0x03, 0x44, 0x50, 0xca, 0x50, 0x1e, 0x16, 0xba, 0x2e, 0x39, 0xef, - 0x78, 0x3e, 0xb1, 0x38, 0xcd, 0x48, 0x5d, 0xa5, 0xef, 0x30, 0xe6, 0x7b, 0x4e, 0x8c, 0x5c, 0xec, - 0xc0, 0x9c, 0x47, 0xed, 0x96, 0xed, 0x62, 0x56, 0x7a, 0x39, 0x49, 0x9d, 0xca, 0x2d, 0xca, 0xa7, - 0xcb, 0x69, 0x56, 0xa6, 0x4b, 0x05, 0x03, 0x84, 0x09, 0xfb, 0x42, 0x87, 0x30, 0x6d, 0xf1, 0x4e, - 0x43, 0x92, 0xce, 0xcc, 0xb8, 0xe7, 0x13, 0xd1, 0x8f, 0xe4, 0x54, 0x39, 0x3f, 0xa5, 0x3f, 0x3f, - 0xa1, 0x31, 0x64, 0x1c, 0x74, 0x00, 0x0b, 0x0c, 0x8a, 0xac, 0x1e, 0x0c, 0x09, 0x1a, 0xb7, 0x15, - 0x09, 0x1c, 0xfe, 0x35, 0xbb, 0xad, 0x0b, 0xc3, 0xe8, 0xfb, 0xcc, 0x3c, 0x89, 0xc8, 0xd0, 0x7f, - 0x87, 0xbb, 0x3d, 0x73, 0x59, 0x76, 0xe1, 0xe1, 0x90, 0xe7, 0x00, 0x75, 0x61, 0xd2, 0xf5, 0x5e, - 0x4a, 0x56, 0xfa, 0x96, 0x8e, 0xbd, 0x20, 0x17, 0xe7, 0xd1, 0xd5, 0x5f, 0x6c, 0x58, 0x80, 0xbc, - 0xe3, 0x35, 0x5f, 0xf4, 0xa2, 0x18, 0x6c, 0x3c, 0xf1, 0xa8, 0x24, 0x9e, 0x80, 0x1f, 0xfc, 0x63, - 0x02, 0xd4, 0x71, 0xff, 0x32, 0xb0, 0xb6, 0x34, 0x9b, 0xab, 0x19, 0x0d, 0x3c, 0xf2, 0xda, 0x7d, - 0x17, 0xde, 0x19, 0xd0, 0xf0, 0x0f, 0xbd, 0x80, 0x0d, 0x3d, 0x5f, 0x33, 0x0a, 0x78, 0xb7, 0x76, - 0x54, 0x2d, 0x28, 0x49, 0xd6, 0xf7, 0x0e, 0x98, 0xe5, 0xcb, 0x25, 0xbd, 0xca, 0xbe, 0xf6, 0xf5, - 0x3c, 0xeb, 0x8b, 0x37, 0xe1, 0xc6, 0x80, 0xfe, 0xf0, 0xa8, 0x5e, 0xd4, 0x8d, 0x30, 0x9a, 0x92, - 0x42, 0x37, 0x60, 0x6d, 0x74, 0x1c, 0x5c, 0x3f, 0xcc, 0x56, 0x95, 0x29, 0x94, 0x85, 0x4f, 0x07, - 0x95, 0x65, 0x43, 0xcf, 0x16, 0x9e, 0xf5, 0x1f, 0xdf, 0x71, 0xcd, 0xc0, 0x46, 0xad, 0x5c, 0xd6, - 0x0b, 0x38, 0x97, 0xcd, 0x1f, 0xe0, 0xc3, 0x5a, 0xbd, 0x5e, 0xca, 0x95, 0x75, 0xde, 0xec, 0x67, - 0x9f, 0x29, 0xd3, 0xe8, 0x3d, 0xb8, 0x3d, 0x10, 0xa2, 0xaa, 0x3f, 0xc1, 0x65, 0x3d, 0x5b, 0xd7, - 0xf1, 0xa1, 0xa1, 0x3f, 0xd6, 0xab, 0x8d, 0x3a, 0x6e, 0x3c, 0xad, 0x2a, 0x69, 0x74, 0x1f, 0xee, - 0x0e, 0x18, 0x36, 0x4a, 0x15, 0xbd, 0xde, 0xc8, 0x56, 0x0e, 0x71, 0x3e, 0x9b, 0x2f, 0xea, 0x72, - 0x4a, 0x7a, 0x41, 0x99, 0xd9, 0x48, 0x7d, 0xf5, 0x75, 0x26, 0xa1, 0xb1, 0xe5, 0x9d, 0x78, 0xf0, - 0xe7, 0xc1, 0xbf, 0x2d, 0x22, 0x7f, 0x81, 0x88, 0x9e, 0xbf, 0x61, 0x3c, 0x1b, 0x5d, 0x5c, 0xfe, - 0xc0, 0xc0, 0x34, 0x4f, 0x8c, 0x52, 0x43, 0xc7, 0x8d, 0x5a, 0x0d, 0xd7, 0xca, 0x6c, 0x39, 0xf9, - 0x8b, 0x04, 0x53, 0xd4, 0x75, 0xa3, 0x94, 0x2d, 0x97, 0x3e, 0xcb, 0xe6, 0xca, 0xba, 0x32, 0x89, - 0x6e, 0xc1, 0xba, 0x90, 0x67, 0xeb, 0xcf, 0xaa, 0x79, 0xe9, 0xb6, 0x9b, 0x2d, 0x95, 0x8f, 0x0c, - 0x5d, 0x99, 0x42, 0x1a, 0x64, 0x84, 0x5a, 0x2c, 0x0c, 0x2e, 0xe8, 0xd9, 0x42, 0xb9, 0x54, 0xd5, - 0xb1, 0xfe, 0x34, 0xaf, 0xeb, 0x05, 0xbd, 0xa0, 0x4c, 0x8b, 0xa4, 0x1f, 0x7c, 0x02, 0x68, 0x14, - 0x05, 0x50, 0x1a, 0x52, 0xd5, 0x5a, 0x55, 0x57, 0x12, 0x68, 0x0e, 0x66, 0xd8, 0x42, 0xd6, 0x76, - 0x77, 0x95, 0x24, 0x5a, 0x80, 0xd9, 0x52, 0xa5, 0xa2, 0x17, 0x4a, 0xd9, 0x86, 0xae, 0x4c, 0xe4, - 0xee, 0x7f, 0xf3, 0xaf, 0x4c, 0xe2, 0x9b, 0xd7, 0x99, 0xe4, 0xb7, 0xaf, 0x33, 0xc9, 0xef, 0x5e, - 0x67, 0x92, 0xff, 0x7c, 0x9d, 0x49, 0xfe, 0xe6, 0x4d, 0x26, 0xf1, 0xed, 0x9b, 0x4c, 0xe2, 0xbb, - 0x37, 0x99, 0xc4, 0x67, 0x33, 0x12, 0x17, 0xfe, 0x13, 0x00, 0x00, 0xff, 0xff, 0xf9, 0xa4, 0x7e, - 0xc7, 0x1d, 0x21, 0x00, 0x00, + 0x02, 0x45, 0xd1, 0xbf, 0xa0, 0xeb, 0x2c, 0x83, 0xae, 0x82, 0x2e, 0x8c, 0xd6, 0xee, 0x5f, 0x91, + 0x55, 0x71, 0x1e, 0x43, 0x0e, 0xc9, 0xa1, 0xac, 0x64, 0x47, 0x7e, 0xaf, 0xf3, 0x9d, 0xd7, 0xef, + 0xfb, 0x7d, 0x67, 0x60, 0x85, 0x7a, 0x66, 0xf3, 0xb4, 0x73, 0xbc, 0x43, 0x28, 0xf5, 0xa8, 0xbf, + 0xdd, 0xa1, 0x5e, 0xe0, 0xa1, 0xa5, 0xa6, 0xd7, 0x7c, 0xc1, 0x35, 0xdb, 0x2f, 0xce, 0xb6, 0x5f, + 0x9c, 0x75, 0x8e, 0x37, 0xae, 0x0b, 0x83, 0x21, 0xcb, 0x0d, 0x14, 0xfa, 0x5b, 0x66, 0x60, 0x4a, + 0xd9, 0x6a, 0x28, 0x6b, 0x93, 0xc0, 0x8c, 0xc8, 0xd5, 0x6e, 0x60, 0x3b, 0x3b, 0xa7, 0x4e, 0x73, + 0x27, 0xb0, 0xdb, 0xc4, 0x0f, 0xcc, 0x76, 0x47, 0x6a, 0x56, 0x5a, 0x5e, 0xcb, 0xe3, 0x3f, 0x77, + 0xd8, 0x2f, 0x21, 0xd5, 0xfe, 0x3a, 0x01, 0xcb, 0x55, 0x2f, 0x28, 0x13, 0xd3, 0x27, 0x45, 0xcf, + 0xb1, 0x08, 0xd5, 0xd9, 0xd0, 0xa8, 0x00, 0x33, 0x94, 0x74, 0x1c, 0xbb, 0x69, 0xaa, 0xc9, 0xad, + 0xe4, 0xbd, 0xb9, 0x87, 0x77, 0xb6, 0xfb, 0xf9, 0xca, 0xb1, 0xb7, 0x0d, 0x61, 0x51, 0x20, 0x7e, + 0x93, 0xda, 0x9d, 0xc0, 0xa3, 0xb9, 0xd4, 0xd7, 0xaf, 0x36, 0x13, 0x46, 0xe8, 0x8a, 0xf6, 0x60, + 0xde, 0x61, 0x91, 0xf1, 0x29, 0x0f, 0xad, 0x4e, 0x5c, 0x3d, 0x94, 0x31, 0xe7, 0xf4, 0x73, 0x42, + 0xdb, 0x30, 0xc5, 0xff, 0xaa, 0x29, 0x1e, 0x41, 0x8d, 0x89, 0xc0, 0xa7, 0x60, 0x08, 0x33, 0xf4, + 0x11, 0xa4, 0xa9, 0xe9, 0xb6, 0x08, 0xb6, 0x2d, 0x75, 0x72, 0x2b, 0x79, 0x6f, 0x32, 0xb7, 0xc1, + 0x32, 0x7b, 0xfd, 0x6a, 0x73, 0xc6, 0x60, 0xf2, 0x52, 0xe1, 0xbb, 0xfe, 0x4f, 0x63, 0x86, 0xdb, + 0x96, 0x2c, 0x74, 0x1b, 0xa0, 0xd9, 0xf5, 0x03, 0xaf, 0x8d, 0xdb, 0x7e, 0x4b, 0x9d, 0xda, 0x4a, + 0xde, 0x9b, 0x95, 0x53, 0x9a, 0x15, 0xf2, 0x8a, 0xdf, 0xd2, 0x56, 0x61, 0xa5, 0xea, 0x59, 0xe4, + 0xc8, 0x35, 0xcf, 0x4c, 0xdb, 0x31, 0x8f, 0x1d, 0xc2, 0x97, 0x4c, 0x5b, 0x87, 0xb5, 0x23, 0xd7, + 0xef, 0x76, 0x3a, 0x1e, 0x0d, 0x88, 0x65, 0x90, 0x2f, 0xba, 0xc4, 0x0f, 0x84, 0xea, 0x67, 0x49, + 0x40, 0x7c, 0xb0, 0xaa, 0x17, 0xec, 0x7a, 0x5d, 0xd7, 0x12, 0x8b, 0x1c, 0xcd, 0x32, 0x79, 0xf5, + 0x2c, 0x3f, 0x82, 0xb4, 0x1f, 0x78, 0x94, 0xbb, 0x4d, 0x0c, 0xba, 0xd5, 0x99, 0x5c, 0xb8, 0xc9, + 0x9f, 0xc6, 0x0c, 0xb7, 0x2d, 0x59, 0xda, 0xef, 0x26, 0xe1, 0x3a, 0x8f, 0x75, 0x40, 0x2e, 0x2a, + 0xb6, 0xdf, 0x36, 0x83, 0xe6, 0xa9, 0xc8, 0xe3, 0x43, 0x58, 0xa2, 0x22, 0x5d, 0xec, 0x07, 0x26, + 0x0d, 0xf0, 0x0b, 0x72, 0xc1, 0x13, 0x9a, 0xcf, 0xcd, 0x7c, 0xf7, 0x6a, 0x73, 0xf2, 0x80, 0x5c, + 0x18, 0xd7, 0xa4, 0x45, 0x9d, 0x19, 0x1c, 0x90, 0x0b, 0xb4, 0x03, 0xa1, 0x08, 0x13, 0xd7, 0xe2, + 0x2e, 0x13, 0x83, 0x2e, 0x0b, 0x52, 0xaf, 0xbb, 0x16, 0x73, 0x38, 0x85, 0x1b, 0x16, 0xe9, 0x50, + 0xd2, 0x34, 0x03, 0x62, 0xe1, 0xb6, 0xcc, 0x80, 0x58, 0x98, 0xcf, 0x8b, 0x6f, 0xd3, 0xdc, 0x43, + 0x2d, 0xee, 0x6c, 0x30, 0xfd, 0xc8, 0x21, 0x5b, 0xef, 0x07, 0xab, 0xf4, 0x62, 0x71, 0x53, 0xf4, + 0x39, 0x6c, 0x44, 0x46, 0xf2, 0xbb, 0xad, 0x16, 0xf1, 0x83, 0xde, 0x40, 0xa9, 0xab, 0x0e, 0x64, + 0xa8, 0xfd, 0x28, 0xf5, 0x30, 0x88, 0x18, 0xa1, 0x0c, 0xd3, 0x3c, 0x98, 0xaf, 0x4e, 0x6d, 0x4d, + 0xde, 0x9b, 0x7b, 0x78, 0x73, 0x5c, 0xb4, 0x92, 0x7b, 0xe2, 0xe5, 0x56, 0xe5, 0xf6, 0x2c, 0x0a, + 0x9f, 0x92, 0x1b, 0x10, 0xea, 0x9a, 0x8e, 0x21, 0x63, 0x68, 0x6f, 0x26, 0x41, 0x33, 0x88, 0x69, + 0x3d, 0xb1, 0x83, 0x53, 0xdb, 0x3d, 0x72, 0x9b, 0x84, 0x06, 0xa6, 0xed, 0x06, 0x17, 0xdc, 0xf2, + 0xcc, 0x74, 0xc4, 0x36, 0xed, 0xc3, 0x22, 0x25, 0xa6, 0x85, 0x7b, 0x37, 0x5b, 0x5e, 0xcd, 0x5b, + 0x91, 0xc1, 0xd9, 0xf5, 0xdf, 0x3e, 0x75, 0x9a, 0xdb, 0x8d, 0xd0, 0x48, 0x2e, 0xd7, 0x02, 0x73, + 0xed, 0x09, 0x91, 0x01, 0x88, 0x9c, 0xdb, 0x7e, 0x60, 0xbb, 0xad, 0x48, 0xbc, 0x89, 0xab, 0xc7, + 0x5b, 0x0a, 0xdd, 0xfb, 0x31, 0x9f, 0xc3, 0x9a, 0xe3, 0x35, 0x4d, 0x07, 0x77, 0xfb, 0x33, 0xc0, + 0x8e, 0xdd, 0xb6, 0x03, 0x7e, 0x95, 0xae, 0x18, 0xf8, 0x3a, 0x8f, 0x11, 0x59, 0x84, 0x32, 0x8b, + 0x80, 0x3e, 0x03, 0xb5, 0xe5, 0x78, 0xc7, 0xb1, 0xd1, 0x27, 0xaf, 0x1e, 0x7d, 0x55, 0x04, 0x19, + 0x09, 0xff, 0x1c, 0x96, 0xbd, 0x63, 0x9f, 0xd0, 0x33, 0x12, 0x59, 0x5f, 0x5f, 0x4d, 0xf1, 0xdd, + 0x8d, 0x03, 0xac, 0x9a, 0xb4, 0x1e, 0x1e, 0x00, 0x79, 0xc3, 0x0a, 0xff, 0x93, 0xd4, 0xef, 0xff, + 0xb0, 0x99, 0xd0, 0x2c, 0x58, 0x6b, 0x50, 0xd3, 0xf5, 0xcd, 0x66, 0x60, 0x7b, 0x6e, 0xf6, 0x98, + 0xc3, 0x84, 0xd8, 0xd9, 0x12, 0x4c, 0x53, 0x62, 0xfa, 0x9e, 0xcb, 0x77, 0x74, 0xf1, 0xe1, 0xfb, + 0xdb, 0x23, 0xc5, 0x61, 0x7b, 0xd4, 0xd7, 0xe0, 0x2e, 0x72, 0x5c, 0x19, 0x40, 0x7b, 0x0e, 0x2b, + 0x11, 0xcb, 0xc3, 0xae, 0x2f, 0xef, 0x78, 0x1e, 0xa0, 0xd3, 0xf5, 0x4f, 0x09, 0xc1, 0xc1, 0xb9, + 0x2b, 0x0f, 0x4e, 0x26, 0x66, 0x5e, 0x11, 0xe7, 0x10, 0xfa, 0x84, 0x5f, 0xe3, 0xdc, 0xd5, 0x7e, + 0x9e, 0x84, 0xeb, 0x11, 0x03, 0x83, 0x04, 0xf4, 0x42, 0x84, 0xdf, 0x1b, 0x9a, 0xc1, 0xfd, 0xcb, + 0x67, 0xc0, 0x3d, 0xe3, 0xf2, 0x47, 0xef, 0xc0, 0x2c, 0x39, 0x0f, 0xa8, 0xc9, 0x11, 0x78, 0x22, + 0x82, 0xc0, 0x69, 0x2e, 0x66, 0x00, 0xfc, 0xb7, 0x24, 0xac, 0x46, 0x62, 0xd5, 0x03, 0x33, 0xe8, + 0xfa, 0x22, 0x8d, 0x55, 0x98, 0x64, 0x7e, 0xc9, 0x88, 0x1f, 0x13, 0xa0, 0x6a, 0x2f, 0xbd, 0x09, + 0x9e, 0xde, 0x7f, 0x5d, 0x9e, 0x5e, 0x24, 0xe4, 0x76, 0xec, 0x2a, 0x3f, 0x82, 0x69, 0x21, 0x47, + 0x08, 0x16, 0x0d, 0x3d, 0x5b, 0xaf, 0x55, 0xf1, 0x51, 0xf5, 0xa0, 0x5a, 0x7b, 0x52, 0x55, 0x12, + 0x48, 0x85, 0x15, 0x29, 0x6b, 0x3c, 0xad, 0xe2, 0x7c, 0xad, 0x52, 0x29, 0x35, 0x1a, 0x7a, 0x41, + 0x99, 0xd0, 0x52, 0xe9, 0xa4, 0x92, 0xd4, 0x7e, 0x9b, 0x04, 0xe5, 0x09, 0xb5, 0x03, 0xc2, 0xee, + 0xb7, 0x2b, 0x6a, 0x04, 0xfa, 0x18, 0x66, 0x6c, 0xfe, 0xd7, 0x57, 0x93, 0xfc, 0xd4, 0xad, 0xc7, + 0xec, 0x8e, 0x70, 0x08, 0xcb, 0xac, 0xb4, 0x47, 0x8f, 0x60, 0x51, 0x94, 0x59, 0x9f, 0x01, 0xae, + 0xdb, 0x24, 0xb2, 0xe6, 0x5d, 0x67, 0x66, 0xdf, 0xbd, 0xda, 0x5c, 0xe0, 0xb5, 0xb1, 0x2e, 0x95, + 0xc6, 0x82, 0x13, 0xfd, 0xbb, 0x9f, 0x4a, 0x4f, 0x28, 0x93, 0xda, 0x1f, 0xc3, 0x9c, 0x1a, 0x9e, + 0x57, 0x73, 0xe4, 0xb9, 0xcc, 0xc2, 0xec, 0x0f, 0x02, 0x9b, 0xbe, 0x17, 0xaa, 0x82, 0x62, 0x36, + 0x83, 0xae, 0xe9, 0xfc, 0x30, 0x98, 0xb9, 0x26, 0x9c, 0x7b, 0x62, 0x6d, 0x05, 0x50, 0xad, 0xc3, + 0x8a, 0xab, 0x4d, 0x89, 0xdf, 0x38, 0x77, 0x45, 0x81, 0xad, 0xc3, 0x4a, 0xde, 0x73, 0x2d, 0x9b, + 0x6d, 0xde, 0xae, 0x69, 0x3b, 0xe1, 0xc5, 0xfa, 0x3f, 0x98, 0x97, 0xa3, 0x9f, 0x99, 0x4e, 0x97, + 0xc8, 0x39, 0xc4, 0xd1, 0x87, 0xc7, 0x4c, 0x6f, 0xcc, 0x09, 0x6b, 0xfe, 0x47, 0xfb, 0x4b, 0x12, + 0x90, 0x60, 0x15, 0xe4, 0xc7, 0xa4, 0xd9, 0xbb, 0xac, 0x19, 0x98, 0x69, 0x13, 0xdf, 0x37, 0x5b, + 0x64, 0xe0, 0x9c, 0x85, 0x42, 0xf4, 0x08, 0x66, 0x65, 0xe1, 0x23, 0x96, 0x9c, 0xea, 0x58, 0xbe, + 0x12, 0xae, 0x57, 0xcf, 0x01, 0x7d, 0x02, 0xe9, 0x10, 0x59, 0x25, 0xae, 0xbd, 0xcd, 0xb9, 0x67, + 0xaf, 0x7d, 0x01, 0x2b, 0xd9, 0xf6, 0xb1, 0xdd, 0xea, 0x7a, 0x5d, 0xdf, 0x20, 0x7e, 0xd7, 0x09, + 0xae, 0x96, 0xf1, 0xc7, 0x30, 0xf7, 0x92, 0x9a, 0x9d, 0x0e, 0xb1, 0x30, 0xa1, 0x34, 0x26, 0xe7, + 0xf0, 0x8a, 0xf0, 0x70, 0x06, 0x48, 0x63, 0x9d, 0x52, 0x6d, 0x8d, 0x71, 0x8a, 0x93, 0x60, 0x8f, + 0x7a, 0xdd, 0x4e, 0x81, 0x38, 0x24, 0x5c, 0x25, 0x0d, 0xc3, 0xaa, 0xe4, 0x74, 0x79, 0x8f, 0xd2, + 0x6e, 0x87, 0xed, 0x8c, 0xc8, 0x86, 0xdd, 0x70, 0xf6, 0x03, 0x0f, 0xdf, 0xd4, 0x34, 0x17, 0x57, + 0xfc, 0x16, 0xd2, 0x60, 0xb6, 0x43, 0xbd, 0x26, 0xf1, 0x7d, 0xb9, 0x84, 0xe9, 0x1e, 0x16, 0x85, + 0x62, 0xad, 0x0e, 0x48, 0x0e, 0x10, 0x3d, 0xb1, 0xff, 0x0f, 0x20, 0xc9, 0x67, 0x48, 0xaa, 0xa6, + 0x72, 0x19, 0x59, 0x7e, 0x67, 0xa5, 0x3d, 0xe7, 0x47, 0xfd, 0x3f, 0x6c, 0xf5, 0xc5, 0x4f, 0x4b, + 0x3b, 0x00, 0xc4, 0x79, 0xd3, 0x08, 0x4f, 0xeb, 0x11, 0xae, 0xe4, 0xd5, 0x09, 0x57, 0x9d, 0x11, + 0xc2, 0x53, 0xd3, 0xb5, 0x1c, 0x86, 0xd5, 0x01, 0xbd, 0xe8, 0x71, 0x45, 0xf4, 0x10, 0x52, 0x1d, + 0x9d, 0xd2, 0x98, 0xf3, 0x38, 0xb0, 0xd4, 0x72, 0xd6, 0xdc, 0x56, 0x56, 0x91, 0x7f, 0x27, 0xe1, + 0xee, 0x30, 0x90, 0x32, 0xde, 0x70, 0xc8, 0xe8, 0xbc, 0x41, 0x4e, 0x28, 0x09, 0x11, 0x7f, 0x1c, + 0x16, 0x3e, 0x87, 0xe9, 0xe0, 0xdc, 0x0d, 0xc9, 0xe3, 0x7c, 0xae, 0xc0, 0x54, 0xff, 0x78, 0xb5, + 0xf9, 0x61, 0xcb, 0x0e, 0x4e, 0xbb, 0xc7, 0xdb, 0x4d, 0xaf, 0xbd, 0xd3, 0xcb, 0xc7, 0x3a, 0xee, + 0xff, 0xde, 0xe9, 0xbc, 0x68, 0xed, 0xf0, 0xfe, 0xa2, 0xdb, 0xb5, 0xad, 0xed, 0xa3, 0xa3, 0x52, + 0xe1, 0xf5, 0xab, 0xcd, 0xa9, 0xc6, 0xb9, 0x5b, 0x2a, 0x18, 0x53, 0xc1, 0xb9, 0x5b, 0xb2, 0xd0, + 0x2e, 0xcc, 0x05, 0xfd, 0xec, 0xe4, 0x09, 0xbe, 0x5a, 0x9d, 0x89, 0x3a, 0x6a, 0xbb, 0xb0, 0xd9, + 0x38, 0x77, 0xb3, 0x0e, 0x63, 0x2d, 0x17, 0xba, 0xdb, 0xf4, 0xba, 0x8c, 0x0a, 0xc9, 0xc3, 0x25, + 0xe6, 0x77, 0x1b, 0xa0, 0x43, 0xc9, 0x19, 0xe6, 0xa7, 0x66, 0x60, 0x9a, 0xb3, 0x4c, 0x2e, 0x8e, + 0xe1, 0x2f, 0x93, 0xb0, 0xc2, 0x40, 0xb3, 0x45, 0x68, 0xed, 0x8c, 0xd0, 0x13, 0xc7, 0x7b, 0x29, + 0xbc, 0xd7, 0x61, 0x32, 0x86, 0xe5, 0x32, 0x19, 0xba, 0x0f, 0x0b, 0xcd, 0x2e, 0xa5, 0xc4, 0x0d, + 0x24, 0x6a, 0x08, 0x92, 0x2d, 0x62, 0xcf, 0x4b, 0x15, 0x87, 0x08, 0xf4, 0x01, 0x5c, 0xb3, 0xdd, + 0x26, 0x25, 0xed, 0xbe, 0xf1, 0x64, 0xc4, 0x78, 0xb1, 0xa7, 0x14, 0x88, 0xf2, 0x55, 0x12, 0x6e, + 0xe4, 0x18, 0x53, 0xed, 0xc3, 0x1c, 0x39, 0xf1, 0x28, 0xd9, 0xcb, 0xf7, 0xf0, 0xb6, 0xf1, 0x83, + 0xf0, 0xb6, 0x4f, 0xc2, 0x58, 0x88, 0x53, 0x76, 0x08, 0x3c, 0xc7, 0xfa, 0x3e, 0x40, 0xdb, 0xf7, + 0xd2, 0xda, 0x80, 0x44, 0x9d, 0xa9, 0xd8, 0xbe, 0x6f, 0xbb, 0x2d, 0x91, 0xdb, 0x23, 0x98, 0x7f, + 0x49, 0x3d, 0xb7, 0x85, 0x45, 0xd5, 0x91, 0xe9, 0x8d, 0x2f, 0x52, 0xc6, 0x1c, 0x37, 0x17, 0x7f, + 0xc2, 0xe5, 0x9e, 0x18, 0x5d, 0x6e, 0xd6, 0x4f, 0x55, 0x08, 0x65, 0x54, 0xf9, 0x90, 0x7a, 0x2d, + 0x4a, 0x7c, 0x51, 0x78, 0xb5, 0x5f, 0x4d, 0xc0, 0x32, 0xe7, 0xd0, 0xbb, 0x44, 0xde, 0x1f, 0x91, + 0xc8, 0xc1, 0x10, 0xd5, 0xf8, 0x20, 0xe6, 0xf6, 0xc4, 0xf8, 0xc5, 0x17, 0xf2, 0x3f, 0x25, 0x7b, + 0x95, 0x7c, 0x03, 0x56, 0x65, 0xd5, 0x36, 0xf4, 0xc3, 0x72, 0x29, 0x9f, 0xc5, 0x86, 0x5e, 0xa9, + 0x3d, 0xd6, 0x0b, 0x4a, 0x02, 0xad, 0x02, 0x0a, 0x75, 0xd9, 0xea, 0x9e, 0x8e, 0xeb, 0x87, 0xe5, + 0x52, 0x43, 0x49, 0xa2, 0x35, 0x58, 0x1e, 0x90, 0x57, 0x74, 0x63, 0x8f, 0x15, 0xfa, 0x08, 0x05, + 0x30, 0xb2, 0xbb, 0x0d, 0x5c, 0xaf, 0x66, 0x0f, 0xeb, 0xc5, 0x5a, 0x43, 0x99, 0x44, 0x19, 0xd8, + 0x90, 0x9a, 0x72, 0x6d, 0xaf, 0x94, 0xcf, 0x96, 0x71, 0xed, 0xb0, 0x8e, 0x2b, 0xa5, 0x7a, 0xbd, + 0x54, 0xdd, 0x53, 0x52, 0x11, 0xcf, 0x7a, 0xb9, 0xf6, 0x04, 0xe7, 0x6b, 0xd5, 0xfa, 0x51, 0x45, + 0x37, 0x94, 0x29, 0xcd, 0x04, 0xb5, 0xe4, 0x5a, 0x24, 0x20, 0xb4, 0x6d, 0xbb, 0x66, 0x40, 0xf2, + 0x5e, 0xbb, 0x6d, 0x4b, 0x88, 0xd7, 0x61, 0xce, 0x0f, 0xcc, 0x16, 0xa7, 0xf3, 0xdf, 0x93, 0xdf, + 0x81, 0x74, 0x64, 0x04, 0x6f, 0x19, 0x96, 0x4a, 0xee, 0x99, 0xe9, 0xd8, 0x16, 0xaf, 0x30, 0x62, + 0x23, 0x32, 0x70, 0xb3, 0xd6, 0x09, 0xec, 0x36, 0xab, 0x32, 0x4d, 0xfd, 0xcc, 0x74, 0xf2, 0x9e, + 0x7b, 0xe2, 0xd8, 0xcd, 0x40, 0x6e, 0xd4, 0xaf, 0x57, 0x61, 0x8e, 0xff, 0x2a, 0x90, 0xc0, 0xb4, + 0x1d, 0x64, 0x80, 0xe2, 0x7a, 0x01, 0x1e, 0xe8, 0xfc, 0x45, 0x42, 0xef, 0xc6, 0x6c, 0x55, 0xcc, + 0xeb, 0x43, 0x31, 0x61, 0x2c, 0xba, 0x03, 0x62, 0x54, 0x83, 0x6b, 0xa2, 0x55, 0x66, 0x91, 0x4f, + 0x18, 0x34, 0xcb, 0xc3, 0x7d, 0x77, 0xdc, 0xee, 0x0f, 0x40, 0x78, 0x91, 0x35, 0x40, 0x51, 0x29, + 0x7a, 0x0a, 0x48, 0x04, 0x7c, 0x41, 0x2e, 0x7a, 0xcd, 0xa8, 0xc4, 0xab, 0x7b, 0xe3, 0x62, 0x0e, + 0x77, 0xce, 0xc5, 0x84, 0xa1, 0xd0, 0x21, 0x05, 0xfa, 0x69, 0x12, 0xb6, 0x78, 0x9f, 0xf6, 0x92, + 0xb7, 0x73, 0x03, 0xfd, 0x8a, 0x2d, 0x1b, 0x3a, 0xd9, 0x84, 0x7e, 0x14, 0x37, 0xd0, 0x5b, 0x1b, + 0xc1, 0x62, 0xc2, 0xb8, 0x45, 0x2f, 0xb3, 0x42, 0x9f, 0xc1, 0x72, 0x04, 0x4c, 0xb1, 0x29, 0xfa, + 0x05, 0xd9, 0x85, 0x3d, 0xb8, 0x52, 0x73, 0x11, 0x8e, 0x84, 0x82, 0x11, 0x15, 0x6a, 0x80, 0x12, + 0x0d, 0xcf, 0xfa, 0x03, 0x75, 0x9a, 0xc7, 0x7e, 0xef, 0xf2, 0xd8, 0xbd, 0x76, 0xa4, 0x98, 0x30, + 0xae, 0x05, 0x83, 0x72, 0xf4, 0x04, 0x96, 0xa2, 0x51, 0x29, 0xbb, 0xb9, 0xea, 0xcc, 0xd8, 0x0d, + 0x89, 0xed, 0x43, 0xd8, 0x86, 0x04, 0x43, 0x0a, 0xf4, 0x29, 0x44, 0x27, 0x81, 0x7d, 0x4e, 0xee, + 0xd5, 0x34, 0x8f, 0x7c, 0xff, 0xca, 0x8d, 0x40, 0x31, 0x61, 0x44, 0xf3, 0x13, 0x1a, 0x54, 0x64, + 0xa8, 0x68, 0x07, 0x24, 0x44, 0xc5, 0x59, 0x1e, 0xf5, 0x76, 0x4c, 0xd4, 0x61, 0xc2, 0x5f, 0x4c, + 0x30, 0x84, 0xec, 0xc9, 0x50, 0x09, 0x16, 0x44, 0xa4, 0xc0, 0xf3, 0x30, 0x03, 0x6f, 0xb8, 0x3c, + 0x54, 0x84, 0xf5, 0xf4, 0x42, 0x09, 0x19, 0xbb, 0x2c, 0x5e, 0x07, 0x53, 0x49, 0x92, 0x39, 0x20, + 0xcc, 0x8d, 0xbd, 0x2c, 0xa3, 0x6c, 0x9a, 0x5d, 0x16, 0x2f, 0x2a, 0x65, 0x1b, 0xde, 0x0c, 0xe9, + 0x35, 0x3e, 0xe1, 0xfc, 0x5a, 0x9d, 0x1f, 0xbb, 0xe1, 0x71, 0x4c, 0x9c, 0x6d, 0x78, 0x73, 0x50, + 0x8e, 0xaa, 0x61, 0xdb, 0x42, 0x25, 0xbf, 0x56, 0x17, 0xc6, 0x66, 0x39, 0xca, 0xc3, 0x59, 0x96, + 0x4e, 0x54, 0xca, 0xb2, 0x74, 0x3d, 0x8b, 0xe0, 0x6e, 0xff, 0x65, 0x4e, 0x5d, 0x1c, 0x9b, 0x65, + 0xdc, 0x1b, 0x1e, 0xcb, 0xd2, 0x1d, 0x94, 0x0b, 0xa0, 0x38, 0x09, 0x70, 0x8b, 0x51, 0x5c, 0x6c, + 0x09, 0x8e, 0xab, 0x2a, 0x97, 0x00, 0x45, 0x0c, 0x1d, 0x16, 0x40, 0x31, 0xa8, 0x60, 0xe7, 0x32, + 0xe4, 0xaa, 0xcd, 0x1e, 0x47, 0x56, 0x97, 0xc6, 0x9e, 0xcb, 0x78, 0x3e, 0xcd, 0xce, 0x25, 0x1d, + 0xd6, 0x70, 0xbc, 0x94, 0xb1, 0xc3, 0xf3, 0x84, 0xc6, 0xe3, 0xe5, 0x08, 0x8f, 0xe6, 0x78, 0x19, + 0x95, 0xb2, 0xc5, 0x35, 0xc3, 0xde, 0x02, 0x53, 0xde, 0x5c, 0xa8, 0x1b, 0x63, 0x17, 0x37, 0xae, + 0x0d, 0x61, 0x8b, 0x6b, 0x0e, 0xca, 0x59, 0x9a, 0x82, 0x59, 0xf7, 0x61, 0xfd, 0xc6, 0xd8, 0x34, + 0x47, 0x99, 0x39, 0x4b, 0xd3, 0x8f, 0x4a, 0xd1, 0x2f, 0x92, 0x70, 0x67, 0x04, 0x45, 0x38, 0x12, + 0x63, 0xfe, 0xe0, 0x8d, 0xa9, 0xa0, 0xc8, 0xea, 0x4d, 0x3e, 0xcc, 0xff, 0x5e, 0x01, 0x58, 0x62, + 0xd9, 0x75, 0x31, 0x61, 0x6c, 0x05, 0x6f, 0x31, 0x64, 0x6b, 0x66, 0x0b, 0xee, 0x89, 0x3d, 0x49, + 0x3e, 0xd5, 0xcd, 0xb1, 0x6b, 0x16, 0x47, 0x53, 0xd9, 0x9a, 0xd9, 0x83, 0x72, 0x06, 0xee, 0xdd, + 0xfe, 0x3b, 0x33, 0x96, 0xad, 0xa3, 0xba, 0x35, 0x16, 0xdc, 0xc7, 0xbc, 0x4a, 0x33, 0x70, 0xef, + 0x8e, 0xa8, 0xd0, 0x73, 0x50, 0x7a, 0x9d, 0x3a, 0x3e, 0xe6, 0xf4, 0x54, 0xd5, 0x78, 0xec, 0xed, + 0x98, 0xd8, 0x97, 0xb0, 0x59, 0x8e, 0xf1, 0x83, 0x1a, 0xf4, 0x12, 0x6e, 0xb1, 0xde, 0xc3, 0x14, + 0xbc, 0x1e, 0x93, 0x3e, 0xb1, 0x97, 0x34, 0xfe, 0x36, 0x1f, 0xe9, 0x61, 0xdc, 0xb6, 0x5c, 0xde, + 0x0e, 0x14, 0x13, 0xc6, 0x46, 0x30, 0xd6, 0x84, 0x61, 0x8d, 0x40, 0x68, 0x56, 0xeb, 0x19, 0xa9, + 0x55, 0xef, 0x8c, 0x3d, 0x67, 0xa3, 0xe4, 0x97, 0x9d, 0x33, 0x3b, 0x2a, 0x45, 0x47, 0xb0, 0xd4, + 0x66, 0xa4, 0x15, 0xdb, 0x2e, 0x3b, 0x58, 0x9c, 0xb6, 0xaa, 0x77, 0xc7, 0xee, 0x6d, 0x1c, 0xc1, + 0x65, 0xeb, 0xd3, 0x1e, 0x94, 0xa3, 0x1f, 0x49, 0x9a, 0x73, 0x42, 0xf8, 0xce, 0xb2, 0x0a, 0xf8, + 0xee, 0x58, 0xe6, 0x14, 0x43, 0x72, 0x19, 0x73, 0xea, 0x05, 0x10, 0xd5, 0xef, 0x73, 0x58, 0xb1, + 0xa3, 0xac, 0x11, 0x37, 0x39, 0x6d, 0x54, 0xdf, 0xe3, 0x71, 0xdf, 0x8f, 0x9d, 0x7f, 0x3c, 0xc9, + 0x2c, 0x26, 0x8c, 0x65, 0x7b, 0x54, 0x87, 0x1e, 0xc3, 0xb2, 0x2d, 0x48, 0xa3, 0xe4, 0x7c, 0x62, + 0x2b, 0xef, 0x8d, 0x7c, 0xec, 0xe9, 0x0f, 0x30, 0x44, 0x31, 0x19, 0x86, 0xd9, 0xc3, 0x42, 0xd4, + 0x86, 0x75, 0xaf, 0xc7, 0x3b, 0x31, 0x39, 0x33, 0x1d, 0xdc, 0x0c, 0x99, 0xa7, 0x7a, 0x9f, 0x47, + 0xdf, 0x89, 0x2d, 0x68, 0xe3, 0xb9, 0x6a, 0x31, 0x61, 0xac, 0x79, 0xf1, 0xfa, 0xdc, 0x0c, 0x4c, + 0xf1, 0x0e, 0x6e, 0x3f, 0x95, 0xbe, 0xa6, 0x28, 0xfb, 0xa9, 0xf4, 0xb2, 0xb2, 0xb2, 0x9f, 0x4a, + 0xaf, 0x28, 0xd7, 0xf7, 0x53, 0xe9, 0xeb, 0xca, 0xea, 0x7e, 0x2a, 0xbd, 0xaa, 0xac, 0xed, 0xa7, + 0xd2, 0x6b, 0x8a, 0xba, 0x9f, 0x4a, 0xab, 0xca, 0xfa, 0x7e, 0x2a, 0xbd, 0xae, 0x6c, 0xec, 0xa7, + 0xd2, 0xb7, 0x94, 0xcc, 0x7e, 0x2a, 0x9d, 0x51, 0x36, 0xf7, 0x53, 0xe9, 0x77, 0x14, 0x4d, 0xbb, + 0xcf, 0x09, 0xf1, 0xa1, 0xe7, 0xf3, 0x72, 0x87, 0x36, 0x60, 0x8a, 0xad, 0xdb, 0xb9, 0x7c, 0x8f, + 0x10, 0xb4, 0x5b, 0x88, 0xb4, 0x2f, 0xa7, 0x60, 0x2a, 0xfc, 0x0a, 0x33, 0xf4, 0x4a, 0xb3, 0x2e, + 0x1f, 0x19, 0x96, 0x22, 0x5f, 0x3a, 0x84, 0x41, 0xff, 0xe9, 0xe6, 0x27, 0x83, 0x4c, 0x8f, 0x12, + 0xfe, 0x01, 0x87, 0xf3, 0xd8, 0xc5, 0xd8, 0xc3, 0x3d, 0x80, 0x6e, 0xdc, 0x38, 0x77, 0x47, 0x8e, + 0x73, 0xb3, 0x3f, 0xce, 0xa8, 0xd5, 0x00, 0x11, 0x94, 0x32, 0x94, 0x87, 0x85, 0xae, 0x4b, 0xce, + 0x3b, 0x9e, 0x4f, 0x2c, 0x4e, 0x33, 0x52, 0x57, 0xe9, 0x3b, 0x8c, 0xf9, 0x9e, 0x13, 0x23, 0x17, + 0x3b, 0x30, 0xe7, 0x51, 0xbb, 0x65, 0xbb, 0x98, 0x95, 0x5e, 0x4e, 0x52, 0xa7, 0x72, 0x8b, 0xf2, + 0xe9, 0x72, 0x9a, 0x95, 0xe9, 0x52, 0xc1, 0x00, 0x61, 0xc2, 0xfe, 0xa1, 0x43, 0x98, 0xb6, 0x78, + 0xa7, 0x21, 0x49, 0x67, 0x66, 0xdc, 0xf3, 0x89, 0xe8, 0x47, 0x72, 0xaa, 0x9c, 0x9f, 0xd2, 0x9f, + 0x9f, 0xd0, 0x18, 0x32, 0x0e, 0x3a, 0x80, 0x05, 0x06, 0x45, 0x56, 0x0f, 0x86, 0x04, 0x8d, 0xdb, + 0x8a, 0x04, 0x0e, 0x3f, 0xcd, 0x6e, 0xeb, 0xc2, 0x30, 0xfa, 0x3e, 0x33, 0x4f, 0x22, 0x32, 0xf4, + 0xdf, 0xe1, 0x6e, 0xcf, 0x5c, 0x96, 0x5d, 0x78, 0x38, 0xe4, 0x39, 0x40, 0x5d, 0x98, 0x74, 0xbd, + 0x97, 0x92, 0x95, 0xbe, 0xa5, 0x63, 0x2f, 0xc8, 0xc5, 0x79, 0x74, 0xf5, 0x17, 0x1b, 0x16, 0x20, + 0xef, 0x78, 0xcd, 0x17, 0xbd, 0x28, 0x06, 0x1b, 0x4f, 0x3c, 0x2a, 0x89, 0x27, 0xe0, 0x07, 0x7f, + 0x9f, 0x00, 0x75, 0xdc, 0x57, 0x06, 0xd6, 0x96, 0x66, 0x73, 0x35, 0xa3, 0x81, 0x47, 0x5e, 0xbb, + 0xef, 0xc2, 0x3b, 0x03, 0x1a, 0xfe, 0x47, 0x2f, 0x60, 0x43, 0xcf, 0xd7, 0x8c, 0x02, 0xde, 0xad, + 0x1d, 0x55, 0x0b, 0x4a, 0x92, 0xf5, 0xbd, 0x03, 0x66, 0xf9, 0x72, 0x49, 0xaf, 0xb2, 0x7f, 0xfb, + 0x7a, 0x9e, 0xf5, 0xc5, 0x9b, 0x70, 0x63, 0x40, 0x7f, 0x78, 0x54, 0x2f, 0xea, 0x46, 0x18, 0x4d, + 0x49, 0xa1, 0x1b, 0xb0, 0x36, 0x3a, 0x0e, 0xae, 0x1f, 0x66, 0xab, 0xca, 0x14, 0xfa, 0x1f, 0xf8, + 0x70, 0x40, 0x29, 0x07, 0xcf, 0x96, 0x0d, 0x3d, 0x5b, 0x78, 0x86, 0x9f, 0x18, 0xa5, 0x46, 0x43, + 0xaf, 0xe2, 0xc3, 0x5a, 0xbd, 0x5e, 0xca, 0x95, 0x75, 0xde, 0xe2, 0x67, 0x9f, 0x29, 0xd3, 0xe8, + 0x3d, 0xb8, 0x3d, 0xe0, 0x58, 0xd5, 0x9f, 0xe0, 0xb2, 0x9e, 0xad, 0xeb, 0xf8, 0xd0, 0xd0, 0x1f, + 0xeb, 0xd5, 0x46, 0x1d, 0x37, 0x9e, 0x56, 0x95, 0x34, 0xba, 0x0f, 0x77, 0x07, 0x0c, 0x1b, 0xa5, + 0x8a, 0x5e, 0x6f, 0x64, 0x2b, 0x87, 0x38, 0x9f, 0xcd, 0x17, 0x75, 0x39, 0x11, 0xbd, 0xa0, 0xcc, + 0x6c, 0xa4, 0xbe, 0xfc, 0x2a, 0x93, 0xd0, 0xd8, 0xa2, 0x4e, 0x3c, 0xf8, 0xf3, 0xe0, 0xc7, 0x8a, + 0xc8, 0x87, 0x0f, 0xd1, 0xe9, 0x37, 0x8c, 0x67, 0xa3, 0x4b, 0xca, 0x9f, 0x15, 0x98, 0x86, 0xe5, + 0xad, 0xe3, 0x46, 0xad, 0x86, 0x6b, 0x65, 0xb6, 0x88, 0xfc, 0x1d, 0x82, 0x29, 0xea, 0xba, 0x51, + 0xca, 0x96, 0x4b, 0x9f, 0x66, 0x73, 0x65, 0x5d, 0x99, 0x44, 0xb7, 0x60, 0x5d, 0xc8, 0xb3, 0xf5, + 0x67, 0xd5, 0xbc, 0x74, 0xdb, 0xcd, 0x96, 0xca, 0x47, 0x86, 0xae, 0x4c, 0x21, 0x0d, 0x32, 0x42, + 0x2d, 0xbe, 0x45, 0xe0, 0x82, 0x9e, 0x2d, 0x94, 0x4b, 0x55, 0x1d, 0xeb, 0x4f, 0xf3, 0xba, 0x5e, + 0xd0, 0x0b, 0xca, 0xb4, 0x48, 0xfa, 0xc1, 0x27, 0x80, 0x46, 0xef, 0x3e, 0x4a, 0x43, 0xaa, 0x5a, + 0xab, 0xea, 0x4a, 0x02, 0xcd, 0xc1, 0x4c, 0x2e, 0x9b, 0x3f, 0xa8, 0xed, 0xee, 0x2a, 0x49, 0xb4, + 0x00, 0xb3, 0xa5, 0x4a, 0x45, 0x2f, 0x94, 0xb2, 0x0d, 0x5d, 0x99, 0xc8, 0xdd, 0xff, 0xfa, 0x5f, + 0x99, 0xc4, 0xd7, 0xaf, 0x33, 0xc9, 0x6f, 0x5e, 0x67, 0x92, 0xdf, 0xbe, 0xce, 0x24, 0xff, 0xf9, + 0x3a, 0x93, 0xfc, 0xcd, 0x9b, 0x4c, 0xe2, 0x9b, 0x37, 0x99, 0xc4, 0xb7, 0x6f, 0x32, 0x89, 0x4f, + 0x67, 0x24, 0x1a, 0xfc, 0x27, 0x00, 0x00, 0xff, 0xff, 0xb5, 0x0e, 0xd4, 0xd3, 0x13, 0x21, 0x00, + 0x00, } func (m *NotLeaseHolderError) Marshal() (dAtA []byte, err error) { diff --git a/pkg/roachpb/errors.proto b/pkg/roachpb/errors.proto index 726c670c3ba5..50ee0945d87f 100644 --- a/pkg/roachpb/errors.proto +++ b/pkg/roachpb/errors.proto @@ -162,14 +162,15 @@ enum TransactionAbortedReason { // meantime because the transaction was aborted. ABORT_REASON_ABORT_SPAN = 5; - // A request attempting to create a transaction record encountered a write - // timestamp cache entry for the txn key, and the entry identifies this - // transaction. This means that the transaction definitely committed or rolled - // back before. So, this request is either a delayed replay of some sort, or - // it raced with an async abort and lost. If a client gets this - // TransactionAbortedError (without it being wrapped in an ambiguous error), - // it must be the latter case, and the transaction can be retried. - ABORT_REASON_ALREADY_COMMITTED_OR_ROLLED_BACK_POSSIBLE_REPLAY = 6; + // A request attempting to create a transaction record encountered a write + // timestamp cache entry for the txn key, and the entry identifies this + // transaction. This means that the transaction already wrote a transaction + // record previously, which may or may not have been removed since. So, this + // request is either a delayed replay of some sort, or it raced with an async + // abort and lost. If a client gets this TransactionAbortedError (without it + // being wrapped in an ambiguous error), it must be the latter case, and the + // transaction can be retried. + ABORT_REASON_RECORD_ALREADY_WRITTEN_POSSIBLE_REPLAY = 6; // A request attempting to create a transaction record is not allowed to // proceed by the timestamp cache because it cannot be verified that the