diff --git a/docs/generated/settings/settings.html b/docs/generated/settings/settings.html index 335dc0e26e93..76bb857bfdac 100644 --- a/docs/generated/settings/settings.html +++ b/docs/generated/settings/settings.html @@ -53,6 +53,7 @@ kv.snapshot_recovery.max_ratebyte size8.0 MiBthe rate limit (bytes/sec) to use for recovery snapshots kv.transaction.max_intents_bytesinteger262144maximum number of bytes used to track write intents in transactions kv.transaction.max_refresh_spans_bytesinteger256000maximum number of bytes used to track refresh spans in serializable transactions +kv.transaction.parallel_commits_enabledbooleantrueif enabled, transactional commits will be parallelized with transactional writes kv.transaction.write_pipelining_enabledbooleantrueif enabled, transactional writes are pipelined through Raft consensus kv.transaction.write_pipelining_max_batch_sizeinteger128if non-zero, defines that maximum size batch that will be pipelined through Raft consensus kv.transaction.write_pipelining_max_outstanding_sizebyte size256 KiBmaximum number of bytes used to track in-flight pipelined writes before disabling pipelining diff --git a/pkg/kv/dist_sender_server_test.go b/pkg/kv/dist_sender_server_test.go index d4c41bde6c0c..e7028a603180 100644 --- a/pkg/kv/dist_sender_server_test.go +++ b/pkg/kv/dist_sender_server_test.go @@ -1816,14 +1816,13 @@ func TestAsyncAbortPoisons(t *testing.T) { // Add a testing request filter which pauses a get request for the // key until after the signal channel is closed. var storeKnobs storage.StoreTestingKnobs - keyA := roachpb.Key("a") - var expectPoison int64 + keyA, keyB := roachpb.Key("a"), roachpb.Key("b") commitCh := make(chan error, 1) storeKnobs.TestingRequestFilter = func(ba roachpb.BatchRequest) *roachpb.Error { for _, req := range ba.Requests { switch r := req.GetInner().(type) { case *roachpb.EndTransactionRequest: - if r.Key.Equal(keyA) && atomic.LoadInt64(&expectPoison) == 1 { + if r.Key.Equal(keyA) { if r.Poison { close(commitCh) } else { @@ -1855,12 +1854,17 @@ func TestAsyncAbortPoisons(t *testing.T) { if err := txn.SetUserPriority(roachpb.MaxUserPriority); err != nil { return err } + // Write to keyB first to locate this txn's record on a different key + // than the initial txn's record. This allows the request filter to + // trivially ignore this transaction. + if err := txn.Put(ctx, keyB, []byte("value2")); err != nil { + return err + } return txn.Put(ctx, keyA, []byte("value2")) }); err != nil { t.Fatal(err) } - atomic.StoreInt64(&expectPoison, 1) expErr := regexp.QuoteMeta("TransactionAbortedError(ABORT_REASON_ABORT_SPAN)") if _, err := txn.Get(ctx, keyA); !testutils.IsError(err, expErr) { t.Fatalf("expected %s, got: %v", expErr, err) diff --git a/pkg/kv/txn_coord_sender.go b/pkg/kv/txn_coord_sender.go index e6f38386f561..09d14ccc0799 100644 --- a/pkg/kv/txn_coord_sender.go +++ b/pkg/kv/txn_coord_sender.go @@ -493,6 +493,11 @@ func (tcf *TxnCoordSenderFactory) TransactionalSender( tcs.stopper, tcs.cleanupTxnLocked, ) + tcs.interceptorAlloc.txnCommitter = txnCommitter{ + st: tcf.st, + stopper: tcs.stopper, + mu: &tcs.mu.Mutex, + } tcs.interceptorAlloc.txnMetricRecorder = txnMetricRecorder{ metrics: &tcs.metrics, clock: tcs.clock, diff --git a/pkg/kv/txn_coord_sender_test.go b/pkg/kv/txn_coord_sender_test.go index 0aead9daa97b..3cada5f9aaa1 100644 --- a/pkg/kv/txn_coord_sender_test.go +++ b/pkg/kv/txn_coord_sender_test.go @@ -232,14 +232,18 @@ func TestTxnCoordSenderCondenseIntentSpans(t *testing.T) { var sendFn simpleSendFn = func( _ context.Context, _ SendOptions, _ ReplicaSlice, args roachpb.BatchRequest, ) (*roachpb.BatchResponse, error) { + resp := args.CreateReply() + resp.Txn = args.Txn if req, ok := args.GetArg(roachpb.EndTransaction); ok { + if !req.(*roachpb.EndTransactionRequest).Commit { + t.Errorf("expected commit to be true") + } et := req.(*roachpb.EndTransactionRequest) if a, e := et.IntentSpans, expIntents; !reflect.DeepEqual(a, e) { t.Errorf("expected end transaction to have intents %+v; got %+v", e, a) } + resp.Txn.Status = roachpb.COMMITTED } - resp := args.CreateReply() - resp.Txn = args.Txn return resp, nil } ambient := log.AmbientContext{Tracer: tracing.NewTracer()} @@ -917,14 +921,15 @@ func TestTxnCoordSenderNoDuplicateIntents(t *testing.T) { var senderFn client.SenderFunc = func(_ context.Context, ba roachpb.BatchRequest) ( *roachpb.BatchResponse, *roachpb.Error) { + br := ba.CreateReply() + br.Txn = ba.Txn.Clone() if rArgs, ok := ba.GetArg(roachpb.EndTransaction); ok { et := rArgs.(*roachpb.EndTransactionRequest) if !reflect.DeepEqual(et.IntentSpans, expectedIntents) { t.Errorf("Invalid intents: %+v; expected %+v", et.IntentSpans, expectedIntents) } + br.Txn.Status = roachpb.COMMITTED } - br := ba.CreateReply() - br.Txn = ba.Txn.Clone() return br, nil } ambient := log.AmbientContext{Tracer: tracing.NewTracer()} @@ -1279,7 +1284,6 @@ func TestAbortTransactionOnCommitErrors(t *testing.T) { union := &br.Responses[0] // avoid operating on copy union.MustSetInner(&roachpb.PutResponse{}) if ba.Txn != nil && br.Txn == nil { - br.Txn = ba.Txn.Clone() br.Txn.Status = roachpb.PENDING } } else if et, hasET := ba.GetArg(roachpb.EndTransaction); hasET { @@ -1578,14 +1582,20 @@ func TestCommitMutatingTransaction(t *testing.T) { var calls []roachpb.Method sender.match(func(ba roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { + br := ba.CreateReply() + br.Txn = ba.Txn.Clone() + calls = append(calls, ba.Methods()...) if !bytes.Equal(ba.Txn.Key, roachpb.Key("a")) { t.Errorf("expected transaction key to be \"a\"; got %s", ba.Txn.Key) } - if et, ok := ba.GetArg(roachpb.EndTransaction); ok && !et.(*roachpb.EndTransactionRequest).Commit { - t.Errorf("expected commit to be true") + if et, ok := ba.GetArg(roachpb.EndTransaction); ok { + if !et.(*roachpb.EndTransactionRequest).Commit { + t.Errorf("expected commit to be true") + } + br.Txn.Status = roachpb.COMMITTED } - return nil, nil + return br, nil }) factory := NewTxnCoordSenderFactory( @@ -1668,14 +1678,20 @@ func TestTxnInsertBeginTransaction(t *testing.T) { var calls []roachpb.Method sender.match(func(ba roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { + br := ba.CreateReply() + br.Txn = ba.Txn.Clone() + calls = append(calls, ba.Methods()...) if bt, ok := ba.GetArg(roachpb.BeginTransaction); ok && !bt.Header().Key.Equal(roachpb.Key("a")) { t.Errorf("expected begin transaction key to be \"a\"; got %s", bt.Header().Key) } - if et, ok := ba.GetArg(roachpb.EndTransaction); ok && !et.(*roachpb.EndTransactionRequest).Commit { - t.Errorf("expected commit to be true") + if et, ok := ba.GetArg(roachpb.EndTransaction); ok { + if !et.(*roachpb.EndTransactionRequest).Commit { + t.Errorf("expected commit to be true") + } + br.Txn.Status = roachpb.COMMITTED } - return nil, nil + return br, nil }) v := cluster.VersionByKey(cluster.Version2_1) @@ -1822,13 +1838,19 @@ func TestEndWriteRestartReadOnlyTransaction(t *testing.T) { var calls []roachpb.Method sender.match(func(ba roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { + br := ba.CreateReply() + br.Txn = ba.Txn.Clone() + calls = append(calls, ba.Methods()...) if _, ok := ba.GetArg(roachpb.Put); ok { return nil, roachpb.NewErrorWithTxn( roachpb.NewTransactionRetryError(roachpb.RETRY_SERIALIZABLE, "test err"), ba.Txn) } - return nil, nil + if _, ok := ba.GetArg(roachpb.EndTransaction); ok { + br.Txn.Status = roachpb.COMMITTED + } + return br, nil }) factory := NewTxnCoordSenderFactory( @@ -1885,11 +1907,13 @@ func TestTransactionKeyNotChangedInRestart(t *testing.T) { keys := []string{"first", "second"} attempt := 0 sender.match(func(ba roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { + br := ba.CreateReply() + br.Txn = ba.Txn.Clone() + // Ignore the final EndTxnRequest. if _, ok := ba.GetArg(roachpb.EndTransaction); ok { - br := ba.CreateReply() - br.Txn = ba.Txn.Clone() - return nil, nil + br.Txn.Status = roachpb.COMMITTED + return br, nil } // Both attempts should have a PutRequest. @@ -1908,7 +1932,7 @@ func TestTransactionKeyNotChangedInRestart(t *testing.T) { roachpb.NewTransactionRetryError(roachpb.RETRY_SERIALIZABLE, "test err"), ba.Txn) } - return nil, nil + return br, nil }) factory := NewTxnCoordSenderFactory( TxnCoordSenderFactoryConfig{ @@ -2005,7 +2029,13 @@ func TestConcurrentTxnRequests(t *testing.T) { callCounts[m]++ } callCountsMu.Unlock() - return nil, nil + + br := ba.CreateReply() + br.Txn = ba.Txn.Clone() + if _, ok := ba.GetArg(roachpb.EndTransaction); ok { + br.Txn.Status = roachpb.COMMITTED + } + return br, nil }) factory := NewTxnCoordSenderFactory( TxnCoordSenderFactoryConfig{ @@ -2186,6 +2216,10 @@ func TestTxnCoordSenderPipelining(t *testing.T) { ctx context.Context, ba roachpb.BatchRequest, ) (*roachpb.BatchResponse, *roachpb.Error) { calls = append(calls, ba.Methods()...) + if et, ok := ba.GetArg(roachpb.EndTransaction); ok { + // Ensure that no transactions enter a STAGING state. + et.(*roachpb.EndTransactionRequest).InFlightWrites = nil + } return distSender.Send(ctx, ba) } @@ -2259,6 +2293,9 @@ func TestAnchorKey(t *testing.T) { } br := ba.CreateReply() br.Txn = ba.Txn.Clone() + if _, ok := ba.GetArg(roachpb.EndTransaction); ok { + br.Txn.Status = roachpb.COMMITTED + } return br, nil } diff --git a/pkg/kv/txn_correctness_test.go b/pkg/kv/txn_correctness_test.go index 4da2a29e6b66..1bf6bbc5405d 100644 --- a/pkg/kv/txn_correctness_test.go +++ b/pkg/kv/txn_correctness_test.go @@ -30,6 +30,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/internal/client" "github.com/cockroachdb/cockroach/pkg/storage" "github.com/cockroachdb/cockroach/pkg/storage/engine/enginepb" + "github.com/cockroachdb/cockroach/pkg/storage/storagebase" "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/testutils/localtestcluster" "github.com/cockroachdb/cockroach/pkg/util/leaktest" @@ -649,7 +650,7 @@ func (hv *historyVerifier) runHistory( } // Execute pre-history if applicable. if hv.preHistoryCmds != nil { - if str, _, err := hv.runCmds(hv.preHistoryCmds, db, t); err != nil { + if str, _, err := hv.runCmds("pre-history", hv.preHistoryCmds, db, t); err != nil { t.Errorf("failed on execution of pre history %s: %s", str, err) return err } @@ -711,7 +712,7 @@ func (hv *historyVerifier) runHistory( actualStr := strings.Join(hv.mu.actual, " ") // Verify history. - verifyStr, verifyEnv, err := hv.runCmds(hv.verifyCmds, db, t) + verifyStr, verifyEnv, err := hv.runCmds("verify", hv.verifyCmds, db, t) if err != nil { t.Errorf("failed on execution of verification history %s: %s", verifyStr, err) return err @@ -730,11 +731,12 @@ func (hv *historyVerifier) runHistory( } func (hv *historyVerifier) runCmds( - cmds []*cmd, db *client.DB, t *testing.T, + txnName string, cmds []*cmd, db *client.DB, t *testing.T, ) (string, map[string]int64, error) { var strs []string env := map[string]int64{} err := db.Txn(context.TODO(), func(ctx context.Context, txn *client.Txn) error { + txn.SetDebugName(txnName) for _, c := range cmds { c.historyIdx = hv.idx c.env = env @@ -821,6 +823,16 @@ func checkConcurrency(name string, txns []string, verify *verifier, t *testing.T s := &localtestcluster.LocalTestCluster{ StoreTestingKnobs: &storage.StoreTestingKnobs{ DontRetryPushTxnFailures: true, + // Immediately attempt to recover pushed transactions with STAGING + // statuses, even if the push would otherwise fail because the + // pushee has not yet expired. This prevents low-priority pushes from + // occasionally throwing retry errors due to DontRetryPushTxnFailures + // after the pushee's commit has already returned successfully. This + // is a result of the asynchronous nature of making transaction commits + // explicit after a parallel commit. + EvalKnobs: storagebase.BatchEvalTestingKnobs{ + RecoverIndeterminateCommitsOnFailedPushes: true, + }, }, } s.Start(t, testutils.NewNodeTestBaseContext(), InitFactoryForLocalTestCluster) diff --git a/pkg/kv/txn_interceptor_committer.go b/pkg/kv/txn_interceptor_committer.go index 9eb77edbbe17..623b50683f3e 100644 --- a/pkg/kv/txn_interceptor_committer.go +++ b/pkg/kv/txn_interceptor_committer.go @@ -16,22 +16,108 @@ package kv import ( "context" + "sync" "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/settings" + "github.com/cockroachdb/cockroach/pkg/settings/cluster" + "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/cockroachdb/cockroach/pkg/util/stop" +) + +var parallelCommitsEnabled = settings.RegisterBoolSetting( + "kv.transaction.parallel_commits_enabled", + "if enabled, transactional commits will be parallelized with transactional writes", + true, ) // txnCommitter is a txnInterceptor that concerns itself with committing and // rolling back transactions. It intercepts EndTransaction requests and -// coordinates their execution. This is either accomplished by issuing them -// directly with proper addressing, eliding them when they are not needed, or -// (eventually) coordinating the execution of committing EndTransaction requests -// in parallel with the rest of their batch. +// coordinates their execution. This is accomplished either by issuing them +// directly with proper addressing if they are alone, eliding them if they are +// not needed, or coordinating their execution in parallel with the rest of +// their batch if they are part of a larger set of requests. +// +// The third operation listed, which we define as a "parallel commit", is the +// most interesting. Marking a transaction record as committed in parallel with +// writing the rest of the transaction's intents is a clear win in terms of +// latency - in theory it removes the cost of an entire consensus round-trip +// from a transaction. However, doing so safely comes with extra complication. +// It requires an extension to the transaction model, additional client-side +// logic, buy-in from concurrency control, and specialized support from a +// transaction recovery mechanism. txnCommitter is responsible for the parallel +// commit-specific client-side logic. +// +// Parallel commits works by defining a committed transaction as a transaction +// that meets one of the two following commit conditions: +// 1. a transaction is *explicitly committed* if it has a transaction record with +// a COMMITTED status +// 2. a transaction is *implicitly committed* if it has a transaction record with +// a STAGING status and intents written for all writes declared as "in-flight" +// on the transaction record at equal or lower timestamps than the transaction +// record's commit timestamp +// +// A transaction may move from satisfying the implicit commit condition to +// satisfying the explicit commit condition. This is desirable because it moves +// the commit condition from a distributed condition to one local to the +// transaction record. Regardless, once either commit condition is satisfied, a +// transaction will remain committed in perpetuity both to itself and to all +// concurrent observers. +// +// The txnCommitter interceptor's role in this is to determine the set of writes +// that will be in-flight during a parallel commit. It collects this set from +// both the writes and the query intent requests that it finds present in the +// same batch as the committing end transaction request. The writes in this +// batch indicate a new intent write and the query intent requests indicate a +// previous pipelined intent write that has not yet been proven as successful. +// Before issuing the batch, the txnCommitter attaches this set to the end +// transaction request. +// +// The txnCommitter then collects the response of the batch when it returns. +// Based on the outcome of the requests in the batch, the interceptor determines +// whether the transaction successfully committed by satisfying the implicit +// commit condition. // -// NOTE: the primary justification for introducing this interceptor is that it -// will house all of the coordinator-side logic for parallel commits. Once -// complete, there will be a nice description of parallel commits here. +// If all requests in the batch succeeded (including the EndTransaction request) +// then the implicit commit condition is satisfied. The interceptor returns a +// successful response up then stack and launches an async task to make the +// commit explicit by moving the transaction record's status from STAGING to +// COMMITTED. +// +// If all requests did not succeed then the implicit commit condition is not +// satisfied and the transaction is still in-progress (and could still be +// committed or aborted at a later time). There are a number of reasons why +// some of the requests in the final batch may have failed: +// - intent writes: these requests may fail to write an intent due to a logical +// error like a ConditionFailedError. They also could have succeeded at writing +// an intent but failed to write it at the desired timestamp because they ran +// into the timestamp cache or another committed value. In the first case, the +// txnCommitter will receive an error. In the second, it will generate one in +// needTxnRetryAfterStaging. +// - query intents: these requests may fail because they discover that one of the +// previously issued writes has failed; either because it never left an intent +// or because it left one at too high of a timestamp. In this case, the request +// will return an error because the requests all have the ErrorIfMissing option +// set. It will also prevent the write from ever succeeding in the future, which +// ensures that the transaction will never suddenly become implicitly committed +// at a later point due to the write eventually succeeding (e.g. after a replay). +// - end txn: this request may fail with a TransactionRetryError for any number of +// reasons, such as if the transaction's provisional commit timestamp has been +// pushed past its read timestamp. In all of these cases, an error will be +// returned and the transaction record will not be staged. +// +// If it is unknown whether all of the requests in the final batch succeeded +// (e.g. due to a network error) then an AmbiguousResultError is returned. The +// logic to enforce this is in DistSender. +// TODO(nvanbenschoten): merge this logic. +// +// In all cases, the interceptor abstracts away the details of this from all +// interceptors above it in the coordinator interceptor stack. type txnCommitter struct { + st *cluster.Settings + stopper *stop.Stopper wrapped lockedSender + mu sync.Locker } // SendLocked implements the lockedSender interface. @@ -47,8 +133,8 @@ func (tc *txnCommitter) SendLocked( // Determine whether we can elide the EndTransaction entirely. We can do // so if the transaction is read-only, which we determine based on whether - // the EndTransaction request contains any intents. - if len(et.IntentSpans) == 0 { + // the EndTransaction request contains any writes. + if len(et.IntentSpans) == 0 && len(et.InFlightWrites) == 0 { return tc.sendLockedWithElidedEndTransaction(ctx, ba, et) } @@ -60,8 +146,91 @@ func (tc *txnCommitter) SendLocked( et.Key = ba.Txn.Key } - // Pass the adjusted batch through the wrapped lockedSender. - return tc.wrapped.SendLocked(ctx, ba) + // Determine whether the commit can be run in parallel with the rest of the + // writes in the batch. If not, move the in-flight writes currently attached + // to the EndTransaction request to the IntentSpans and clear the in-flight + // write set; no writes will be in-flight concurrently with the EndTransaction + // request. + if len(et.InFlightWrites) > 0 && !tc.canCommitInParallelWithWrites(ba, et) { + et.IntentSpans = mergeIntoSpans(et.IntentSpans, et.InFlightWrites) + // Disable parallel commits. + et.InFlightWrites = nil + } + + // If the EndTransaction request is a rollback, pass it through. + if !et.Commit { + return tc.wrapped.SendLocked(ctx, ba) + } + + // Send the adjusted batch through the wrapped lockedSender. Unlocks while + // sending then re-locks. + br, pErr := tc.wrapped.SendLocked(ctx, ba) + if pErr != nil { + // If the batch resulted in an error but the EndTransaction request + // succeeded, staging the transaction record in the process, downgrade + // the status back to PENDING. Even though the transaction record may + // have a status of STAGING, we know that the transaction failed to + // implicitly commit, so interceptors above the txnCommitter in the + // stack don't need to be made aware that the record is staging. + if txn := pErr.GetTxn(); txn != nil && txn.Status == roachpb.STAGING { + pErr.SetTxn(cloneWithStatus(txn, roachpb.PENDING)) + } + // Same deal with MixedSuccessErrors. + // TODO(nvanbenschoten): We can remove this once MixedSuccessErrors + // are removed. + if aPSErr, ok := pErr.GetDetail().(*roachpb.MixedSuccessError); ok { + if txn := aPSErr.Wrapped.GetTxn(); txn != nil && txn.Status == roachpb.STAGING { + aPSErr.Wrapped.SetTxn(cloneWithStatus(txn, roachpb.PENDING)) + } + } + return nil, pErr + } + + // Determine next steps based on the status of the transaction. + switch br.Txn.Status { + case roachpb.STAGING: + // Continue with STAGING-specific validation and cleanup. + case roachpb.COMMITTED: + // The transaction is explicitly committed. This is possible if all + // in-flight writes were sent to the same range as the EndTransaction + // request, in a single batch. In this case, a range can determine that + // all in-flight writes will succeed with the EndTransaction and can + // decide to skip the STAGING state. + // + // This is also possible if we never attached any in-flight writes to the + // EndTransaction request, either because canCommitInParallelWithWrites + // returned false or because there were no unproven in-flight writes + // (see txnPipeliner) and there were no writes in the batch request. + return br, nil + default: + return nil, roachpb.NewErrorf("unexpected response status without error: %v", br.Txn) + } + + // Determine whether the transaction needs to either retry or refresh. When + // the EndTransaction request evaluated while STAGING the transaction + // record, it performed this check. However, the transaction proto may have + // changed due to writes evaluated concurrently with the EndTransaction even + // if none of those writes returned an error. Remember that the transaction + // proto we see here could be a combination of protos from responses, all + // merged by DistSender. + if pErr := needTxnRetryAfterStaging(br); pErr != nil { + return nil, pErr + } + + // If the transaction doesn't need to retry then it is implicitly committed! + // We're the only ones who know that though -- other concurrent transactions + // will need to go through the full status resolution process to make a + // determination about the status of our STAGING transaction. To avoid this, + // we transition to an explicitly committed transaction as soon as possible. + // This also has the side-effect of kicking off intent resolution. + mergedIntentSpans := mergeIntoSpans(et.IntentSpans, et.InFlightWrites) + tc.makeTxnCommitExplicitAsync(ctx, br.Txn, mergedIntentSpans) + + // Switch the status on the batch response's transaction to COMMITTED. No + // interceptor above this one in the stack should ever need to deal with + // transaction proto in the STAGING state. + br.Txn = cloneWithStatus(br.Txn, roachpb.COMMITTED) + return br, nil } // sendLockedWithElidedEndTransaction sends the provided batch without its @@ -114,6 +283,134 @@ func (tc *txnCommitter) sendLockedWithElidedEndTransaction( return br, nil } +// canCommitInParallelWithWrites determines whether the batch can issue its +// committing EndTransaction in parallel with other in-flight writes. +func (tc *txnCommitter) canCommitInParallelWithWrites( + ba roachpb.BatchRequest, et *roachpb.EndTransactionRequest, +) bool { + if !parallelCommitsEnabled.Get(&tc.st.SV) { + return false + } + + // We're trying to parallel commit, not parallel abort. + if !et.Commit { + return false + } + + // If the transaction has a commit trigger, we don't allow it to commit in + // parallel with writes. There's no fundamental reason for this restriction, + // but for now it's not worth the complication. + if et.InternalCommitTrigger != nil { + return false + } + + // Similar to how we can't pipeline ranged writes, we also can't commit in + // parallel with them. The reason for this is that the status resolution + // process for STAGING transactions wouldn't know where to look for the + // intents. + for _, ru := range ba.Requests[:len(ba.Requests)-1] { + req := ru.GetInner() + if roachpb.IsTransactionWrite(req) && roachpb.IsRange(req) { + return false + } + } + return true +} + +func mergeIntoSpans(s []roachpb.Span, ws []roachpb.SequencedWrite) []roachpb.Span { + if s == nil { + s = make([]roachpb.Span, 0, len(ws)) + } + for _, w := range ws { + s = append(s, roachpb.Span{Key: w.Key}) + } + s, _ = roachpb.MergeSpans(s) + return s +} + +// needTxnRetryAfterStaging determines whether the transaction needs to refresh +// (see txnSpanRefresher) or retry based on the batch response of a parallel +// commit attempt. +func needTxnRetryAfterStaging(br *roachpb.BatchResponse) *roachpb.Error { + if len(br.Responses) == 0 { + return roachpb.NewErrorf("no responses in BatchResponse: %v", br) + } + lastResp := br.Responses[len(br.Responses)-1].GetInner() + etResp, ok := lastResp.(*roachpb.EndTransactionResponse) + if !ok { + return roachpb.NewErrorf("unexpected response in BatchResponse: %v", lastResp) + } + if etResp.StagingTimestamp.IsEmpty() { + return roachpb.NewErrorf("empty StagingTimestamp in EndTransactionResponse: %v", etResp) + } + if etResp.StagingTimestamp.Less(br.Txn.Timestamp) { + // If the timestamp that the transaction record was staged at + // is less than the timestamp of the transaction in the batch + // response then one of the concurrent writes was pushed to + // a higher timestamp. This violates the "implicit commit" + // condition and neither the transaction coordinator nor any + // other concurrent actor will consider this transaction to + // be committed as is. + err := roachpb.NewTransactionRetryError(roachpb.RETRY_SERIALIZABLE, "" /* extraMsg */) + txn := cloneWithStatus(br.Txn, roachpb.PENDING) + return roachpb.NewErrorWithTxn(err, txn) + } + return nil +} + +// makeTxnCommitExplicitAsync launches an async task that attempts to move the +// transaction from implicitly committed (STAGING status with all intents +// written) to explicitly committed (COMMITTED status). It does so by sending a +// second EndTransactionRequest, this time with no InFlightWrites attached. +func (tc *txnCommitter) makeTxnCommitExplicitAsync( + ctx context.Context, txn *roachpb.Transaction, intentSpans []roachpb.Span, +) { + // TODO(nvanbenschoten): consider adding tracing for this request. + // TODO(nvanbenschoten): add a timeout to this request. + // TODO(nvanbenschoten): consider making this semi-synchronous to + // backpressure client writes when these start to slow down. This + // would be similar to what we do for intent resolution. + log.VEventf(ctx, 2, "making txn commit explicit: %s", txn) + if err := tc.stopper.RunAsyncTask( + context.Background(), "txnCommitter: making txn commit explicit", func(ctx context.Context) { + tc.mu.Lock() + defer tc.mu.Unlock() + if err := makeTxnCommitExplicitLocked(ctx, tc.wrapped, txn, intentSpans); err != nil { + log.VErrEventf(ctx, 1, "making txn commit explicit failed for %s: %v", txn, err) + } + }, + ); err != nil { + log.VErrEventf(ctx, 1, "failed to make txn commit explicit: %v", err) + } +} + +func makeTxnCommitExplicitLocked( + ctx context.Context, s lockedSender, txn *roachpb.Transaction, intentSpans []roachpb.Span, +) error { + // Clone the txn to prevent data races. + txn = txn.Clone() + + // Construct a new batch with just an EndTransaction request. + ba := roachpb.BatchRequest{} + ba.Header = roachpb.Header{Txn: txn} + et := roachpb.EndTransactionRequest{Commit: true} + et.Key = txn.Key + et.IntentSpans = intentSpans + ba.Add(&et) + + _, pErr := s.SendLocked(ctx, ba) + if pErr != nil { + // Detect whether the error indicates that someone else beat + // us to explicitly committing the transaction record. + tse, ok := pErr.GetDetail().(*roachpb.TransactionStatusError) + if ok && tse.Reason == roachpb.TransactionStatusError_REASON_TXN_COMMITTED { + return nil + } + return pErr.GoError() + } + return nil +} + // setWrapped implements the txnInterceptor interface. func (tc *txnCommitter) setWrapped(wrapped lockedSender) { tc.wrapped = wrapped } diff --git a/pkg/kv/txn_interceptor_committer_test.go b/pkg/kv/txn_interceptor_committer_test.go index d1a715046879..d02efc9937c7 100644 --- a/pkg/kv/txn_interceptor_committer_test.go +++ b/pkg/kv/txn_interceptor_committer_test.go @@ -19,24 +19,96 @@ import ( "testing" "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/util/leaktest" + "github.com/cockroachdb/cockroach/pkg/util/stop" + "github.com/cockroachdb/cockroach/pkg/util/syncutil" "github.com/stretchr/testify/require" ) func makeMockTxnCommitter() (txnCommitter, *mockLockedSender) { mockSender := &mockLockedSender{} return txnCommitter{ + st: cluster.MakeTestingClusterSettings(), + stopper: stop.NewStopper(), wrapped: mockSender, + mu: new(syncutil.Mutex), }, mockSender } +// TestTxnCommitterElideEndTransaction tests that EndTransaction requests for +// read-only transactions are removed from their batches because they are not +// necessary. The test verifies the case where the EndTransaction request is +// part of a batch with other requests and the case where it is alone in its +// batch. +func TestTxnCommitterElideEndTransaction(t *testing.T) { + defer leaktest.AfterTest(t)() + ctx := context.Background() + tc, mockSender := makeMockTxnCommitter() + defer tc.stopper.Stop(ctx) + + txn := makeTxnProto() + keyA := roachpb.Key("a") + + // Test with both commits and rollbacks. + testutils.RunTrueAndFalse(t, "commit", func(t *testing.T, commit bool) { + expStatus := roachpb.COMMITTED + if !commit { + expStatus = roachpb.ABORTED + } + + // Test the case where the EndTransaction request is part of a larger + // batch of requests. + var ba roachpb.BatchRequest + ba.Header = roachpb.Header{Txn: &txn} + ba.Add(&roachpb.GetRequest{RequestHeader: roachpb.RequestHeader{Key: keyA}}) + ba.Add(&roachpb.PutRequest{RequestHeader: roachpb.RequestHeader{Key: keyA}}) + ba.Add(&roachpb.EndTransactionRequest{Commit: commit, IntentSpans: nil}) + + mockSender.MockSend(func(ba roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { + require.Len(t, ba.Requests, 2) + require.IsType(t, &roachpb.GetRequest{}, ba.Requests[0].GetInner()) + require.IsType(t, &roachpb.PutRequest{}, ba.Requests[1].GetInner()) + + br := ba.CreateReply() + br.Txn = ba.Txn + // The Sender did not receive an EndTransaction request, so it keeps + // the Txn status as PENDING. + br.Txn.Status = roachpb.PENDING + return br, nil + }) + + br, pErr := tc.SendLocked(ctx, ba) + require.Nil(t, pErr) + require.NotNil(t, br) + require.NotNil(t, br.Txn) + require.Equal(t, expStatus, br.Txn.Status) + + // Test the case where the EndTransaction request is alone. + ba.Requests = nil + ba.Add(&roachpb.EndTransactionRequest{Commit: commit, IntentSpans: nil}) + + mockSender.MockSend(func(ba roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { + require.Fail(t, "should not have issued batch request", ba) + return nil, nil + }) + + br, pErr = tc.SendLocked(ctx, ba) + require.Nil(t, pErr) + require.NotNil(t, br) + require.NotNil(t, br.Txn) + require.Equal(t, expStatus, br.Txn.Status) + }) +} + // TestTxnCommitterAttachesTxnKey tests that the txnCommitter attaches the // transaction key to committing and aborting EndTransaction requests. func TestTxnCommitterAttachesTxnKey(t *testing.T) { defer leaktest.AfterTest(t)() ctx := context.Background() tc, mockSender := makeMockTxnCommitter() + defer tc.stopper.Stop(ctx) txn := makeTxnProto() keyA := roachpb.Key("a") @@ -63,8 +135,8 @@ func TestTxnCommitterAttachesTxnKey(t *testing.T) { }) br, pErr := tc.SendLocked(ctx, ba) - require.NotNil(t, br) require.Nil(t, pErr) + require.NotNil(t, br) // Verify that the txn key is attached to aborting EndTransaction requests. ba.Requests = nil @@ -81,70 +153,271 @@ func TestTxnCommitterAttachesTxnKey(t *testing.T) { }) br, pErr = tc.SendLocked(ctx, ba) + require.Nil(t, pErr) + require.NotNil(t, br) +} + +// TestTxnCommitterStripsInFlightWrites tests that the txnCommitter strips the +// pipelined writes that have yet to be proven and the new writes that are part +// of the same batch as an EndTransaction request from the in-flight write set +// when a parallel commit is not desired. It also tests that it keeps the +// in-flight writes attached to the EndTransaction request when a parallel +// commit is desired. +func TestTxnCommitterStripsInFlightWrites(t *testing.T) { + defer leaktest.AfterTest(t)() + + ctx := context.Background() + tc, mockSender := makeMockTxnCommitter() + defer tc.stopper.Stop(ctx) + + // Start with parallel commits disabled. Should NOT attach in-flight writes. + parallelCommitsEnabled.Override(&tc.st.SV, false) + + txn := makeTxnProto() + keyA, keyB := roachpb.Key("a"), roachpb.Key("b") + + // Verify that the QueryIntent and the Put are both attached as intent spans + // to the committing EndTransaction request when expected. + var ba roachpb.BatchRequest + ba.Header = roachpb.Header{Txn: &txn} + qiArgs := roachpb.QueryIntentRequest{RequestHeader: roachpb.RequestHeader{Key: keyA}} + putArgs := roachpb.PutRequest{RequestHeader: roachpb.RequestHeader{Key: keyB}} + etArgs := roachpb.EndTransactionRequest{Commit: true} + qiArgs.Txn.Sequence = 1 + putArgs.Sequence = 2 + etArgs.Sequence = 3 + etArgs.InFlightWrites = []roachpb.SequencedWrite{ + {Key: keyA, Sequence: 1}, {Key: keyB, Sequence: 2}, + } + etArgsCopy := etArgs + ba.Add(&putArgs, &qiArgs, &etArgsCopy) + + mockSender.MockSend(func(ba roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { + require.Len(t, ba.Requests, 3) + require.IsType(t, &roachpb.EndTransactionRequest{}, ba.Requests[2].GetInner()) + + et := ba.Requests[2].GetInner().(*roachpb.EndTransactionRequest) + require.True(t, et.Commit) + require.Len(t, et.IntentSpans, 2) + require.Equal(t, []roachpb.Span{{Key: keyA}, {Key: keyB}}, et.IntentSpans) + require.Len(t, et.InFlightWrites, 0) + + br := ba.CreateReply() + br.Txn = ba.Txn + br.Txn.Status = roachpb.COMMITTED + return br, nil + }) + + br, pErr := tc.SendLocked(ctx, ba) + require.Nil(t, pErr) + require.NotNil(t, br) + + // Enable parallel commits and send the same batch. Should attach in-flight writes. + parallelCommitsEnabled.Override(&tc.st.SV, true) + + ba.Requests = nil + etArgsCopy = etArgs + ba.Add(&putArgs, &qiArgs, &etArgsCopy) + + mockSender.MockSend(func(ba roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { + require.Len(t, ba.Requests, 3) + require.IsType(t, &roachpb.EndTransactionRequest{}, ba.Requests[2].GetInner()) + + et := ba.Requests[2].GetInner().(*roachpb.EndTransactionRequest) + require.True(t, et.Commit) + require.Len(t, et.InFlightWrites, 2) + require.Equal(t, roachpb.SequencedWrite{Key: keyA, Sequence: 1}, et.InFlightWrites[0]) + require.Equal(t, roachpb.SequencedWrite{Key: keyB, Sequence: 2}, et.InFlightWrites[1]) + + br = ba.CreateReply() + br.Txn = ba.Txn + br.Txn.Status = roachpb.COMMITTED + return br, nil + }) + + br, pErr = tc.SendLocked(ctx, ba) + require.Nil(t, pErr) require.NotNil(t, br) + + // Send the same batch but with an EndTransaction containing a commit trigger. + // In-flight writes should not be attached because commit triggers disable + // parallel commits. + ba.Requests = nil + etArgsWithTrigger := etArgs + etArgsWithTrigger.InternalCommitTrigger = &roachpb.InternalCommitTrigger{ + ModifiedSpanTrigger: &roachpb.ModifiedSpanTrigger{ + SystemConfigSpan: true, + }, + } + ba.Add(&putArgs, &qiArgs, &etArgsWithTrigger) + + mockSender.MockSend(func(ba roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { + require.Len(t, ba.Requests, 3) + require.IsType(t, &roachpb.EndTransactionRequest{}, ba.Requests[2].GetInner()) + + et := ba.Requests[2].GetInner().(*roachpb.EndTransactionRequest) + require.True(t, et.Commit) + require.Len(t, et.IntentSpans, 2) + require.Equal(t, []roachpb.Span{{Key: keyA}, {Key: keyB}}, et.IntentSpans) + require.Len(t, et.InFlightWrites, 0) + + br = ba.CreateReply() + br.Txn = ba.Txn + br.Txn.Status = roachpb.COMMITTED + return br, nil + }) + + br, pErr = tc.SendLocked(ctx, ba) + require.Nil(t, pErr) + require.NotNil(t, br) + + // Send the same batch but with a ranged write instead of a point write. + // In-flight writes should not be attached because ranged writes cannot + // be parallelized with a commit. + ba.Requests = nil + delRngArgs := roachpb.DeleteRangeRequest{RequestHeader: roachpb.RequestHeader{Key: keyA, EndKey: keyB}} + delRngArgs.Sequence = 2 + etArgsWithRangedIntentSpan := etArgs + etArgsWithRangedIntentSpan.IntentSpans = []roachpb.Span{{Key: keyA, EndKey: keyB}} + etArgsWithRangedIntentSpan.InFlightWrites = []roachpb.SequencedWrite{{Key: keyA, Sequence: 1}} + ba.Add(&delRngArgs, &qiArgs, &etArgsWithRangedIntentSpan) + + mockSender.MockSend(func(ba roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { + require.Len(t, ba.Requests, 3) + require.IsType(t, &roachpb.EndTransactionRequest{}, ba.Requests[2].GetInner()) + + et := ba.Requests[2].GetInner().(*roachpb.EndTransactionRequest) + require.True(t, et.Commit) + require.Len(t, et.IntentSpans, 1) + require.Equal(t, []roachpb.Span{{Key: keyA, EndKey: keyB}}, et.IntentSpans) + require.Len(t, et.InFlightWrites, 0) + + br = ba.CreateReply() + br.Txn = ba.Txn + br.Txn.Status = roachpb.COMMITTED + return br, nil + }) + + br, pErr = tc.SendLocked(ctx, ba) require.Nil(t, pErr) + require.NotNil(t, br) } -// TestTxnCommitterElideEndTransaction tests that EndTransaction requests for -// read-only transactions are removed from their batches because they are not -// necessary. The test verifies the case where the EndTransaction request is -// part of a batch with other requests and the case where it is alone in its -// batch. -func TestTxnCommitterElideEndTransaction(t *testing.T) { +// TestTxnCommitterAsyncExplicitCommitTask verifies that when txnCommitter +// performs a parallel commit and receives a STAGING transaction status, +// it launches an async task to make the transaction commit explicit. +func TestTxnCommitterAsyncExplicitCommitTask(t *testing.T) { defer leaktest.AfterTest(t)() ctx := context.Background() tc, mockSender := makeMockTxnCommitter() + defer tc.stopper.Stop(ctx) txn := makeTxnProto() keyA := roachpb.Key("a") - // Test with both commits and rollbacks. - testutils.RunTrueAndFalse(t, "commit", func(t *testing.T, commit bool) { - expStatus := roachpb.COMMITTED - if !commit { - expStatus = roachpb.ABORTED - } + // Verify that the Put is attached as in-flight write to the committing + // EndTransaction request. + var ba roachpb.BatchRequest + ba.Header = roachpb.Header{Txn: &txn} + putArgs := roachpb.PutRequest{RequestHeader: roachpb.RequestHeader{Key: keyA}} + etArgs := roachpb.EndTransactionRequest{Commit: true} + putArgs.Sequence = 1 + etArgs.Sequence = 2 + etArgs.InFlightWrites = []roachpb.SequencedWrite{{Key: keyA, Sequence: 1}} + ba.Add(&putArgs, &etArgs) - // Test the case where the EndTransaction request is part of a larger - // batch of requests. - var ba roachpb.BatchRequest - ba.Header = roachpb.Header{Txn: &txn} - ba.Add(&roachpb.GetRequest{RequestHeader: roachpb.RequestHeader{Key: keyA}}) - ba.Add(&roachpb.PutRequest{RequestHeader: roachpb.RequestHeader{Key: keyA}}) - ba.Add(&roachpb.EndTransactionRequest{Commit: commit, IntentSpans: nil}) + explicitCommitCh := make(chan struct{}) + mockSender.MockSend(func(ba roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { + require.Len(t, ba.Requests, 2) + require.IsType(t, &roachpb.PutRequest{}, ba.Requests[0].GetInner()) + require.IsType(t, &roachpb.EndTransactionRequest{}, ba.Requests[1].GetInner()) + + et := ba.Requests[1].GetInner().(*roachpb.EndTransactionRequest) + require.True(t, et.Commit) + require.Len(t, et.InFlightWrites, 1) + require.Equal(t, roachpb.SequencedWrite{Key: keyA, Sequence: 1}, et.InFlightWrites[0]) + br := ba.CreateReply() + br.Txn = ba.Txn + br.Txn.Status = roachpb.STAGING + br.Responses[1].GetInner().(*roachpb.EndTransactionResponse).StagingTimestamp = br.Txn.Timestamp + + // Before returning, mock out the sender again to test against the async + // task that should be sent to make the implicit txn commit explicit. mockSender.MockSend(func(ba roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { - require.Len(t, ba.Requests, 2) - require.IsType(t, &roachpb.GetRequest{}, ba.Requests[0].GetInner()) - require.IsType(t, &roachpb.PutRequest{}, ba.Requests[1].GetInner()) + defer close(explicitCommitCh) + require.Len(t, ba.Requests, 1) + require.IsType(t, &roachpb.EndTransactionRequest{}, ba.Requests[0].GetInner()) - br := ba.CreateReply() + et := ba.Requests[0].GetInner().(*roachpb.EndTransactionRequest) + require.True(t, et.Commit) + require.Len(t, et.InFlightWrites, 0) + + br = ba.CreateReply() br.Txn = ba.Txn - // The Sender did not receive an EndTransaction request, so it keeps - // the Txn status as PENDING. - br.Txn.Status = roachpb.PENDING + br.Txn.Status = roachpb.COMMITTED return br, nil }) + return br, nil + }) - br, pErr := tc.SendLocked(ctx, ba) - require.NotNil(t, br) - require.Nil(t, pErr) - require.NotNil(t, br.Txn) - require.Equal(t, expStatus, br.Txn.Status) + br, pErr := tc.SendLocked(ctx, ba) + require.Nil(t, pErr) + require.NotNil(t, br) - // Test the case where the EndTransaction request is alone. - ba.Requests = nil - ba.Add(&roachpb.EndTransactionRequest{Commit: commit, IntentSpans: nil}) + // Wait until the explicit commit succeeds. + <-explicitCommitCh +} - mockSender.MockSend(func(ba roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { - require.Fail(t, "should not have issued batch request", ba) - return nil, nil - }) +// TestTxnCommitterRetryAfterStaging verifies that txnCommitter returns a retry +// error when a write performed in parallel with staging a transaction is pushed +// to a timestamp above the staging timestamp. +func TestTxnCommitterRetryAfterStaging(t *testing.T) { + defer leaktest.AfterTest(t)() + ctx := context.Background() + tc, mockSender := makeMockTxnCommitter() + defer tc.stopper.Stop(ctx) - br, pErr = tc.SendLocked(ctx, ba) - require.NotNil(t, br) - require.Nil(t, pErr) - require.NotNil(t, br.Txn) - require.Equal(t, expStatus, br.Txn.Status) + txn := makeTxnProto() + keyA := roachpb.Key("a") + + var ba roachpb.BatchRequest + ba.Header = roachpb.Header{Txn: &txn} + putArgs := roachpb.PutRequest{RequestHeader: roachpb.RequestHeader{Key: keyA}} + etArgs := roachpb.EndTransactionRequest{Commit: true} + putArgs.Sequence = 1 + etArgs.Sequence = 2 + etArgs.InFlightWrites = []roachpb.SequencedWrite{{Key: keyA, Sequence: 1}} + ba.Add(&putArgs, &etArgs) + + mockSender.MockSend(func(ba roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { + require.Len(t, ba.Requests, 2) + require.IsType(t, &roachpb.PutRequest{}, ba.Requests[0].GetInner()) + require.IsType(t, &roachpb.EndTransactionRequest{}, ba.Requests[1].GetInner()) + + et := ba.Requests[1].GetInner().(*roachpb.EndTransactionRequest) + require.True(t, et.Commit) + require.Len(t, et.InFlightWrites, 1) + require.Equal(t, roachpb.SequencedWrite{Key: keyA, Sequence: 1}, et.InFlightWrites[0]) + + br := ba.CreateReply() + br.Txn = ba.Txn + br.Txn.Status = roachpb.STAGING + br.Responses[1].GetInner().(*roachpb.EndTransactionResponse).StagingTimestamp = br.Txn.Timestamp + + // Pretend the PutRequest was split and sent to a different Range. It + // could hit a WriteTooOld error (which marks the WriteTooOld flag) and + // have its timestamp pushed if it attempts to write under a committed + // value. The intent will be written but the response transaction's + // timestamp will be larger than the staging timestamp. + br.Txn.WriteTooOld = true + br.Txn.Timestamp = br.Txn.Timestamp.Add(1, 0) + return br, nil }) + + br, pErr := tc.SendLocked(ctx, ba) + require.Nil(t, br) + require.NotNil(t, pErr) + require.IsType(t, &roachpb.TransactionRetryError{}, pErr.GetDetail()) + require.Equal(t, roachpb.RETRY_SERIALIZABLE, pErr.GetDetail().(*roachpb.TransactionRetryError).Reason) } diff --git a/pkg/kv/txn_interceptor_heartbeater.go b/pkg/kv/txn_interceptor_heartbeater.go index ff469de1ad7a..21fb49b55df3 100644 --- a/pkg/kv/txn_interceptor_heartbeater.go +++ b/pkg/kv/txn_interceptor_heartbeater.go @@ -441,13 +441,21 @@ func (h *txnHeartbeater) heartbeat(ctx context.Context) bool { // This appears to be benign, but it's still somewhat disconcerting. If this // ever causes any issues, we'll need to be smarter about detecting this race // on the client and conditionally ignoring the result of heartbeat responses. - h.mu.txn.Update(respTxn) - if h.mu.txn.Status != roachpb.PENDING { - if h.mu.txn.Status == roachpb.ABORTED { - log.VEventf(ctx, 1, "Heartbeat detected aborted txn. Cleaning up.") - h.abortTxnAsyncLocked(ctx) + if respTxn != nil { + if respTxn.Status == roachpb.STAGING { + // Consider STAGING transactions to be PENDING for the purpose of + // the heartbeat loop. Interceptors above the txnCommitter should + // be oblivious to parallel commits. + respTxn.Status = roachpb.PENDING + } + h.mu.txn.Update(respTxn) + if h.mu.txn.Status != roachpb.PENDING { + if h.mu.txn.Status == roachpb.ABORTED { + log.VEventf(ctx, 1, "Heartbeat detected aborted txn. Cleaning up.") + h.abortTxnAsyncLocked(ctx) + } + return false } - return false } return true } diff --git a/pkg/kv/txn_interceptor_pipeliner.go b/pkg/kv/txn_interceptor_pipeliner.go index beb65779d475..dedb5ab61ee2 100644 --- a/pkg/kv/txn_interceptor_pipeliner.go +++ b/pkg/kv/txn_interceptor_pipeliner.go @@ -78,10 +78,10 @@ var trackedWritesMaxSize = settings.RegisterIntSetting( // requests chain on to them by first proving that the async writes succeeded. // The interceptor also ensures that when committing a transaction all writes // that have been proposed but not proven to have succeeded are first checked -// before committing. These async writes are referred to as "in-flight writes" -// and this process of proving that an in-flight write succeeded is called -// "proving" the write. Once writes are proven to have finished, they are -// considered "stable". +// before considering the transaction committed. These async writes are referred +// to as "in-flight writes" and this process of proving that an in-flight write +// succeeded is called "proving" the write. Once writes are proven to have +// finished, they are considered "stable". // // Chaining on to in-flight async writes is important for two main reasons to // txnPipeliner: @@ -111,15 +111,29 @@ var trackedWritesMaxSize = settings.RegisterIntSetting( // txnPipeliner uses chaining to throw an error when these re-orderings would // have affected the order that transactional requests evaluate in. // -// The interceptor proves all in-flight writes before committing a transaction -// by tacking on a QueryIntent request for each one to the front of an -// EndTransaction(Commit=true) requests. The result of this is that the -// EndTransaction needs to wait at the DistSender level for all of QueryIntent -// requests to succeed at before executing itself [1]. This is a little -// unfortunate because a transaction could have accumulated a large number of -// in-flight writes without proving any of them, and the more of these writes -// there are, the more chance querying one of them gets delayed and delays the -// overall transaction. +// The interceptor proves all in-flight writes before explicitly committing a +// transaction by tacking on a QueryIntent request for each one to the front of +// an EndTransaction(Commit=true) request. The in-flight writes that are being +// queried in the batch with the EndTransaction request are treated as in-flight +// writes for the purposes of parallel commits. The effect of this is that the +// in-flight writes must all be proven for a transaction to be considered +// implicitly committed. It also follows that they will need to be queried +// during transaction recovery. +// +// This is fantastic from the standpoint of transaction latency because it means +// that the consensus latency for every write in a transaction, including the +// write to the transaction record, is paid in parallel (mod pipeline stalls) +// and an entire transaction can commit in a single consensus round-trip! +// +// On the flip side, this means that every unproven write is considered +// in-flight at the time of the commit and needs to be proven at the time of the +// commit. This is a little unfortunate because a transaction could have +// accumulated a large number of in-flight writes over a long period of time +// without proving any of them, and the more of these writes there are, the +// greater the chance that querying one of them gets delayed and delays the +// overall transaction. Additionally, the more of these writes there are, the +// more expensive transaction recovery will be if the transaction ends up stuck +// in an indeterminate commit state. // // Three approaches have been considered to address this, all of which revolve // around the idea that earlier writes in a transaction may have finished @@ -153,13 +167,6 @@ var trackedWritesMaxSize = settings.RegisterIntSetting( // they finish consensus without any extra RPCs. // // So far, none of these approaches have been integrated. -// -// [1] A proposal called "parallel commits" (#24194) exists that would allow all -// QueryIntent requests and the EndTransaction request that they are prepended -// to to be sent by the DistSender in parallel. This would help with this -// issue by hiding the cost of the QueryIntent requests behind the cost of the -// "staging" EndTransaction request. -// type txnPipeliner struct { st *cluster.Settings // Optional; used to condense intent spans, if provided. If not provided, @@ -192,9 +199,6 @@ type txnPipeliner struct { func (tp *txnPipeliner) SendLocked( ctx context.Context, ba roachpb.BatchRequest, ) (*roachpb.BatchResponse, *roachpb.Error) { - // Adjust the batch so that it doesn't miss any in-flight writes. - ba = tp.chainToInFlightWrites(ba) - // If an EndTransaction request is part of this batch, attach the // in-flight writes and the write footprint to it. ba, pErr := tp.attachWritesToEndTxn(ctx, ba) @@ -202,18 +206,89 @@ func (tp *txnPipeliner) SendLocked( return nil, pErr } + // Adjust the batch so that it doesn't miss any in-flight writes. + ba = tp.chainToInFlightWrites(ba) + // Send through wrapped lockedSender. Unlocks while sending then re-locks. br, pErr := tp.wrapped.SendLocked(ctx, ba) // Update the in-flight write set and the write footprint with the results // of the request. - tp.updateWriteTracking(ctx, ba, br, pErr) + tp.updateWriteTracking(ctx, ba, br) if pErr != nil { return nil, tp.adjustError(ctx, ba, pErr) } return tp.stripQueryIntents(br), nil } +// attachWritesToEndTxn attaches the in-flight writes and the write footprint +// that the interceptor has been tracking to any EndTransaction requests present +// in the provided batch. It augments these sets with writes from the current +// batch. +func (tp *txnPipeliner) attachWritesToEndTxn( + ctx context.Context, ba roachpb.BatchRequest, +) (roachpb.BatchRequest, *roachpb.Error) { + args, hasET := ba.GetArg(roachpb.EndTransaction) + if !hasET { + return ba, nil + } + et := args.(*roachpb.EndTransactionRequest) + if len(et.IntentSpans) > 0 { + return ba, roachpb.NewErrorf("client must not pass intents to EndTransaction") + } + if len(et.InFlightWrites) > 0 { + return ba, roachpb.NewErrorf("client must not pass in-flight writes to EndTransaction") + } + + // Populate et.IntentSpans and et.InFlightWrites. + if !tp.footprint.empty() { + et.IntentSpans = append([]roachpb.Span(nil), tp.footprint.asSlice()...) + } + if inFlight := tp.ifWrites.len(); inFlight != 0 { + et.InFlightWrites = make([]roachpb.SequencedWrite, 0, inFlight) + tp.ifWrites.ascend(func(w *inFlightWrite) { + et.InFlightWrites = append(et.InFlightWrites, w.SequencedWrite) + }) + } + + // Augment et.IntentSpans and et.InFlightWrites with writes from + // the current batch. + for _, ru := range ba.Requests[:len(ba.Requests)-1] { + req := ru.GetInner() + h := req.Header() + if roachpb.IsTransactionWrite(req) { + // Ranged writes are added immediately to the intent spans because + // it's not clear where they will actually leave intents. Point + // writes are added to the in-flight writes set. + // + // If we see any ranged writes then we know that the txnCommitter + // will fold the in-flight writes into the intent spans immediately + // and forgo a parallel commit, but let's not break that abstraction + // boundary here. + if roachpb.IsRange(req) { + et.IntentSpans = append(et.IntentSpans, h.Span()) + } else { + w := roachpb.SequencedWrite{Key: h.Key, Sequence: h.Sequence} + et.InFlightWrites = append(et.InFlightWrites, w) + } + } + } + + // Sort both sets and condense the intent spans. + et.IntentSpans, _ = roachpb.MergeSpans(et.IntentSpans) + sort.Sort(roachpb.SequencedWriteBySeq(et.InFlightWrites)) + + if log.V(3) { + for _, intent := range et.IntentSpans { + log.Infof(ctx, "intent: [%s,%s)", intent.Key, intent.EndKey) + } + for _, write := range et.InFlightWrites { + log.Infof(ctx, "in-flight: %d:%s", write.Sequence, write.Key) + } + } + return ba, nil +} + // chainToInFlightWrites ensures that we "chain" on to any in-flight writes that // overlap the keys we're trying to read/write. We do this by prepending // QueryIntent requests with the ErrorIfMissing option before each request that @@ -361,64 +436,10 @@ func (tp *txnPipeliner) chainToInFlightWrites(ba roachpb.BatchRequest) roachpb.B return ba } -// attachWritesToEndTxn attaches the in-flight writes and the write footprint -// that the interceptor has been tracking to any EndTransaction requests present -// in the provided batch. -func (tp *txnPipeliner) attachWritesToEndTxn( - ctx context.Context, ba roachpb.BatchRequest, -) (roachpb.BatchRequest, *roachpb.Error) { - args, hasET := ba.GetArg(roachpb.EndTransaction) - if !hasET { - return ba, nil - } - et := args.(*roachpb.EndTransactionRequest) - if len(et.IntentSpans) > 0 { - return ba, roachpb.NewErrorf("client must not pass intents to EndTransaction") - } - - // Defensively set distinctSpans to false if we had any previous writes in - // this transaction. This effectively limits the distinct spans optimization - // to 1pc transactions. - ba.Header.DistinctSpans = !tp.haveWrites() - - // Insert into the in-flight write set and the write footprint assuming that - // the request will succeed. We need to eagerly populate these sets before - // we send the batch because we can't wait until the batch returns to attach - // the new writes to the EndTransaction request. - // - // We can't pass in a batch response here to better limit the key spans as - // we don't know what is going to be affected. This will affect queries such - // as `DELETE FROM my.table LIMIT 10` when executed as a 1PC transaction. - // e.g.: a (BeginTransaction, DeleteRange, EndTransaction) batch. - tp.updateWriteTracking(ctx, ba, nil, nil) - - // For now we include all in-flight writes in the IntentSpans slice on - // EndTransaction requests. In the future, EndTransaction request will have - // a separate InFlightWrites field and in-flight writes will be merged into - // the IntentSpans slice as they succeed. - tp.ifWrites.ascend(func(w *inFlightWrite) { - tp.footprint.insert(roachpb.Span{Key: w.Key}) - }) - - // Populate et.IntentSpans, taking into account both any existing and new - // writes, and taking care to perform proper deduplication. - // TODO(peter): Populate DistinctSpans on all batches, not just batches - // which contain an EndTransactionRequest. - ba.Header.DistinctSpans = tp.footprint.mergeAndSort() && ba.Header.DistinctSpans - et.IntentSpans = append([]roachpb.Span(nil), tp.footprint.asSlice()...) - - if log.V(3) { - for _, intent := range et.IntentSpans { - log.Infof(ctx, "intent: [%s,%s)", intent.Key, intent.EndKey) - } - } - return ba, nil -} - -// updateWriteTracking reads the response for the given request and uses it -// to update the tracked in-flight write set and write footprint. It does so -// by performing three actions: -// 1. it adds all async writes that the request performed to the in-flight +// updateWriteTracking reads the response for the given request and uses it to +// update the tracked in-flight write set and write footprint. It does so by +// performing three actions: 1. it adds all async writes that the request +// performed to the in-flight // write set. // 2. it adds all non-async writes that the request performed to the write // footprint. @@ -428,15 +449,11 @@ func (tp *txnPipeliner) attachWritesToEndTxn( // After updating the write sets, the write footprint is condensed to ensure // that it remains under its memory limit. // -// If an error is provided, all writes from the batch are added directly -// to the write footprint to avoid leaking any intents when the transaction -// cleans up. -// -// If no response or error are provided, the method adds to the write sets -// assuming that the request will succeed but does not remove anything from -// the in-flight writes set. +// If no response is provided (indicating an error), all writes from the batch +// are added directly to the write footprint to avoid leaking any intents when +// the transaction cleans up. func (tp *txnPipeliner) updateWriteTracking( - ctx context.Context, ba roachpb.BatchRequest, br *roachpb.BatchResponse, pErr *roachpb.Error, + ctx context.Context, ba roachpb.BatchRequest, br *roachpb.BatchResponse, ) { // After adding new writes to the write footprint, check whether we need to // condense the set to stay below memory limits. @@ -445,7 +462,7 @@ func (tp *txnPipeliner) updateWriteTracking( // If the request failed, add all intent writes directly to the write // footprint. This reduces the likelihood of dangling intents blocking // concurrent writers for extended periods of time. See #3346. - if pErr != nil { + if br == nil { // The transaction cannot continue in this epoch whether this is // a retryable error or not. ba.IntentSpanIterate(nil, tp.footprint.insert) @@ -454,19 +471,17 @@ func (tp *txnPipeliner) updateWriteTracking( // If the transaction is no longer pending, clear the in-flight writes set // and immediately return. - if br != nil && br.Txn != nil && br.Txn.Status != roachpb.PENDING { + // TODO(nvanbenschoten): Do we have to handle missing Txn's anymore? + if br.Txn != nil && br.Txn.Status != roachpb.PENDING { tp.ifWrites.clear(false /* reuse */) return } for i, ru := range ba.Requests { req := ru.GetInner() - var resp roachpb.Response - if br != nil { - resp = br.Responses[i].GetInner() - } + resp := br.Responses[i].GetInner() - if qiReq, ok := req.(*roachpb.QueryIntentRequest); ok && resp != nil { + if qiReq, ok := req.(*roachpb.QueryIntentRequest); ok { // Remove any in-flight writes that were proven to exist. // It shouldn't be possible for a QueryIntentRequest with // the ErrorIfMissing option set to return without error @@ -903,7 +918,8 @@ func (s *condensableSpanSet) maybeCondense(ctx context.Context, ri *RangeIterato // asSlice returns the set as a slice of spans. func (s *condensableSpanSet) asSlice() []roachpb.Span { - return s.s + l := len(s.s) + return s.s[:l:l] // immutable on append } // empty returns whether the set is empty or whether it contains spans. diff --git a/pkg/kv/txn_interceptor_pipeliner_test.go b/pkg/kv/txn_interceptor_pipeliner_test.go index 9ad2617de31d..59a99d698a30 100644 --- a/pkg/kv/txn_interceptor_pipeliner_test.go +++ b/pkg/kv/txn_interceptor_pipeliner_test.go @@ -72,7 +72,9 @@ func TestTxnPipeliner1PCTransaction(t *testing.T) { var ba roachpb.BatchRequest ba.Header = roachpb.Header{Txn: &txn} - ba.Add(&roachpb.PutRequest{RequestHeader: roachpb.RequestHeader{Key: keyA}}) + putArgs := roachpb.PutRequest{RequestHeader: roachpb.RequestHeader{Key: keyA}} + putArgs.Sequence = 1 + ba.Add(&putArgs) ba.Add(&roachpb.EndTransactionRequest{Commit: true}) mockSender.MockSend(func(ba roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { @@ -82,7 +84,8 @@ func TestTxnPipeliner1PCTransaction(t *testing.T) { require.IsType(t, &roachpb.EndTransactionRequest{}, ba.Requests[1].GetInner()) etReq := ba.Requests[1].GetInner().(*roachpb.EndTransactionRequest) - require.Equal(t, []roachpb.Span{{Key: keyA}}, etReq.IntentSpans) + require.Len(t, etReq.IntentSpans, 0) + require.Equal(t, []roachpb.SequencedWrite{{Key: keyA, Sequence: 1}}, etReq.InFlightWrites) br := ba.CreateReply() br.Txn = ba.Txn @@ -224,8 +227,14 @@ func TestTxnPipelinerTrackInFlightWrites(t *testing.T) { require.Equal(t, enginepb.TxnSeq(5), qiReq3.Txn.Sequence) etReq := ba.Requests[4].GetInner().(*roachpb.EndTransactionRequest) - exp := []roachpb.Span{{Key: keyA}, {Key: keyB}, {Key: keyC}, {Key: keyD}} - require.Equal(t, exp, etReq.IntentSpans) + require.Equal(t, []roachpb.Span{{Key: keyA}}, etReq.IntentSpans) + expInFlight := []roachpb.SequencedWrite{ + {Key: keyA, Sequence: 2}, + {Key: keyB, Sequence: 3}, + {Key: keyC, Sequence: 5}, + {Key: keyD, Sequence: 6}, + } + require.Equal(t, expInFlight, etReq.InFlightWrites) br = ba.CreateReply() br.Txn = ba.Txn @@ -638,7 +647,8 @@ func TestTxnPipelinerTransactionAbort(t *testing.T) { require.IsType(t, &roachpb.EndTransactionRequest{}, ba.Requests[0].GetInner()) etReq := ba.Requests[0].GetInner().(*roachpb.EndTransactionRequest) - require.Equal(t, []roachpb.Span{{Key: keyA}}, etReq.IntentSpans) + require.Len(t, etReq.IntentSpans, 0) + require.Equal(t, []roachpb.SequencedWrite{{Key: keyA, Sequence: 1}}, etReq.InFlightWrites) br = ba.CreateReply() br.Txn = ba.Txn @@ -665,7 +675,8 @@ func TestTxnPipelinerTransactionAbort(t *testing.T) { require.IsType(t, &roachpb.EndTransactionRequest{}, ba.Requests[0].GetInner()) etReq := ba.Requests[0].GetInner().(*roachpb.EndTransactionRequest) - require.Equal(t, []roachpb.Span{{Key: keyA}}, etReq.IntentSpans) + require.Len(t, etReq.IntentSpans, 0) + require.Equal(t, []roachpb.SequencedWrite{{Key: keyA, Sequence: 1}}, etReq.InFlightWrites) br = ba.CreateReply() br.Txn = ba.Txn @@ -865,7 +876,8 @@ func TestTxnPipelinerEnableDisableMixTxn(t *testing.T) { require.Equal(t, enginepb.TxnSeq(3), qiReq.Txn.Sequence) etReq := ba.Requests[1].GetInner().(*roachpb.EndTransactionRequest) - require.Equal(t, []roachpb.Span{{Key: keyA}, {Key: keyC}}, etReq.IntentSpans) + require.Equal(t, []roachpb.Span{{Key: keyA}}, etReq.IntentSpans) + require.Equal(t, []roachpb.SequencedWrite{{Key: keyC, Sequence: 3}}, etReq.InFlightWrites) br = ba.CreateReply() br.Txn = ba.Txn diff --git a/pkg/roachpb/api.pb.go b/pkg/roachpb/api.pb.go index 1ef3cb84e56c..d167ded615ec 100644 --- a/pkg/roachpb/api.pb.go +++ b/pkg/roachpb/api.pb.go @@ -70,7 +70,7 @@ func (x ReadConsistencyType) String() string { return proto.EnumName(ReadConsistencyType_name, int32(x)) } func (ReadConsistencyType) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_c24af62f3f876bfa, []int{0} + return fileDescriptor_api_08e22abf3c79051e, []int{0} } // ScanFormat is an enumeration of the available response formats for MVCCScan @@ -98,7 +98,7 @@ func (x ScanFormat) String() string { return proto.EnumName(ScanFormat_name, int32(x)) } func (ScanFormat) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_c24af62f3f876bfa, []int{1} + return fileDescriptor_api_08e22abf3c79051e, []int{1} } type ChecksumMode int32 @@ -145,7 +145,7 @@ func (x ChecksumMode) String() string { return proto.EnumName(ChecksumMode_name, int32(x)) } func (ChecksumMode) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_c24af62f3f876bfa, []int{2} + return fileDescriptor_api_08e22abf3c79051e, []int{2} } // PushTxnType determines what action to take when pushing a transaction. @@ -176,7 +176,7 @@ func (x PushTxnType) String() string { return proto.EnumName(PushTxnType_name, int32(x)) } func (PushTxnType) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_c24af62f3f876bfa, []int{3} + return fileDescriptor_api_08e22abf3c79051e, []int{3} } type ExportStorageProvider int32 @@ -214,7 +214,7 @@ func (x ExportStorageProvider) String() string { return proto.EnumName(ExportStorageProvider_name, int32(x)) } func (ExportStorageProvider) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_c24af62f3f876bfa, []int{4} + return fileDescriptor_api_08e22abf3c79051e, []int{4} } type MVCCFilter int32 @@ -237,7 +237,7 @@ func (x MVCCFilter) String() string { return proto.EnumName(MVCCFilter_name, int32(x)) } func (MVCCFilter) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_c24af62f3f876bfa, []int{5} + return fileDescriptor_api_08e22abf3c79051e, []int{5} } type ResponseHeader_ResumeReason int32 @@ -269,7 +269,7 @@ func (x ResponseHeader_ResumeReason) String() string { return proto.EnumName(ResponseHeader_ResumeReason_name, int32(x)) } func (ResponseHeader_ResumeReason) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_c24af62f3f876bfa, []int{2, 0} + return fileDescriptor_api_08e22abf3c79051e, []int{2, 0} } type CheckConsistencyResponse_Status int32 @@ -311,7 +311,7 @@ func (x CheckConsistencyResponse_Status) String() string { return proto.EnumName(CheckConsistencyResponse_Status_name, int32(x)) } func (CheckConsistencyResponse_Status) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_c24af62f3f876bfa, []int{25, 0} + return fileDescriptor_api_08e22abf3c79051e, []int{25, 0} } // RangeInfo describes a range which executed a request. It contains @@ -327,7 +327,7 @@ func (m *RangeInfo) Reset() { *m = RangeInfo{} } func (m *RangeInfo) String() string { return proto.CompactTextString(m) } func (*RangeInfo) ProtoMessage() {} func (*RangeInfo) Descriptor() ([]byte, []int) { - return fileDescriptor_api_c24af62f3f876bfa, []int{0} + return fileDescriptor_api_08e22abf3c79051e, []int{0} } func (m *RangeInfo) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -372,7 +372,7 @@ func (m *RequestHeader) Reset() { *m = RequestHeader{} } func (m *RequestHeader) String() string { return proto.CompactTextString(m) } func (*RequestHeader) ProtoMessage() {} func (*RequestHeader) Descriptor() ([]byte, []int) { - return fileDescriptor_api_c24af62f3f876bfa, []int{1} + return fileDescriptor_api_08e22abf3c79051e, []int{1} } func (m *RequestHeader) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -432,7 +432,7 @@ func (m *ResponseHeader) Reset() { *m = ResponseHeader{} } func (m *ResponseHeader) String() string { return proto.CompactTextString(m) } func (*ResponseHeader) ProtoMessage() {} func (*ResponseHeader) Descriptor() ([]byte, []int) { - return fileDescriptor_api_c24af62f3f876bfa, []int{2} + return fileDescriptor_api_08e22abf3c79051e, []int{2} } func (m *ResponseHeader) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -468,7 +468,7 @@ func (m *GetRequest) Reset() { *m = GetRequest{} } func (m *GetRequest) String() string { return proto.CompactTextString(m) } func (*GetRequest) ProtoMessage() {} func (*GetRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_c24af62f3f876bfa, []int{3} + return fileDescriptor_api_08e22abf3c79051e, []int{3} } func (m *GetRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -508,7 +508,7 @@ func (m *GetResponse) Reset() { *m = GetResponse{} } func (m *GetResponse) String() string { return proto.CompactTextString(m) } func (*GetResponse) ProtoMessage() {} func (*GetResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_c24af62f3f876bfa, []int{4} + return fileDescriptor_api_08e22abf3c79051e, []int{4} } func (m *GetResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -553,7 +553,7 @@ func (m *PutRequest) Reset() { *m = PutRequest{} } func (m *PutRequest) String() string { return proto.CompactTextString(m) } func (*PutRequest) ProtoMessage() {} func (*PutRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_c24af62f3f876bfa, []int{5} + return fileDescriptor_api_08e22abf3c79051e, []int{5} } func (m *PutRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -589,7 +589,7 @@ func (m *PutResponse) Reset() { *m = PutResponse{} } func (m *PutResponse) String() string { return proto.CompactTextString(m) } func (*PutResponse) ProtoMessage() {} func (*PutResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_c24af62f3f876bfa, []int{6} + return fileDescriptor_api_08e22abf3c79051e, []int{6} } func (m *PutResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -645,7 +645,7 @@ func (m *ConditionalPutRequest) Reset() { *m = ConditionalPutRequest{} } func (m *ConditionalPutRequest) String() string { return proto.CompactTextString(m) } func (*ConditionalPutRequest) ProtoMessage() {} func (*ConditionalPutRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_c24af62f3f876bfa, []int{7} + return fileDescriptor_api_08e22abf3c79051e, []int{7} } func (m *ConditionalPutRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -682,7 +682,7 @@ func (m *ConditionalPutResponse) Reset() { *m = ConditionalPutResponse{} func (m *ConditionalPutResponse) String() string { return proto.CompactTextString(m) } func (*ConditionalPutResponse) ProtoMessage() {} func (*ConditionalPutResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_c24af62f3f876bfa, []int{8} + return fileDescriptor_api_08e22abf3c79051e, []int{8} } func (m *ConditionalPutResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -730,7 +730,7 @@ func (m *InitPutRequest) Reset() { *m = InitPutRequest{} } func (m *InitPutRequest) String() string { return proto.CompactTextString(m) } func (*InitPutRequest) ProtoMessage() {} func (*InitPutRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_c24af62f3f876bfa, []int{9} + return fileDescriptor_api_08e22abf3c79051e, []int{9} } func (m *InitPutRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -766,7 +766,7 @@ func (m *InitPutResponse) Reset() { *m = InitPutResponse{} } func (m *InitPutResponse) String() string { return proto.CompactTextString(m) } func (*InitPutResponse) ProtoMessage() {} func (*InitPutResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_c24af62f3f876bfa, []int{10} + return fileDescriptor_api_08e22abf3c79051e, []int{10} } func (m *InitPutResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -808,7 +808,7 @@ func (m *IncrementRequest) Reset() { *m = IncrementRequest{} } func (m *IncrementRequest) String() string { return proto.CompactTextString(m) } func (*IncrementRequest) ProtoMessage() {} func (*IncrementRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_c24af62f3f876bfa, []int{11} + return fileDescriptor_api_08e22abf3c79051e, []int{11} } func (m *IncrementRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -847,7 +847,7 @@ func (m *IncrementResponse) Reset() { *m = IncrementResponse{} } func (m *IncrementResponse) String() string { return proto.CompactTextString(m) } func (*IncrementResponse) ProtoMessage() {} func (*IncrementResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_c24af62f3f876bfa, []int{12} + return fileDescriptor_api_08e22abf3c79051e, []int{12} } func (m *IncrementResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -883,7 +883,7 @@ func (m *DeleteRequest) Reset() { *m = DeleteRequest{} } func (m *DeleteRequest) String() string { return proto.CompactTextString(m) } func (*DeleteRequest) ProtoMessage() {} func (*DeleteRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_c24af62f3f876bfa, []int{13} + return fileDescriptor_api_08e22abf3c79051e, []int{13} } func (m *DeleteRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -919,7 +919,7 @@ func (m *DeleteResponse) Reset() { *m = DeleteResponse{} } func (m *DeleteResponse) String() string { return proto.CompactTextString(m) } func (*DeleteResponse) ProtoMessage() {} func (*DeleteResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_c24af62f3f876bfa, []int{14} + return fileDescriptor_api_08e22abf3c79051e, []int{14} } func (m *DeleteResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -970,7 +970,7 @@ func (m *DeleteRangeRequest) Reset() { *m = DeleteRangeRequest{} } func (m *DeleteRangeRequest) String() string { return proto.CompactTextString(m) } func (*DeleteRangeRequest) ProtoMessage() {} func (*DeleteRangeRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_c24af62f3f876bfa, []int{15} + return fileDescriptor_api_08e22abf3c79051e, []int{15} } func (m *DeleteRangeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1009,7 +1009,7 @@ func (m *DeleteRangeResponse) Reset() { *m = DeleteRangeResponse{} } func (m *DeleteRangeResponse) String() string { return proto.CompactTextString(m) } func (*DeleteRangeResponse) ProtoMessage() {} func (*DeleteRangeResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_c24af62f3f876bfa, []int{16} + return fileDescriptor_api_08e22abf3c79051e, []int{16} } func (m *DeleteRangeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1058,7 +1058,7 @@ func (m *ClearRangeRequest) Reset() { *m = ClearRangeRequest{} } func (m *ClearRangeRequest) String() string { return proto.CompactTextString(m) } func (*ClearRangeRequest) ProtoMessage() {} func (*ClearRangeRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_c24af62f3f876bfa, []int{17} + return fileDescriptor_api_08e22abf3c79051e, []int{17} } func (m *ClearRangeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1094,7 +1094,7 @@ func (m *ClearRangeResponse) Reset() { *m = ClearRangeResponse{} } func (m *ClearRangeResponse) String() string { return proto.CompactTextString(m) } func (*ClearRangeResponse) ProtoMessage() {} func (*ClearRangeResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_c24af62f3f876bfa, []int{18} + return fileDescriptor_api_08e22abf3c79051e, []int{18} } func (m *ClearRangeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1165,7 +1165,7 @@ func (m *ScanOptions) Reset() { *m = ScanOptions{} } func (m *ScanOptions) String() string { return proto.CompactTextString(m) } func (*ScanOptions) ProtoMessage() {} func (*ScanOptions) Descriptor() ([]byte, []int) { - return fileDescriptor_api_c24af62f3f876bfa, []int{19} + return fileDescriptor_api_08e22abf3c79051e, []int{19} } func (m *ScanOptions) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1207,7 +1207,7 @@ func (m *ScanRequest) Reset() { *m = ScanRequest{} } func (m *ScanRequest) String() string { return proto.CompactTextString(m) } func (*ScanRequest) ProtoMessage() {} func (*ScanRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_c24af62f3f876bfa, []int{20} + return fileDescriptor_api_08e22abf3c79051e, []int{20} } func (m *ScanRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1256,7 +1256,7 @@ func (m *ScanResponse) Reset() { *m = ScanResponse{} } func (m *ScanResponse) String() string { return proto.CompactTextString(m) } func (*ScanResponse) ProtoMessage() {} func (*ScanResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_c24af62f3f876bfa, []int{21} + return fileDescriptor_api_08e22abf3c79051e, []int{21} } func (m *ScanResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1298,7 +1298,7 @@ func (m *ReverseScanRequest) Reset() { *m = ReverseScanRequest{} } func (m *ReverseScanRequest) String() string { return proto.CompactTextString(m) } func (*ReverseScanRequest) ProtoMessage() {} func (*ReverseScanRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_c24af62f3f876bfa, []int{22} + return fileDescriptor_api_08e22abf3c79051e, []int{22} } func (m *ReverseScanRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1347,7 +1347,7 @@ func (m *ReverseScanResponse) Reset() { *m = ReverseScanResponse{} } func (m *ReverseScanResponse) String() string { return proto.CompactTextString(m) } func (*ReverseScanResponse) ProtoMessage() {} func (*ReverseScanResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_c24af62f3f876bfa, []int{23} + return fileDescriptor_api_08e22abf3c79051e, []int{23} } func (m *ReverseScanResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1395,7 +1395,7 @@ func (m *CheckConsistencyRequest) Reset() { *m = CheckConsistencyRequest func (m *CheckConsistencyRequest) String() string { return proto.CompactTextString(m) } func (*CheckConsistencyRequest) ProtoMessage() {} func (*CheckConsistencyRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_c24af62f3f876bfa, []int{24} + return fileDescriptor_api_08e22abf3c79051e, []int{24} } func (m *CheckConsistencyRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1434,7 +1434,7 @@ func (m *CheckConsistencyResponse) Reset() { *m = CheckConsistencyRespon func (m *CheckConsistencyResponse) String() string { return proto.CompactTextString(m) } func (*CheckConsistencyResponse) ProtoMessage() {} func (*CheckConsistencyResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_c24af62f3f876bfa, []int{25} + return fileDescriptor_api_08e22abf3c79051e, []int{25} } func (m *CheckConsistencyResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1480,7 +1480,7 @@ func (m *CheckConsistencyResponse_Result) Reset() { *m = CheckConsistenc func (m *CheckConsistencyResponse_Result) String() string { return proto.CompactTextString(m) } func (*CheckConsistencyResponse_Result) ProtoMessage() {} func (*CheckConsistencyResponse_Result) Descriptor() ([]byte, []int) { - return fileDescriptor_api_c24af62f3f876bfa, []int{25, 0} + return fileDescriptor_api_08e22abf3c79051e, []int{25, 0} } func (m *CheckConsistencyResponse_Result) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1530,7 +1530,7 @@ func (m *RecomputeStatsRequest) Reset() { *m = RecomputeStatsRequest{} } func (m *RecomputeStatsRequest) String() string { return proto.CompactTextString(m) } func (*RecomputeStatsRequest) ProtoMessage() {} func (*RecomputeStatsRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_c24af62f3f876bfa, []int{26} + return fileDescriptor_api_08e22abf3c79051e, []int{26} } func (m *RecomputeStatsRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1568,7 +1568,7 @@ func (m *RecomputeStatsResponse) Reset() { *m = RecomputeStatsResponse{} func (m *RecomputeStatsResponse) String() string { return proto.CompactTextString(m) } func (*RecomputeStatsResponse) ProtoMessage() {} func (*RecomputeStatsResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_c24af62f3f876bfa, []int{27} + return fileDescriptor_api_08e22abf3c79051e, []int{27} } func (m *RecomputeStatsResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1604,7 +1604,7 @@ func (m *BeginTransactionRequest) Reset() { *m = BeginTransactionRequest func (m *BeginTransactionRequest) String() string { return proto.CompactTextString(m) } func (*BeginTransactionRequest) ProtoMessage() {} func (*BeginTransactionRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_c24af62f3f876bfa, []int{28} + return fileDescriptor_api_08e22abf3c79051e, []int{28} } func (m *BeginTransactionRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1640,7 +1640,7 @@ func (m *BeginTransactionResponse) Reset() { *m = BeginTransactionRespon func (m *BeginTransactionResponse) String() string { return proto.CompactTextString(m) } func (*BeginTransactionResponse) ProtoMessage() {} func (*BeginTransactionResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_c24af62f3f876bfa, []int{29} + return fileDescriptor_api_08e22abf3c79051e, []int{29} } func (m *BeginTransactionResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1686,7 +1686,28 @@ type EndTransactionRequest struct { // are spans which must be resolved on txn completion. Note that these // spans may be condensed to cover aggregate spans if the keys written // by the transaction exceeded a size threshold. + // + // The set logically extends to include the keys of all writes in the + // in-flight write set. However, those keys are not stored in this set + // to avoid duplication. This means that elements that are removed from + // that set should be merged into this one. + // + // The slice is maintained in sorted order and all spans are maximally + // merged such that no two spans here overlap each other. IntentSpans []Span `protobuf:"bytes,5,rep,name=intent_spans,json=intentSpans,proto3" json:"intent_spans"` + // Set of in-flight intent writes that have been issued by the transaction but + // which may not have succeeded yet. If any promised writes are provided, a + // committing EndTransaction request will move a PENDING transaction to the + // STAGING status instead of the COMMITTED status. These in-flight writes must + // then all be confirmed as successful before the transaction can be moved + // from STAGING to COMMITTED. For more, see txnCommitter. + // + // The slice is maintained in sorted order by sequence number. This provides + // O(log n) access to individual writes in this set based on their sequence + // number. See SequencedWriteBySeq.Find and its uses. The set can contain + // multiple SequencedWrites with the same key, but all sequence numbers are + // unique. + InFlightWrites []SequencedWrite `protobuf:"bytes,17,rep,name=in_flight_writes,json=inFlightWrites,proto3" json:"in_flight_writes"` // Requires that the transaction completes as a 1 phase commit. This // guarantees that all writes are to the same range and that no // intents are left in the event of an error. @@ -1709,7 +1730,7 @@ func (m *EndTransactionRequest) Reset() { *m = EndTransactionRequest{} } func (m *EndTransactionRequest) String() string { return proto.CompactTextString(m) } func (*EndTransactionRequest) ProtoMessage() {} func (*EndTransactionRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_c24af62f3f876bfa, []int{30} + return fileDescriptor_api_08e22abf3c79051e, []int{30} } func (m *EndTransactionRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1746,16 +1767,19 @@ type EndTransactionResponse struct { // True if the transaction committed on the one phase commit path. // This means that all writes which were part of the transaction // were written as a single, atomic write batch to just one range. - OnePhaseCommit bool `protobuf:"varint,4,opt,name=one_phase_commit,json=onePhaseCommit,proto3" json:"one_phase_commit,omitempty"` - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_sizecache int32 `json:"-"` + OnePhaseCommit bool `protobuf:"varint,4,opt,name=one_phase_commit,json=onePhaseCommit,proto3" json:"one_phase_commit,omitempty"` + // The commit timestamp of the STAGING transaction record written + // by the request. Only set if the transaction record was staged. + StagingTimestamp hlc.Timestamp `protobuf:"bytes,5,opt,name=staging_timestamp,json=stagingTimestamp,proto3" json:"staging_timestamp"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } func (m *EndTransactionResponse) Reset() { *m = EndTransactionResponse{} } func (m *EndTransactionResponse) String() string { return proto.CompactTextString(m) } func (*EndTransactionResponse) ProtoMessage() {} func (*EndTransactionResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_c24af62f3f876bfa, []int{31} + return fileDescriptor_api_08e22abf3c79051e, []int{31} } func (m *EndTransactionResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1811,7 +1835,7 @@ func (m *AdminSplitRequest) Reset() { *m = AdminSplitRequest{} } func (m *AdminSplitRequest) String() string { return proto.CompactTextString(m) } func (*AdminSplitRequest) ProtoMessage() {} func (*AdminSplitRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_c24af62f3f876bfa, []int{32} + return fileDescriptor_api_08e22abf3c79051e, []int{32} } func (m *AdminSplitRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1848,7 +1872,7 @@ func (m *AdminSplitResponse) Reset() { *m = AdminSplitResponse{} } func (m *AdminSplitResponse) String() string { return proto.CompactTextString(m) } func (*AdminSplitResponse) ProtoMessage() {} func (*AdminSplitResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_c24af62f3f876bfa, []int{33} + return fileDescriptor_api_08e22abf3c79051e, []int{33} } func (m *AdminSplitResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1892,7 +1916,7 @@ func (m *AdminMergeRequest) Reset() { *m = AdminMergeRequest{} } func (m *AdminMergeRequest) String() string { return proto.CompactTextString(m) } func (*AdminMergeRequest) ProtoMessage() {} func (*AdminMergeRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_c24af62f3f876bfa, []int{34} + return fileDescriptor_api_08e22abf3c79051e, []int{34} } func (m *AdminMergeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1929,7 +1953,7 @@ func (m *AdminMergeResponse) Reset() { *m = AdminMergeResponse{} } func (m *AdminMergeResponse) String() string { return proto.CompactTextString(m) } func (*AdminMergeResponse) ProtoMessage() {} func (*AdminMergeResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_c24af62f3f876bfa, []int{35} + return fileDescriptor_api_08e22abf3c79051e, []int{35} } func (m *AdminMergeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1969,7 +1993,7 @@ func (m *AdminTransferLeaseRequest) Reset() { *m = AdminTransferLeaseReq func (m *AdminTransferLeaseRequest) String() string { return proto.CompactTextString(m) } func (*AdminTransferLeaseRequest) ProtoMessage() {} func (*AdminTransferLeaseRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_c24af62f3f876bfa, []int{36} + return fileDescriptor_api_08e22abf3c79051e, []int{36} } func (m *AdminTransferLeaseRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2004,7 +2028,7 @@ func (m *AdminTransferLeaseResponse) Reset() { *m = AdminTransferLeaseRe func (m *AdminTransferLeaseResponse) String() string { return proto.CompactTextString(m) } func (*AdminTransferLeaseResponse) ProtoMessage() {} func (*AdminTransferLeaseResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_c24af62f3f876bfa, []int{37} + return fileDescriptor_api_08e22abf3c79051e, []int{37} } func (m *AdminTransferLeaseResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2058,7 +2082,7 @@ func (m *AdminChangeReplicasRequest) Reset() { *m = AdminChangeReplicasR func (m *AdminChangeReplicasRequest) String() string { return proto.CompactTextString(m) } func (*AdminChangeReplicasRequest) ProtoMessage() {} func (*AdminChangeReplicasRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_c24af62f3f876bfa, []int{38} + return fileDescriptor_api_08e22abf3c79051e, []int{38} } func (m *AdminChangeReplicasRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2097,7 +2121,7 @@ func (m *AdminChangeReplicasResponse) Reset() { *m = AdminChangeReplicas func (m *AdminChangeReplicasResponse) String() string { return proto.CompactTextString(m) } func (*AdminChangeReplicasResponse) ProtoMessage() {} func (*AdminChangeReplicasResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_c24af62f3f876bfa, []int{39} + return fileDescriptor_api_08e22abf3c79051e, []int{39} } func (m *AdminChangeReplicasResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2136,7 +2160,7 @@ func (m *AdminRelocateRangeRequest) Reset() { *m = AdminRelocateRangeReq func (m *AdminRelocateRangeRequest) String() string { return proto.CompactTextString(m) } func (*AdminRelocateRangeRequest) ProtoMessage() {} func (*AdminRelocateRangeRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_c24af62f3f876bfa, []int{40} + return fileDescriptor_api_08e22abf3c79051e, []int{40} } func (m *AdminRelocateRangeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2171,7 +2195,7 @@ func (m *AdminRelocateRangeResponse) Reset() { *m = AdminRelocateRangeRe func (m *AdminRelocateRangeResponse) String() string { return proto.CompactTextString(m) } func (*AdminRelocateRangeResponse) ProtoMessage() {} func (*AdminRelocateRangeResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_c24af62f3f876bfa, []int{41} + return fileDescriptor_api_08e22abf3c79051e, []int{41} } func (m *AdminRelocateRangeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2212,7 +2236,7 @@ func (m *HeartbeatTxnRequest) Reset() { *m = HeartbeatTxnRequest{} } func (m *HeartbeatTxnRequest) String() string { return proto.CompactTextString(m) } func (*HeartbeatTxnRequest) ProtoMessage() {} func (*HeartbeatTxnRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_c24af62f3f876bfa, []int{42} + return fileDescriptor_api_08e22abf3c79051e, []int{42} } func (m *HeartbeatTxnRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2251,7 +2275,7 @@ func (m *HeartbeatTxnResponse) Reset() { *m = HeartbeatTxnResponse{} } func (m *HeartbeatTxnResponse) String() string { return proto.CompactTextString(m) } func (*HeartbeatTxnResponse) ProtoMessage() {} func (*HeartbeatTxnResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_c24af62f3f876bfa, []int{43} + return fileDescriptor_api_08e22abf3c79051e, []int{43} } func (m *HeartbeatTxnResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2296,7 +2320,7 @@ func (m *GCRequest) Reset() { *m = GCRequest{} } func (m *GCRequest) String() string { return proto.CompactTextString(m) } func (*GCRequest) ProtoMessage() {} func (*GCRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_c24af62f3f876bfa, []int{44} + return fileDescriptor_api_08e22abf3c79051e, []int{44} } func (m *GCRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2332,7 +2356,7 @@ func (m *GCRequest_GCKey) Reset() { *m = GCRequest_GCKey{} } func (m *GCRequest_GCKey) String() string { return proto.CompactTextString(m) } func (*GCRequest_GCKey) ProtoMessage() {} func (*GCRequest_GCKey) Descriptor() ([]byte, []int) { - return fileDescriptor_api_c24af62f3f876bfa, []int{44, 0} + return fileDescriptor_api_08e22abf3c79051e, []int{44, 0} } func (m *GCRequest_GCKey) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2368,7 +2392,7 @@ func (m *GCResponse) Reset() { *m = GCResponse{} } func (m *GCResponse) String() string { return proto.CompactTextString(m) } func (*GCResponse) ProtoMessage() {} func (*GCResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_c24af62f3f876bfa, []int{45} + return fileDescriptor_api_08e22abf3c79051e, []int{45} } func (m *GCResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2452,7 +2476,7 @@ func (m *PushTxnRequest) Reset() { *m = PushTxnRequest{} } func (m *PushTxnRequest) String() string { return proto.CompactTextString(m) } func (*PushTxnRequest) ProtoMessage() {} func (*PushTxnRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_c24af62f3f876bfa, []int{46} + return fileDescriptor_api_08e22abf3c79051e, []int{46} } func (m *PushTxnRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2497,7 +2521,7 @@ func (m *PushTxnResponse) Reset() { *m = PushTxnResponse{} } func (m *PushTxnResponse) String() string { return proto.CompactTextString(m) } func (*PushTxnResponse) ProtoMessage() {} func (*PushTxnResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_c24af62f3f876bfa, []int{47} + return fileDescriptor_api_08e22abf3c79051e, []int{47} } func (m *PushTxnResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2546,7 +2570,7 @@ func (m *RecoverTxnRequest) Reset() { *m = RecoverTxnRequest{} } func (m *RecoverTxnRequest) String() string { return proto.CompactTextString(m) } func (*RecoverTxnRequest) ProtoMessage() {} func (*RecoverTxnRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_c24af62f3f876bfa, []int{48} + return fileDescriptor_api_08e22abf3c79051e, []int{48} } func (m *RecoverTxnRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2584,7 +2608,7 @@ func (m *RecoverTxnResponse) Reset() { *m = RecoverTxnResponse{} } func (m *RecoverTxnResponse) String() string { return proto.CompactTextString(m) } func (*RecoverTxnResponse) ProtoMessage() {} func (*RecoverTxnResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_c24af62f3f876bfa, []int{49} + return fileDescriptor_api_08e22abf3c79051e, []int{49} } func (m *RecoverTxnResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2630,7 +2654,7 @@ func (m *QueryTxnRequest) Reset() { *m = QueryTxnRequest{} } func (m *QueryTxnRequest) String() string { return proto.CompactTextString(m) } func (*QueryTxnRequest) ProtoMessage() {} func (*QueryTxnRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_c24af62f3f876bfa, []int{50} + return fileDescriptor_api_08e22abf3c79051e, []int{50} } func (m *QueryTxnRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2671,7 +2695,7 @@ func (m *QueryTxnResponse) Reset() { *m = QueryTxnResponse{} } func (m *QueryTxnResponse) String() string { return proto.CompactTextString(m) } func (*QueryTxnResponse) ProtoMessage() {} func (*QueryTxnResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_c24af62f3f876bfa, []int{51} + return fileDescriptor_api_08e22abf3c79051e, []int{51} } func (m *QueryTxnResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2733,7 +2757,7 @@ func (m *QueryIntentRequest) Reset() { *m = QueryIntentRequest{} } func (m *QueryIntentRequest) String() string { return proto.CompactTextString(m) } func (*QueryIntentRequest) ProtoMessage() {} func (*QueryIntentRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_c24af62f3f876bfa, []int{52} + return fileDescriptor_api_08e22abf3c79051e, []int{52} } func (m *QueryIntentRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2771,7 +2795,7 @@ func (m *QueryIntentResponse) Reset() { *m = QueryIntentResponse{} } func (m *QueryIntentResponse) String() string { return proto.CompactTextString(m) } func (*QueryIntentResponse) ProtoMessage() {} func (*QueryIntentResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_c24af62f3f876bfa, []int{53} + return fileDescriptor_api_08e22abf3c79051e, []int{53} } func (m *QueryIntentResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2817,7 +2841,7 @@ func (m *ResolveIntentRequest) Reset() { *m = ResolveIntentRequest{} } func (m *ResolveIntentRequest) String() string { return proto.CompactTextString(m) } func (*ResolveIntentRequest) ProtoMessage() {} func (*ResolveIntentRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_c24af62f3f876bfa, []int{54} + return fileDescriptor_api_08e22abf3c79051e, []int{54} } func (m *ResolveIntentRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2854,7 +2878,7 @@ func (m *ResolveIntentResponse) Reset() { *m = ResolveIntentResponse{} } func (m *ResolveIntentResponse) String() string { return proto.CompactTextString(m) } func (*ResolveIntentResponse) ProtoMessage() {} func (*ResolveIntentResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_c24af62f3f876bfa, []int{55} + return fileDescriptor_api_08e22abf3c79051e, []int{55} } func (m *ResolveIntentResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2904,7 +2928,7 @@ func (m *ResolveIntentRangeRequest) Reset() { *m = ResolveIntentRangeReq func (m *ResolveIntentRangeRequest) String() string { return proto.CompactTextString(m) } func (*ResolveIntentRangeRequest) ProtoMessage() {} func (*ResolveIntentRangeRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_c24af62f3f876bfa, []int{56} + return fileDescriptor_api_08e22abf3c79051e, []int{56} } func (m *ResolveIntentRangeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2941,7 +2965,7 @@ func (m *ResolveIntentRangeResponse) Reset() { *m = ResolveIntentRangeRe func (m *ResolveIntentRangeResponse) String() string { return proto.CompactTextString(m) } func (*ResolveIntentRangeResponse) ProtoMessage() {} func (*ResolveIntentRangeResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_c24af62f3f876bfa, []int{57} + return fileDescriptor_api_08e22abf3c79051e, []int{57} } func (m *ResolveIntentRangeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2980,7 +3004,7 @@ func (m *MergeRequest) Reset() { *m = MergeRequest{} } func (m *MergeRequest) String() string { return proto.CompactTextString(m) } func (*MergeRequest) ProtoMessage() {} func (*MergeRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_c24af62f3f876bfa, []int{58} + return fileDescriptor_api_08e22abf3c79051e, []int{58} } func (m *MergeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3016,7 +3040,7 @@ func (m *MergeResponse) Reset() { *m = MergeResponse{} } func (m *MergeResponse) String() string { return proto.CompactTextString(m) } func (*MergeResponse) ProtoMessage() {} func (*MergeResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_c24af62f3f876bfa, []int{59} + return fileDescriptor_api_08e22abf3c79051e, []int{59} } func (m *MergeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3063,7 +3087,7 @@ func (m *TruncateLogRequest) Reset() { *m = TruncateLogRequest{} } func (m *TruncateLogRequest) String() string { return proto.CompactTextString(m) } func (*TruncateLogRequest) ProtoMessage() {} func (*TruncateLogRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_c24af62f3f876bfa, []int{60} + return fileDescriptor_api_08e22abf3c79051e, []int{60} } func (m *TruncateLogRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3099,7 +3123,7 @@ func (m *TruncateLogResponse) Reset() { *m = TruncateLogResponse{} } func (m *TruncateLogResponse) String() string { return proto.CompactTextString(m) } func (*TruncateLogResponse) ProtoMessage() {} func (*TruncateLogResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_c24af62f3f876bfa, []int{61} + return fileDescriptor_api_08e22abf3c79051e, []int{61} } func (m *TruncateLogResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3145,7 +3169,7 @@ func (m *RequestLeaseRequest) Reset() { *m = RequestLeaseRequest{} } func (m *RequestLeaseRequest) String() string { return proto.CompactTextString(m) } func (*RequestLeaseRequest) ProtoMessage() {} func (*RequestLeaseRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_c24af62f3f876bfa, []int{62} + return fileDescriptor_api_08e22abf3c79051e, []int{62} } func (m *RequestLeaseRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3196,7 +3220,7 @@ func (m *TransferLeaseRequest) Reset() { *m = TransferLeaseRequest{} } func (m *TransferLeaseRequest) String() string { return proto.CompactTextString(m) } func (*TransferLeaseRequest) ProtoMessage() {} func (*TransferLeaseRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_c24af62f3f876bfa, []int{63} + return fileDescriptor_api_08e22abf3c79051e, []int{63} } func (m *TransferLeaseRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3235,7 +3259,7 @@ func (m *LeaseInfoRequest) Reset() { *m = LeaseInfoRequest{} } func (m *LeaseInfoRequest) String() string { return proto.CompactTextString(m) } func (*LeaseInfoRequest) ProtoMessage() {} func (*LeaseInfoRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_c24af62f3f876bfa, []int{64} + return fileDescriptor_api_08e22abf3c79051e, []int{64} } func (m *LeaseInfoRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3274,7 +3298,7 @@ func (m *LeaseInfoResponse) Reset() { *m = LeaseInfoResponse{} } func (m *LeaseInfoResponse) String() string { return proto.CompactTextString(m) } func (*LeaseInfoResponse) ProtoMessage() {} func (*LeaseInfoResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_c24af62f3f876bfa, []int{65} + return fileDescriptor_api_08e22abf3c79051e, []int{65} } func (m *LeaseInfoResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3311,7 +3335,7 @@ func (m *RequestLeaseResponse) Reset() { *m = RequestLeaseResponse{} } func (m *RequestLeaseResponse) String() string { return proto.CompactTextString(m) } func (*RequestLeaseResponse) ProtoMessage() {} func (*RequestLeaseResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_c24af62f3f876bfa, []int{66} + return fileDescriptor_api_08e22abf3c79051e, []int{66} } func (m *RequestLeaseResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3363,7 +3387,7 @@ func (m *ComputeChecksumRequest) Reset() { *m = ComputeChecksumRequest{} func (m *ComputeChecksumRequest) String() string { return proto.CompactTextString(m) } func (*ComputeChecksumRequest) ProtoMessage() {} func (*ComputeChecksumRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_c24af62f3f876bfa, []int{67} + return fileDescriptor_api_08e22abf3c79051e, []int{67} } func (m *ComputeChecksumRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3402,7 +3426,7 @@ func (m *ComputeChecksumResponse) Reset() { *m = ComputeChecksumResponse func (m *ComputeChecksumResponse) String() string { return proto.CompactTextString(m) } func (*ComputeChecksumResponse) ProtoMessage() {} func (*ComputeChecksumResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_c24af62f3f876bfa, []int{68} + return fileDescriptor_api_08e22abf3c79051e, []int{68} } func (m *ComputeChecksumResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3443,7 +3467,7 @@ func (m *ExportStorage) Reset() { *m = ExportStorage{} } func (m *ExportStorage) String() string { return proto.CompactTextString(m) } func (*ExportStorage) ProtoMessage() {} func (*ExportStorage) Descriptor() ([]byte, []int) { - return fileDescriptor_api_c24af62f3f876bfa, []int{69} + return fileDescriptor_api_08e22abf3c79051e, []int{69} } func (m *ExportStorage) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3479,7 +3503,7 @@ func (m *ExportStorage_LocalFilePath) Reset() { *m = ExportStorage_Local func (m *ExportStorage_LocalFilePath) String() string { return proto.CompactTextString(m) } func (*ExportStorage_LocalFilePath) ProtoMessage() {} func (*ExportStorage_LocalFilePath) Descriptor() ([]byte, []int) { - return fileDescriptor_api_c24af62f3f876bfa, []int{69, 0} + return fileDescriptor_api_08e22abf3c79051e, []int{69, 0} } func (m *ExportStorage_LocalFilePath) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3514,7 +3538,7 @@ func (m *ExportStorage_Http) Reset() { *m = ExportStorage_Http{} } func (m *ExportStorage_Http) String() string { return proto.CompactTextString(m) } func (*ExportStorage_Http) ProtoMessage() {} func (*ExportStorage_Http) Descriptor() ([]byte, []int) { - return fileDescriptor_api_c24af62f3f876bfa, []int{69, 1} + return fileDescriptor_api_08e22abf3c79051e, []int{69, 1} } func (m *ExportStorage_Http) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3555,7 +3579,7 @@ func (m *ExportStorage_S3) Reset() { *m = ExportStorage_S3{} } func (m *ExportStorage_S3) String() string { return proto.CompactTextString(m) } func (*ExportStorage_S3) ProtoMessage() {} func (*ExportStorage_S3) Descriptor() ([]byte, []int) { - return fileDescriptor_api_c24af62f3f876bfa, []int{69, 2} + return fileDescriptor_api_08e22abf3c79051e, []int{69, 2} } func (m *ExportStorage_S3) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3596,7 +3620,7 @@ func (m *ExportStorage_GCS) Reset() { *m = ExportStorage_GCS{} } func (m *ExportStorage_GCS) String() string { return proto.CompactTextString(m) } func (*ExportStorage_GCS) ProtoMessage() {} func (*ExportStorage_GCS) Descriptor() ([]byte, []int) { - return fileDescriptor_api_c24af62f3f876bfa, []int{69, 3} + return fileDescriptor_api_08e22abf3c79051e, []int{69, 3} } func (m *ExportStorage_GCS) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3634,7 +3658,7 @@ func (m *ExportStorage_Azure) Reset() { *m = ExportStorage_Azure{} } func (m *ExportStorage_Azure) String() string { return proto.CompactTextString(m) } func (*ExportStorage_Azure) ProtoMessage() {} func (*ExportStorage_Azure) Descriptor() ([]byte, []int) { - return fileDescriptor_api_c24af62f3f876bfa, []int{69, 4} + return fileDescriptor_api_08e22abf3c79051e, []int{69, 4} } func (m *ExportStorage_Azure) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3675,7 +3699,7 @@ func (m *ExportStorage_Workload) Reset() { *m = ExportStorage_Workload{} func (m *ExportStorage_Workload) String() string { return proto.CompactTextString(m) } func (*ExportStorage_Workload) ProtoMessage() {} func (*ExportStorage_Workload) Descriptor() ([]byte, []int) { - return fileDescriptor_api_c24af62f3f876bfa, []int{69, 5} + return fileDescriptor_api_08e22abf3c79051e, []int{69, 5} } func (m *ExportStorage_Workload) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3717,7 +3741,7 @@ func (m *WriteBatchRequest) Reset() { *m = WriteBatchRequest{} } func (m *WriteBatchRequest) String() string { return proto.CompactTextString(m) } func (*WriteBatchRequest) ProtoMessage() {} func (*WriteBatchRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_c24af62f3f876bfa, []int{70} + return fileDescriptor_api_08e22abf3c79051e, []int{70} } func (m *WriteBatchRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3753,7 +3777,7 @@ func (m *WriteBatchResponse) Reset() { *m = WriteBatchResponse{} } func (m *WriteBatchResponse) String() string { return proto.CompactTextString(m) } func (*WriteBatchResponse) ProtoMessage() {} func (*WriteBatchResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_c24af62f3f876bfa, []int{71} + return fileDescriptor_api_08e22abf3c79051e, []int{71} } func (m *WriteBatchResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3813,7 +3837,7 @@ func (m *ExportRequest) Reset() { *m = ExportRequest{} } func (m *ExportRequest) String() string { return proto.CompactTextString(m) } func (*ExportRequest) ProtoMessage() {} func (*ExportRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_c24af62f3f876bfa, []int{72} + return fileDescriptor_api_08e22abf3c79051e, []int{72} } func (m *ExportRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3851,7 +3875,7 @@ func (m *BulkOpSummary) Reset() { *m = BulkOpSummary{} } func (m *BulkOpSummary) String() string { return proto.CompactTextString(m) } func (*BulkOpSummary) ProtoMessage() {} func (*BulkOpSummary) Descriptor() ([]byte, []int) { - return fileDescriptor_api_c24af62f3f876bfa, []int{73} + return fileDescriptor_api_08e22abf3c79051e, []int{73} } func (m *BulkOpSummary) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3889,7 +3913,7 @@ func (m *ExportResponse) Reset() { *m = ExportResponse{} } func (m *ExportResponse) String() string { return proto.CompactTextString(m) } func (*ExportResponse) ProtoMessage() {} func (*ExportResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_c24af62f3f876bfa, []int{74} + return fileDescriptor_api_08e22abf3c79051e, []int{74} } func (m *ExportResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3930,7 +3954,7 @@ func (m *ExportResponse_File) Reset() { *m = ExportResponse_File{} } func (m *ExportResponse_File) String() string { return proto.CompactTextString(m) } func (*ExportResponse_File) ProtoMessage() {} func (*ExportResponse_File) Descriptor() ([]byte, []int) { - return fileDescriptor_api_c24af62f3f876bfa, []int{74, 0} + return fileDescriptor_api_08e22abf3c79051e, []int{74, 0} } func (m *ExportResponse_File) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3982,7 +4006,7 @@ func (m *ImportRequest) Reset() { *m = ImportRequest{} } func (m *ImportRequest) String() string { return proto.CompactTextString(m) } func (*ImportRequest) ProtoMessage() {} func (*ImportRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_c24af62f3f876bfa, []int{75} + return fileDescriptor_api_08e22abf3c79051e, []int{75} } func (m *ImportRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4019,7 +4043,7 @@ func (m *ImportRequest_File) Reset() { *m = ImportRequest_File{} } func (m *ImportRequest_File) String() string { return proto.CompactTextString(m) } func (*ImportRequest_File) ProtoMessage() {} func (*ImportRequest_File) Descriptor() ([]byte, []int) { - return fileDescriptor_api_c24af62f3f876bfa, []int{75, 0} + return fileDescriptor_api_08e22abf3c79051e, []int{75, 0} } func (m *ImportRequest_File) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4057,7 +4081,7 @@ func (m *ImportRequest_TableRekey) Reset() { *m = ImportRequest_TableRek func (m *ImportRequest_TableRekey) String() string { return proto.CompactTextString(m) } func (*ImportRequest_TableRekey) ProtoMessage() {} func (*ImportRequest_TableRekey) Descriptor() ([]byte, []int) { - return fileDescriptor_api_c24af62f3f876bfa, []int{75, 1} + return fileDescriptor_api_08e22abf3c79051e, []int{75, 1} } func (m *ImportRequest_TableRekey) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4094,7 +4118,7 @@ func (m *ImportResponse) Reset() { *m = ImportResponse{} } func (m *ImportResponse) String() string { return proto.CompactTextString(m) } func (*ImportResponse) ProtoMessage() {} func (*ImportResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_c24af62f3f876bfa, []int{76} + return fileDescriptor_api_08e22abf3c79051e, []int{76} } func (m *ImportResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4134,7 +4158,7 @@ func (m *AdminScatterRequest) Reset() { *m = AdminScatterRequest{} } func (m *AdminScatterRequest) String() string { return proto.CompactTextString(m) } func (*AdminScatterRequest) ProtoMessage() {} func (*AdminScatterRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_c24af62f3f876bfa, []int{77} + return fileDescriptor_api_08e22abf3c79051e, []int{77} } func (m *AdminScatterRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4171,7 +4195,7 @@ func (m *AdminScatterResponse) Reset() { *m = AdminScatterResponse{} } func (m *AdminScatterResponse) String() string { return proto.CompactTextString(m) } func (*AdminScatterResponse) ProtoMessage() {} func (*AdminScatterResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_c24af62f3f876bfa, []int{78} + return fileDescriptor_api_08e22abf3c79051e, []int{78} } func (m *AdminScatterResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4206,7 +4230,7 @@ func (m *AdminScatterResponse_Range) Reset() { *m = AdminScatterResponse func (m *AdminScatterResponse_Range) String() string { return proto.CompactTextString(m) } func (*AdminScatterResponse_Range) ProtoMessage() {} func (*AdminScatterResponse_Range) Descriptor() ([]byte, []int) { - return fileDescriptor_api_c24af62f3f876bfa, []int{78, 0} + return fileDescriptor_api_08e22abf3c79051e, []int{78, 0} } func (m *AdminScatterResponse_Range) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4244,7 +4268,7 @@ func (m *AddSSTableRequest) Reset() { *m = AddSSTableRequest{} } func (m *AddSSTableRequest) String() string { return proto.CompactTextString(m) } func (*AddSSTableRequest) ProtoMessage() {} func (*AddSSTableRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_c24af62f3f876bfa, []int{79} + return fileDescriptor_api_08e22abf3c79051e, []int{79} } func (m *AddSSTableRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4280,7 +4304,7 @@ func (m *AddSSTableResponse) Reset() { *m = AddSSTableResponse{} } func (m *AddSSTableResponse) String() string { return proto.CompactTextString(m) } func (*AddSSTableResponse) ProtoMessage() {} func (*AddSSTableResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_c24af62f3f876bfa, []int{80} + return fileDescriptor_api_08e22abf3c79051e, []int{80} } func (m *AddSSTableResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4326,7 +4350,7 @@ func (m *RefreshRequest) Reset() { *m = RefreshRequest{} } func (m *RefreshRequest) String() string { return proto.CompactTextString(m) } func (*RefreshRequest) ProtoMessage() {} func (*RefreshRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_c24af62f3f876bfa, []int{81} + return fileDescriptor_api_08e22abf3c79051e, []int{81} } func (m *RefreshRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4362,7 +4386,7 @@ func (m *RefreshResponse) Reset() { *m = RefreshResponse{} } func (m *RefreshResponse) String() string { return proto.CompactTextString(m) } func (*RefreshResponse) ProtoMessage() {} func (*RefreshResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_c24af62f3f876bfa, []int{82} + return fileDescriptor_api_08e22abf3c79051e, []int{82} } func (m *RefreshResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4404,7 +4428,7 @@ func (m *RefreshRangeRequest) Reset() { *m = RefreshRangeRequest{} } func (m *RefreshRangeRequest) String() string { return proto.CompactTextString(m) } func (*RefreshRangeRequest) ProtoMessage() {} func (*RefreshRangeRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_c24af62f3f876bfa, []int{83} + return fileDescriptor_api_08e22abf3c79051e, []int{83} } func (m *RefreshRangeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4440,7 +4464,7 @@ func (m *RefreshRangeResponse) Reset() { *m = RefreshRangeResponse{} } func (m *RefreshRangeResponse) String() string { return proto.CompactTextString(m) } func (*RefreshRangeResponse) ProtoMessage() {} func (*RefreshRangeResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_c24af62f3f876bfa, []int{84} + return fileDescriptor_api_08e22abf3c79051e, []int{84} } func (m *RefreshRangeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4494,7 +4518,7 @@ func (m *SubsumeRequest) Reset() { *m = SubsumeRequest{} } func (m *SubsumeRequest) String() string { return proto.CompactTextString(m) } func (*SubsumeRequest) ProtoMessage() {} func (*SubsumeRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_c24af62f3f876bfa, []int{85} + return fileDescriptor_api_08e22abf3c79051e, []int{85} } func (m *SubsumeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4541,7 +4565,7 @@ func (m *SubsumeResponse) Reset() { *m = SubsumeResponse{} } func (m *SubsumeResponse) String() string { return proto.CompactTextString(m) } func (*SubsumeResponse) ProtoMessage() {} func (*SubsumeResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_c24af62f3f876bfa, []int{86} + return fileDescriptor_api_08e22abf3c79051e, []int{86} } func (m *SubsumeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4578,7 +4602,7 @@ func (m *RangeStatsRequest) Reset() { *m = RangeStatsRequest{} } func (m *RangeStatsRequest) String() string { return proto.CompactTextString(m) } func (*RangeStatsRequest) ProtoMessage() {} func (*RangeStatsRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_c24af62f3f876bfa, []int{87} + return fileDescriptor_api_08e22abf3c79051e, []int{87} } func (m *RangeStatsRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4619,7 +4643,7 @@ func (m *RangeStatsResponse) Reset() { *m = RangeStatsResponse{} } func (m *RangeStatsResponse) String() string { return proto.CompactTextString(m) } func (*RangeStatsResponse) ProtoMessage() {} func (*RangeStatsResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_c24af62f3f876bfa, []int{88} + return fileDescriptor_api_08e22abf3c79051e, []int{88} } func (m *RangeStatsResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4702,7 +4726,7 @@ func (m *RequestUnion) Reset() { *m = RequestUnion{} } func (m *RequestUnion) String() string { return proto.CompactTextString(m) } func (*RequestUnion) ProtoMessage() {} func (*RequestUnion) Descriptor() ([]byte, []int) { - return fileDescriptor_api_c24af62f3f876bfa, []int{89} + return fileDescriptor_api_08e22abf3c79051e, []int{89} } func (m *RequestUnion) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -6092,7 +6116,7 @@ func (m *ResponseUnion) Reset() { *m = ResponseUnion{} } func (m *ResponseUnion) String() string { return proto.CompactTextString(m) } func (*ResponseUnion) ProtoMessage() {} func (*ResponseUnion) Descriptor() ([]byte, []int) { - return fileDescriptor_api_c24af62f3f876bfa, []int{90} + return fileDescriptor_api_08e22abf3c79051e, []int{90} } func (m *ResponseUnion) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -7471,7 +7495,7 @@ func (m *Header) Reset() { *m = Header{} } func (m *Header) String() string { return proto.CompactTextString(m) } func (*Header) ProtoMessage() {} func (*Header) Descriptor() ([]byte, []int) { - return fileDescriptor_api_c24af62f3f876bfa, []int{91} + return fileDescriptor_api_08e22abf3c79051e, []int{91} } func (m *Header) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -7509,7 +7533,7 @@ type BatchRequest struct { func (m *BatchRequest) Reset() { *m = BatchRequest{} } func (*BatchRequest) ProtoMessage() {} func (*BatchRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_c24af62f3f876bfa, []int{92} + return fileDescriptor_api_08e22abf3c79051e, []int{92} } func (m *BatchRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -7548,7 +7572,7 @@ type BatchResponse struct { func (m *BatchResponse) Reset() { *m = BatchResponse{} } func (*BatchResponse) ProtoMessage() {} func (*BatchResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_c24af62f3f876bfa, []int{93} + return fileDescriptor_api_08e22abf3c79051e, []int{93} } func (m *BatchResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -7609,7 +7633,7 @@ func (m *BatchResponse_Header) Reset() { *m = BatchResponse_Header{} } func (m *BatchResponse_Header) String() string { return proto.CompactTextString(m) } func (*BatchResponse_Header) ProtoMessage() {} func (*BatchResponse_Header) Descriptor() ([]byte, []int) { - return fileDescriptor_api_c24af62f3f876bfa, []int{93, 0} + return fileDescriptor_api_08e22abf3c79051e, []int{93, 0} } func (m *BatchResponse_Header) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -7647,7 +7671,7 @@ func (m *RangeFeedRequest) Reset() { *m = RangeFeedRequest{} } func (m *RangeFeedRequest) String() string { return proto.CompactTextString(m) } func (*RangeFeedRequest) ProtoMessage() {} func (*RangeFeedRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_c24af62f3f876bfa, []int{94} + return fileDescriptor_api_08e22abf3c79051e, []int{94} } func (m *RangeFeedRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -7685,7 +7709,7 @@ func (m *RangeFeedValue) Reset() { *m = RangeFeedValue{} } func (m *RangeFeedValue) String() string { return proto.CompactTextString(m) } func (*RangeFeedValue) ProtoMessage() {} func (*RangeFeedValue) Descriptor() ([]byte, []int) { - return fileDescriptor_api_c24af62f3f876bfa, []int{95} + return fileDescriptor_api_08e22abf3c79051e, []int{95} } func (m *RangeFeedValue) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -7728,7 +7752,7 @@ func (m *RangeFeedCheckpoint) Reset() { *m = RangeFeedCheckpoint{} } func (m *RangeFeedCheckpoint) String() string { return proto.CompactTextString(m) } func (*RangeFeedCheckpoint) ProtoMessage() {} func (*RangeFeedCheckpoint) Descriptor() ([]byte, []int) { - return fileDescriptor_api_c24af62f3f876bfa, []int{96} + return fileDescriptor_api_08e22abf3c79051e, []int{96} } func (m *RangeFeedCheckpoint) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -7767,7 +7791,7 @@ func (m *RangeFeedError) Reset() { *m = RangeFeedError{} } func (m *RangeFeedError) String() string { return proto.CompactTextString(m) } func (*RangeFeedError) ProtoMessage() {} func (*RangeFeedError) Descriptor() ([]byte, []int) { - return fileDescriptor_api_c24af62f3f876bfa, []int{97} + return fileDescriptor_api_08e22abf3c79051e, []int{97} } func (m *RangeFeedError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -7806,7 +7830,7 @@ func (m *RangeFeedEvent) Reset() { *m = RangeFeedEvent{} } func (m *RangeFeedEvent) String() string { return proto.CompactTextString(m) } func (*RangeFeedEvent) ProtoMessage() {} func (*RangeFeedEvent) Descriptor() ([]byte, []int) { - return fileDescriptor_api_c24af62f3f876bfa, []int{98} + return fileDescriptor_api_08e22abf3c79051e, []int{98} } func (m *RangeFeedEvent) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8418,6 +8442,14 @@ func (this *EndTransactionRequest) Equal(that interface{}) bool { return false } } + if len(this.InFlightWrites) != len(that1.InFlightWrites) { + return false + } + for i := range this.InFlightWrites { + if !this.InFlightWrites[i].Equal(&that1.InFlightWrites[i]) { + return false + } + } if this.Require1PC != that1.Require1PC { return false } @@ -11037,6 +11069,20 @@ func (m *EndTransactionRequest) MarshalTo(dAtA []byte) (int, error) { } i++ } + if len(m.InFlightWrites) > 0 { + for _, msg := range m.InFlightWrites { + dAtA[i] = 0x8a + i++ + dAtA[i] = 0x1 + i++ + i = encodeVarintApi(dAtA, i, uint64(msg.Size())) + n, err := msg.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n + } + } return i, nil } @@ -11073,6 +11119,14 @@ func (m *EndTransactionResponse) MarshalTo(dAtA []byte) (int, error) { } i++ } + dAtA[i] = 0x2a + i++ + i = encodeVarintApi(dAtA, i, uint64(m.StagingTimestamp.Size())) + n42, err := m.StagingTimestamp.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n42 return i, nil } @@ -11094,11 +11148,11 @@ func (m *AdminSplitRequest) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.RequestHeader.Size())) - n42, err := m.RequestHeader.MarshalTo(dAtA[i:]) + n43, err := m.RequestHeader.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n42 + i += n43 if len(m.SplitKey) > 0 { dAtA[i] = 0x12 i++ @@ -11126,11 +11180,11 @@ func (m *AdminSplitResponse) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.ResponseHeader.Size())) - n43, err := m.ResponseHeader.MarshalTo(dAtA[i:]) + n44, err := m.ResponseHeader.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n43 + i += n44 return i, nil } @@ -11152,11 +11206,11 @@ func (m *AdminMergeRequest) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.RequestHeader.Size())) - n44, err := m.RequestHeader.MarshalTo(dAtA[i:]) + n45, err := m.RequestHeader.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n44 + i += n45 return i, nil } @@ -11178,11 +11232,11 @@ func (m *AdminMergeResponse) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.ResponseHeader.Size())) - n45, err := m.ResponseHeader.MarshalTo(dAtA[i:]) + n46, err := m.ResponseHeader.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n45 + i += n46 return i, nil } @@ -11204,11 +11258,11 @@ func (m *AdminTransferLeaseRequest) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.RequestHeader.Size())) - n46, err := m.RequestHeader.MarshalTo(dAtA[i:]) + n47, err := m.RequestHeader.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n46 + i += n47 if m.Target != 0 { dAtA[i] = 0x10 i++ @@ -11235,11 +11289,11 @@ func (m *AdminTransferLeaseResponse) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.ResponseHeader.Size())) - n47, err := m.ResponseHeader.MarshalTo(dAtA[i:]) + n48, err := m.ResponseHeader.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n47 + i += n48 return i, nil } @@ -11261,11 +11315,11 @@ func (m *AdminChangeReplicasRequest) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.RequestHeader.Size())) - n48, err := m.RequestHeader.MarshalTo(dAtA[i:]) + n49, err := m.RequestHeader.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n48 + i += n49 if m.ChangeType != 0 { dAtA[i] = 0x10 i++ @@ -11287,11 +11341,11 @@ func (m *AdminChangeReplicasRequest) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x22 i++ i = encodeVarintApi(dAtA, i, uint64(m.ExpDesc.Size())) - n49, err := m.ExpDesc.MarshalTo(dAtA[i:]) + n50, err := m.ExpDesc.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n49 + i += n50 } return i, nil } @@ -11314,20 +11368,20 @@ func (m *AdminChangeReplicasResponse) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.ResponseHeader.Size())) - n50, err := m.ResponseHeader.MarshalTo(dAtA[i:]) + n51, err := m.ResponseHeader.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n50 + i += n51 if m.Desc != nil { dAtA[i] = 0x12 i++ i = encodeVarintApi(dAtA, i, uint64(m.Desc.Size())) - n51, err := m.Desc.MarshalTo(dAtA[i:]) + n52, err := m.Desc.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n51 + i += n52 } return i, nil } @@ -11350,11 +11404,11 @@ func (m *AdminRelocateRangeRequest) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.RequestHeader.Size())) - n52, err := m.RequestHeader.MarshalTo(dAtA[i:]) + n53, err := m.RequestHeader.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n52 + i += n53 if len(m.Targets) > 0 { for _, msg := range m.Targets { dAtA[i] = 0x12 @@ -11388,11 +11442,11 @@ func (m *AdminRelocateRangeResponse) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.ResponseHeader.Size())) - n53, err := m.ResponseHeader.MarshalTo(dAtA[i:]) + n54, err := m.ResponseHeader.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n53 + i += n54 return i, nil } @@ -11414,19 +11468,19 @@ func (m *HeartbeatTxnRequest) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.RequestHeader.Size())) - n54, err := m.RequestHeader.MarshalTo(dAtA[i:]) + n55, err := m.RequestHeader.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n54 + i += n55 dAtA[i] = 0x12 i++ i = encodeVarintApi(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 return i, nil } @@ -11448,11 +11502,11 @@ func (m *HeartbeatTxnResponse) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.ResponseHeader.Size())) - n56, err := m.ResponseHeader.MarshalTo(dAtA[i:]) + n57, err := m.ResponseHeader.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n56 + i += n57 return i, nil } @@ -11474,11 +11528,11 @@ func (m *GCRequest) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.RequestHeader.Size())) - n57, err := m.RequestHeader.MarshalTo(dAtA[i:]) + n58, err := m.RequestHeader.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n57 + i += n58 if len(m.Keys) > 0 { for _, msg := range m.Keys { dAtA[i] = 0x1a @@ -11494,19 +11548,19 @@ func (m *GCRequest) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x22 i++ i = encodeVarintApi(dAtA, i, uint64(m.Threshold.Size())) - n58, err := m.Threshold.MarshalTo(dAtA[i:]) + n59, err := m.Threshold.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n58 + i += n59 dAtA[i] = 0x2a i++ i = encodeVarintApi(dAtA, i, uint64(m.TxnSpanGCThreshold.Size())) - n59, err := m.TxnSpanGCThreshold.MarshalTo(dAtA[i:]) + n60, err := m.TxnSpanGCThreshold.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n59 + i += n60 return i, nil } @@ -11534,11 +11588,11 @@ func (m *GCRequest_GCKey) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x12 i++ i = encodeVarintApi(dAtA, i, uint64(m.Timestamp.Size())) - n60, err := m.Timestamp.MarshalTo(dAtA[i:]) + n61, err := m.Timestamp.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n60 + i += n61 return i, nil } @@ -11560,11 +11614,11 @@ func (m *GCResponse) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.ResponseHeader.Size())) - n61, err := m.ResponseHeader.MarshalTo(dAtA[i:]) + n62, err := m.ResponseHeader.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n61 + i += n62 return i, nil } @@ -11586,43 +11640,43 @@ func (m *PushTxnRequest) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.RequestHeader.Size())) - n62, err := m.RequestHeader.MarshalTo(dAtA[i:]) + n63, err := m.RequestHeader.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n62 + i += n63 dAtA[i] = 0x12 i++ i = encodeVarintApi(dAtA, i, uint64(m.PusherTxn.Size())) - n63, err := m.PusherTxn.MarshalTo(dAtA[i:]) + n64, err := m.PusherTxn.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n63 + i += n64 dAtA[i] = 0x1a i++ i = encodeVarintApi(dAtA, i, uint64(m.PusheeTxn.Size())) - n64, err := m.PusheeTxn.MarshalTo(dAtA[i:]) + n65, err := m.PusheeTxn.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n64 + i += n65 dAtA[i] = 0x22 i++ i = encodeVarintApi(dAtA, i, uint64(m.PushTo.Size())) - n65, err := m.PushTo.MarshalTo(dAtA[i:]) + n66, err := m.PushTo.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n65 + i += n66 dAtA[i] = 0x2a i++ i = encodeVarintApi(dAtA, i, uint64(m.DeprecatedNow.Size())) - n66, err := m.DeprecatedNow.MarshalTo(dAtA[i:]) + n67, err := m.DeprecatedNow.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n66 + i += n67 if m.PushType != 0 { dAtA[i] = 0x30 i++ @@ -11669,19 +11723,19 @@ func (m *PushTxnResponse) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.ResponseHeader.Size())) - n67, err := m.ResponseHeader.MarshalTo(dAtA[i:]) + n68, err := m.ResponseHeader.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n67 + i += n68 dAtA[i] = 0x12 i++ i = encodeVarintApi(dAtA, i, uint64(m.PusheeTxn.Size())) - n68, err := m.PusheeTxn.MarshalTo(dAtA[i:]) + n69, err := m.PusheeTxn.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n68 + i += n69 return i, nil } @@ -11703,19 +11757,19 @@ func (m *RecoverTxnRequest) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.RequestHeader.Size())) - n69, err := m.RequestHeader.MarshalTo(dAtA[i:]) + n70, err := m.RequestHeader.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n69 + i += n70 dAtA[i] = 0x12 i++ i = encodeVarintApi(dAtA, i, uint64(m.Txn.Size())) - n70, err := m.Txn.MarshalTo(dAtA[i:]) + n71, err := m.Txn.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n70 + i += n71 if m.ImplicitlyCommitted { dAtA[i] = 0x18 i++ @@ -11747,19 +11801,19 @@ func (m *RecoverTxnResponse) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.ResponseHeader.Size())) - n71, err := m.ResponseHeader.MarshalTo(dAtA[i:]) + n72, err := m.ResponseHeader.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n71 + i += n72 dAtA[i] = 0x12 i++ i = encodeVarintApi(dAtA, i, uint64(m.RecoveredTxn.Size())) - n72, err := m.RecoveredTxn.MarshalTo(dAtA[i:]) + n73, err := m.RecoveredTxn.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n72 + i += n73 return i, nil } @@ -11781,19 +11835,19 @@ func (m *QueryTxnRequest) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.RequestHeader.Size())) - n73, err := m.RequestHeader.MarshalTo(dAtA[i:]) + n74, err := m.RequestHeader.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n73 + i += n74 dAtA[i] = 0x12 i++ i = encodeVarintApi(dAtA, i, uint64(m.Txn.Size())) - n74, err := m.Txn.MarshalTo(dAtA[i:]) + n75, err := m.Txn.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n74 + i += n75 if m.WaitForUpdate { dAtA[i] = 0x18 i++ @@ -11837,19 +11891,19 @@ func (m *QueryTxnResponse) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.ResponseHeader.Size())) - n75, err := m.ResponseHeader.MarshalTo(dAtA[i:]) + n76, err := m.ResponseHeader.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n75 + i += n76 dAtA[i] = 0x12 i++ i = encodeVarintApi(dAtA, i, uint64(m.QueriedTxn.Size())) - n76, err := m.QueriedTxn.MarshalTo(dAtA[i:]) + n77, err := m.QueriedTxn.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n76 + i += n77 if len(m.WaitingTxns) > 0 { for _, msg := range m.WaitingTxns { dAtA[i] = 0x1a @@ -11883,19 +11937,19 @@ func (m *QueryIntentRequest) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.RequestHeader.Size())) - n77, err := m.RequestHeader.MarshalTo(dAtA[i:]) + n78, err := m.RequestHeader.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n77 + i += n78 dAtA[i] = 0x12 i++ i = encodeVarintApi(dAtA, i, uint64(m.Txn.Size())) - n78, err := m.Txn.MarshalTo(dAtA[i:]) + n79, err := m.Txn.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n78 + i += n79 if m.ErrorIfMissing { dAtA[i] = 0x18 i++ @@ -11927,11 +11981,11 @@ func (m *QueryIntentResponse) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.ResponseHeader.Size())) - n79, err := m.ResponseHeader.MarshalTo(dAtA[i:]) + n80, err := m.ResponseHeader.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n79 + i += n80 if m.FoundIntent { dAtA[i] = 0x10 i++ @@ -11963,19 +12017,19 @@ func (m *ResolveIntentRequest) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.RequestHeader.Size())) - n80, err := m.RequestHeader.MarshalTo(dAtA[i:]) + n81, err := m.RequestHeader.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n80 + i += n81 dAtA[i] = 0x12 i++ i = encodeVarintApi(dAtA, i, uint64(m.IntentTxn.Size())) - n81, err := m.IntentTxn.MarshalTo(dAtA[i:]) + n82, err := m.IntentTxn.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n81 + i += n82 if m.Status != 0 { dAtA[i] = 0x18 i++ @@ -12012,11 +12066,11 @@ func (m *ResolveIntentResponse) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.ResponseHeader.Size())) - n82, err := m.ResponseHeader.MarshalTo(dAtA[i:]) + n83, err := m.ResponseHeader.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n82 + i += n83 return i, nil } @@ -12038,19 +12092,19 @@ func (m *ResolveIntentRangeRequest) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.RequestHeader.Size())) - n83, err := m.RequestHeader.MarshalTo(dAtA[i:]) + n84, err := m.RequestHeader.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n83 + i += n84 dAtA[i] = 0x12 i++ i = encodeVarintApi(dAtA, i, uint64(m.IntentTxn.Size())) - n84, err := m.IntentTxn.MarshalTo(dAtA[i:]) + n85, err := m.IntentTxn.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n84 + i += n85 if m.Status != 0 { dAtA[i] = 0x18 i++ @@ -12069,11 +12123,11 @@ func (m *ResolveIntentRangeRequest) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2a i++ i = encodeVarintApi(dAtA, i, uint64(m.MinTimestamp.Size())) - n85, err := m.MinTimestamp.MarshalTo(dAtA[i:]) + n86, err := m.MinTimestamp.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n85 + i += n86 return i, nil } @@ -12095,11 +12149,11 @@ func (m *ResolveIntentRangeResponse) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.ResponseHeader.Size())) - n86, err := m.ResponseHeader.MarshalTo(dAtA[i:]) + n87, err := m.ResponseHeader.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n86 + i += n87 return i, nil } @@ -12121,19 +12175,19 @@ func (m *MergeRequest) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.RequestHeader.Size())) - n87, err := m.RequestHeader.MarshalTo(dAtA[i:]) + n88, err := m.RequestHeader.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n87 + i += n88 dAtA[i] = 0x12 i++ i = encodeVarintApi(dAtA, i, uint64(m.Value.Size())) - n88, err := m.Value.MarshalTo(dAtA[i:]) + n89, err := m.Value.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n88 + i += n89 return i, nil } @@ -12155,11 +12209,11 @@ func (m *MergeResponse) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.ResponseHeader.Size())) - n89, err := m.ResponseHeader.MarshalTo(dAtA[i:]) + n90, err := m.ResponseHeader.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n89 + i += n90 return i, nil } @@ -12181,11 +12235,11 @@ func (m *TruncateLogRequest) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.RequestHeader.Size())) - n90, err := m.RequestHeader.MarshalTo(dAtA[i:]) + n91, err := m.RequestHeader.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n90 + i += n91 if m.Index != 0 { dAtA[i] = 0x10 i++ @@ -12217,11 +12271,11 @@ func (m *TruncateLogResponse) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.ResponseHeader.Size())) - n91, err := m.ResponseHeader.MarshalTo(dAtA[i:]) + n92, err := m.ResponseHeader.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n91 + i += n92 return i, nil } @@ -12243,36 +12297,36 @@ func (m *RequestLeaseRequest) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.RequestHeader.Size())) - n92, err := m.RequestHeader.MarshalTo(dAtA[i:]) + n93, err := m.RequestHeader.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n92 + i += n93 dAtA[i] = 0x12 i++ i = encodeVarintApi(dAtA, i, uint64(m.Lease.Size())) - n93, err := m.Lease.MarshalTo(dAtA[i:]) + n94, err := m.Lease.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n93 + i += n94 dAtA[i] = 0x1a i++ i = encodeVarintApi(dAtA, i, uint64(m.PrevLease.Size())) - n94, err := m.PrevLease.MarshalTo(dAtA[i:]) + n95, err := m.PrevLease.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n94 + i += n95 if m.MinProposedTS != nil { dAtA[i] = 0x22 i++ i = encodeVarintApi(dAtA, i, uint64(m.MinProposedTS.Size())) - n95, err := m.MinProposedTS.MarshalTo(dAtA[i:]) + n96, err := m.MinProposedTS.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n95 + i += n96 } return i, nil } @@ -12295,27 +12349,27 @@ func (m *TransferLeaseRequest) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.RequestHeader.Size())) - n96, err := m.RequestHeader.MarshalTo(dAtA[i:]) + n97, err := m.RequestHeader.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n96 + i += n97 dAtA[i] = 0x12 i++ i = encodeVarintApi(dAtA, i, uint64(m.Lease.Size())) - n97, err := m.Lease.MarshalTo(dAtA[i:]) + n98, err := m.Lease.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n97 + i += n98 dAtA[i] = 0x1a i++ i = encodeVarintApi(dAtA, i, uint64(m.PrevLease.Size())) - n98, err := m.PrevLease.MarshalTo(dAtA[i:]) + n99, err := m.PrevLease.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n98 + i += n99 return i, nil } @@ -12337,11 +12391,11 @@ func (m *LeaseInfoRequest) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.RequestHeader.Size())) - n99, err := m.RequestHeader.MarshalTo(dAtA[i:]) + n100, err := m.RequestHeader.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n99 + i += n100 return i, nil } @@ -12363,19 +12417,19 @@ func (m *LeaseInfoResponse) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.ResponseHeader.Size())) - n100, err := m.ResponseHeader.MarshalTo(dAtA[i:]) + n101, err := m.ResponseHeader.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n100 + i += n101 dAtA[i] = 0x12 i++ i = encodeVarintApi(dAtA, i, uint64(m.Lease.Size())) - n101, err := m.Lease.MarshalTo(dAtA[i:]) + n102, err := m.Lease.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n101 + i += n102 return i, nil } @@ -12397,11 +12451,11 @@ func (m *RequestLeaseResponse) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.ResponseHeader.Size())) - n102, err := m.ResponseHeader.MarshalTo(dAtA[i:]) + n103, err := m.ResponseHeader.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n102 + i += n103 return i, nil } @@ -12423,11 +12477,11 @@ func (m *ComputeChecksumRequest) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.RequestHeader.Size())) - n103, err := m.RequestHeader.MarshalTo(dAtA[i:]) + n104, err := m.RequestHeader.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n103 + i += n104 if m.Version != 0 { dAtA[i] = 0x10 i++ @@ -12479,19 +12533,19 @@ func (m *ComputeChecksumResponse) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.ResponseHeader.Size())) - n104, err := m.ResponseHeader.MarshalTo(dAtA[i:]) + n105, err := m.ResponseHeader.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n104 + i += n105 dAtA[i] = 0x12 i++ i = encodeVarintApi(dAtA, i, uint64(m.ChecksumID.Size())) - n105, err := m.ChecksumID.MarshalTo(dAtA[i:]) + n106, err := m.ChecksumID.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n105 + i += n106 return i, nil } @@ -12518,58 +12572,58 @@ func (m *ExportStorage) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x12 i++ i = encodeVarintApi(dAtA, i, uint64(m.LocalFile.Size())) - n106, err := m.LocalFile.MarshalTo(dAtA[i:]) + n107, err := m.LocalFile.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n106 + i += n107 dAtA[i] = 0x1a i++ i = encodeVarintApi(dAtA, i, uint64(m.HttpPath.Size())) - n107, err := m.HttpPath.MarshalTo(dAtA[i:]) + n108, err := m.HttpPath.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n107 + i += n108 if m.GoogleCloudConfig != nil { dAtA[i] = 0x22 i++ i = encodeVarintApi(dAtA, i, uint64(m.GoogleCloudConfig.Size())) - n108, err := m.GoogleCloudConfig.MarshalTo(dAtA[i:]) + n109, err := m.GoogleCloudConfig.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n108 + i += n109 } if m.S3Config != nil { dAtA[i] = 0x2a i++ i = encodeVarintApi(dAtA, i, uint64(m.S3Config.Size())) - n109, err := m.S3Config.MarshalTo(dAtA[i:]) + n110, err := m.S3Config.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n109 + i += n110 } if m.AzureConfig != nil { dAtA[i] = 0x32 i++ i = encodeVarintApi(dAtA, i, uint64(m.AzureConfig.Size())) - n110, err := m.AzureConfig.MarshalTo(dAtA[i:]) + n111, err := m.AzureConfig.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n110 + i += n111 } if m.WorkloadConfig != nil { dAtA[i] = 0x3a i++ i = encodeVarintApi(dAtA, i, uint64(m.WorkloadConfig.Size())) - n111, err := m.WorkloadConfig.MarshalTo(dAtA[i:]) + n112, err := m.WorkloadConfig.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n111 + i += n112 } return i, nil } @@ -12862,19 +12916,19 @@ func (m *WriteBatchRequest) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.RequestHeader.Size())) - n112, err := m.RequestHeader.MarshalTo(dAtA[i:]) + n113, err := m.RequestHeader.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n112 + i += n113 dAtA[i] = 0x12 i++ i = encodeVarintApi(dAtA, i, uint64(m.DataSpan.Size())) - n113, err := m.DataSpan.MarshalTo(dAtA[i:]) + n114, err := m.DataSpan.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n113 + i += n114 if len(m.Data) > 0 { dAtA[i] = 0x1a i++ @@ -12902,11 +12956,11 @@ func (m *WriteBatchResponse) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.ResponseHeader.Size())) - n114, err := m.ResponseHeader.MarshalTo(dAtA[i:]) + n115, err := m.ResponseHeader.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n114 + i += n115 return i, nil } @@ -12928,27 +12982,27 @@ func (m *ExportRequest) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.RequestHeader.Size())) - n115, err := m.RequestHeader.MarshalTo(dAtA[i:]) + n116, err := m.RequestHeader.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n115 + i += n116 dAtA[i] = 0x12 i++ i = encodeVarintApi(dAtA, i, uint64(m.Storage.Size())) - n116, err := m.Storage.MarshalTo(dAtA[i:]) + n117, err := m.Storage.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n116 + i += n117 dAtA[i] = 0x1a i++ i = encodeVarintApi(dAtA, i, uint64(m.StartTime.Size())) - n117, err := m.StartTime.MarshalTo(dAtA[i:]) + n118, err := m.StartTime.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n117 + i += n118 if m.MVCCFilter != 0 { dAtA[i] = 0x20 i++ @@ -13043,11 +13097,11 @@ func (m *ExportResponse) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.ResponseHeader.Size())) - n118, err := m.ResponseHeader.MarshalTo(dAtA[i:]) + n119, err := m.ResponseHeader.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n118 + i += n119 if len(m.Files) > 0 { for _, msg := range m.Files { dAtA[i] = 0x12 @@ -13063,11 +13117,11 @@ func (m *ExportResponse) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1a i++ i = encodeVarintApi(dAtA, i, uint64(m.StartTime.Size())) - n119, err := m.StartTime.MarshalTo(dAtA[i:]) + n120, err := m.StartTime.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n119 + i += n120 return i, nil } @@ -13089,11 +13143,11 @@ func (m *ExportResponse_File) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.Span.Size())) - n120, err := m.Span.MarshalTo(dAtA[i:]) + n121, err := m.Span.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n120 + i += n121 if len(m.Path) > 0 { dAtA[i] = 0x12 i++ @@ -13109,11 +13163,11 @@ func (m *ExportResponse_File) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x32 i++ i = encodeVarintApi(dAtA, i, uint64(m.Exported.Size())) - n121, err := m.Exported.MarshalTo(dAtA[i:]) + n122, err := m.Exported.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n121 + i += n122 if len(m.SST) > 0 { dAtA[i] = 0x3a i++ @@ -13141,11 +13195,11 @@ func (m *ImportRequest) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.RequestHeader.Size())) - n122, err := m.RequestHeader.MarshalTo(dAtA[i:]) + n123, err := m.RequestHeader.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n122 + i += n123 if len(m.Files) > 0 { for _, msg := range m.Files { dAtA[i] = 0x12 @@ -13161,11 +13215,11 @@ func (m *ImportRequest) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1a i++ i = encodeVarintApi(dAtA, i, uint64(m.DataSpan.Size())) - n123, err := m.DataSpan.MarshalTo(dAtA[i:]) + n124, err := m.DataSpan.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n123 + i += n124 if len(m.Rekeys) > 0 { for _, msg := range m.Rekeys { dAtA[i] = 0x2a @@ -13181,11 +13235,11 @@ func (m *ImportRequest) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x32 i++ i = encodeVarintApi(dAtA, i, uint64(m.EndTime.Size())) - n124, err := m.EndTime.MarshalTo(dAtA[i:]) + n125, err := m.EndTime.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n124 + i += n125 return i, nil } @@ -13207,11 +13261,11 @@ func (m *ImportRequest_File) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.Dir.Size())) - n125, err := m.Dir.MarshalTo(dAtA[i:]) + n126, err := m.Dir.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n125 + i += n126 if len(m.Path) > 0 { dAtA[i] = 0x12 i++ @@ -13274,19 +13328,19 @@ func (m *ImportResponse) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.ResponseHeader.Size())) - n126, err := m.ResponseHeader.MarshalTo(dAtA[i:]) + n127, err := m.ResponseHeader.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n126 + i += n127 dAtA[i] = 0x1a i++ i = encodeVarintApi(dAtA, i, uint64(m.Imported.Size())) - n127, err := m.Imported.MarshalTo(dAtA[i:]) + n128, err := m.Imported.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n127 + i += n128 return i, nil } @@ -13308,11 +13362,11 @@ func (m *AdminScatterRequest) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.RequestHeader.Size())) - n128, err := m.RequestHeader.MarshalTo(dAtA[i:]) + n129, err := m.RequestHeader.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n128 + i += n129 if m.RandomizeLeases { dAtA[i] = 0x10 i++ @@ -13344,11 +13398,11 @@ func (m *AdminScatterResponse) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.ResponseHeader.Size())) - n129, err := m.ResponseHeader.MarshalTo(dAtA[i:]) + n130, err := m.ResponseHeader.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n129 + i += n130 if len(m.Ranges) > 0 { for _, msg := range m.Ranges { dAtA[i] = 0x12 @@ -13382,11 +13436,11 @@ func (m *AdminScatterResponse_Range) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.Span.Size())) - n130, err := m.Span.MarshalTo(dAtA[i:]) + n131, err := m.Span.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n130 + i += n131 return i, nil } @@ -13408,11 +13462,11 @@ func (m *AddSSTableRequest) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.RequestHeader.Size())) - n131, err := m.RequestHeader.MarshalTo(dAtA[i:]) + n132, err := m.RequestHeader.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n131 + i += n132 if len(m.Data) > 0 { dAtA[i] = 0x12 i++ @@ -13440,11 +13494,11 @@ func (m *AddSSTableResponse) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.ResponseHeader.Size())) - n132, err := m.ResponseHeader.MarshalTo(dAtA[i:]) + n133, err := m.ResponseHeader.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n132 + i += n133 return i, nil } @@ -13466,11 +13520,11 @@ func (m *RefreshRequest) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.RequestHeader.Size())) - n133, err := m.RequestHeader.MarshalTo(dAtA[i:]) + n134, err := m.RequestHeader.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n133 + i += n134 if m.Write { dAtA[i] = 0x10 i++ @@ -13502,11 +13556,11 @@ func (m *RefreshResponse) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.ResponseHeader.Size())) - n134, err := m.ResponseHeader.MarshalTo(dAtA[i:]) + n135, err := m.ResponseHeader.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n134 + i += n135 return i, nil } @@ -13528,11 +13582,11 @@ func (m *RefreshRangeRequest) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.RequestHeader.Size())) - n135, err := m.RequestHeader.MarshalTo(dAtA[i:]) + n136, err := m.RequestHeader.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n135 + i += n136 if m.Write { dAtA[i] = 0x10 i++ @@ -13564,11 +13618,11 @@ func (m *RefreshRangeResponse) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.ResponseHeader.Size())) - n136, err := m.ResponseHeader.MarshalTo(dAtA[i:]) + n137, err := m.ResponseHeader.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n136 + i += n137 return i, nil } @@ -13590,28 +13644,28 @@ func (m *SubsumeRequest) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.RequestHeader.Size())) - n137, err := m.RequestHeader.MarshalTo(dAtA[i:]) + n138, err := m.RequestHeader.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n137 + i += n138 dAtA[i] = 0x12 i++ i = encodeVarintApi(dAtA, i, uint64(m.LeftDesc.Size())) - n138, err := m.LeftDesc.MarshalTo(dAtA[i:]) + n139, err := m.LeftDesc.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n138 + i += n139 if m.RightDesc != nil { dAtA[i] = 0x1a i++ i = encodeVarintApi(dAtA, i, uint64(m.RightDesc.Size())) - n139, err := m.RightDesc.MarshalTo(dAtA[i:]) + n140, err := m.RightDesc.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n139 + i += n140 } return i, nil } @@ -13634,19 +13688,19 @@ func (m *SubsumeResponse) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.ResponseHeader.Size())) - n140, err := m.ResponseHeader.MarshalTo(dAtA[i:]) + n141, err := m.ResponseHeader.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n140 + i += n141 dAtA[i] = 0x1a i++ i = encodeVarintApi(dAtA, i, uint64(m.MVCCStats.Size())) - n141, err := m.MVCCStats.MarshalTo(dAtA[i:]) + n142, err := m.MVCCStats.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n141 + i += n142 if m.LeaseAppliedIndex != 0 { dAtA[i] = 0x20 i++ @@ -13655,11 +13709,11 @@ func (m *SubsumeResponse) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2a i++ i = encodeVarintApi(dAtA, i, uint64(m.FreezeStart.Size())) - n142, err := m.FreezeStart.MarshalTo(dAtA[i:]) + n143, err := m.FreezeStart.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n142 + i += n143 return i, nil } @@ -13681,11 +13735,11 @@ func (m *RangeStatsRequest) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.RequestHeader.Size())) - n143, err := m.RequestHeader.MarshalTo(dAtA[i:]) + n144, err := m.RequestHeader.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n143 + i += n144 return i, nil } @@ -13707,19 +13761,19 @@ func (m *RangeStatsResponse) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.ResponseHeader.Size())) - n144, err := m.ResponseHeader.MarshalTo(dAtA[i:]) + n145, err := m.ResponseHeader.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n144 + i += n145 dAtA[i] = 0x12 i++ i = encodeVarintApi(dAtA, i, uint64(m.MVCCStats.Size())) - n145, err := m.MVCCStats.MarshalTo(dAtA[i:]) + n146, err := m.MVCCStats.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n145 + i += n146 if m.QueriesPerSecond != 0 { dAtA[i] = 0x19 i++ @@ -13745,11 +13799,11 @@ func (m *RequestUnion) MarshalTo(dAtA []byte) (int, error) { var l int _ = l if m.Value != nil { - nn146, err := m.Value.MarshalTo(dAtA[i:]) + nn147, err := m.Value.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += nn146 + i += nn147 } return i, nil } @@ -13760,11 +13814,11 @@ func (m *RequestUnion_Get) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.Get.Size())) - n147, err := m.Get.MarshalTo(dAtA[i:]) + n148, err := m.Get.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n147 + i += n148 } return i, nil } @@ -13774,11 +13828,11 @@ func (m *RequestUnion_Put) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x12 i++ i = encodeVarintApi(dAtA, i, uint64(m.Put.Size())) - n148, err := m.Put.MarshalTo(dAtA[i:]) + n149, err := m.Put.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n148 + i += n149 } return i, nil } @@ -13788,11 +13842,11 @@ func (m *RequestUnion_ConditionalPut) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1a i++ i = encodeVarintApi(dAtA, i, uint64(m.ConditionalPut.Size())) - n149, err := m.ConditionalPut.MarshalTo(dAtA[i:]) + n150, err := m.ConditionalPut.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n149 + i += n150 } return i, nil } @@ -13802,11 +13856,11 @@ func (m *RequestUnion_Increment) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x22 i++ i = encodeVarintApi(dAtA, i, uint64(m.Increment.Size())) - n150, err := m.Increment.MarshalTo(dAtA[i:]) + n151, err := m.Increment.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n150 + i += n151 } return i, nil } @@ -13816,11 +13870,11 @@ func (m *RequestUnion_Delete) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2a i++ i = encodeVarintApi(dAtA, i, uint64(m.Delete.Size())) - n151, err := m.Delete.MarshalTo(dAtA[i:]) + n152, err := m.Delete.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n151 + i += n152 } return i, nil } @@ -13830,11 +13884,11 @@ func (m *RequestUnion_DeleteRange) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x32 i++ i = encodeVarintApi(dAtA, i, uint64(m.DeleteRange.Size())) - n152, err := m.DeleteRange.MarshalTo(dAtA[i:]) + n153, err := m.DeleteRange.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n152 + i += n153 } return i, nil } @@ -13844,11 +13898,11 @@ func (m *RequestUnion_Scan) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x3a i++ i = encodeVarintApi(dAtA, i, uint64(m.Scan.Size())) - n153, err := m.Scan.MarshalTo(dAtA[i:]) + n154, err := m.Scan.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n153 + i += n154 } return i, nil } @@ -13858,11 +13912,11 @@ func (m *RequestUnion_BeginTransaction) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x42 i++ i = encodeVarintApi(dAtA, i, uint64(m.BeginTransaction.Size())) - n154, err := m.BeginTransaction.MarshalTo(dAtA[i:]) + n155, err := m.BeginTransaction.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n154 + i += n155 } return i, nil } @@ -13872,11 +13926,11 @@ func (m *RequestUnion_EndTransaction) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x4a i++ i = encodeVarintApi(dAtA, i, uint64(m.EndTransaction.Size())) - n155, err := m.EndTransaction.MarshalTo(dAtA[i:]) + n156, err := m.EndTransaction.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n155 + i += n156 } return i, nil } @@ -13886,11 +13940,11 @@ func (m *RequestUnion_AdminSplit) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x52 i++ i = encodeVarintApi(dAtA, i, uint64(m.AdminSplit.Size())) - n156, err := m.AdminSplit.MarshalTo(dAtA[i:]) + n157, err := m.AdminSplit.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n156 + i += n157 } return i, nil } @@ -13900,11 +13954,11 @@ func (m *RequestUnion_AdminMerge) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x5a i++ i = encodeVarintApi(dAtA, i, uint64(m.AdminMerge.Size())) - n157, err := m.AdminMerge.MarshalTo(dAtA[i:]) + n158, err := m.AdminMerge.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n157 + i += n158 } return i, nil } @@ -13914,11 +13968,11 @@ func (m *RequestUnion_HeartbeatTxn) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x62 i++ i = encodeVarintApi(dAtA, i, uint64(m.HeartbeatTxn.Size())) - n158, err := m.HeartbeatTxn.MarshalTo(dAtA[i:]) + n159, err := m.HeartbeatTxn.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n158 + i += n159 } return i, nil } @@ -13928,11 +13982,11 @@ func (m *RequestUnion_Gc) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x6a i++ i = encodeVarintApi(dAtA, i, uint64(m.Gc.Size())) - n159, err := m.Gc.MarshalTo(dAtA[i:]) + n160, err := m.Gc.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n159 + i += n160 } return i, nil } @@ -13942,11 +13996,11 @@ func (m *RequestUnion_PushTxn) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x72 i++ i = encodeVarintApi(dAtA, i, uint64(m.PushTxn.Size())) - n160, err := m.PushTxn.MarshalTo(dAtA[i:]) + n161, err := m.PushTxn.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n160 + i += n161 } return i, nil } @@ -13958,11 +14012,11 @@ func (m *RequestUnion_ResolveIntent) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.ResolveIntent.Size())) - n161, err := m.ResolveIntent.MarshalTo(dAtA[i:]) + n162, err := m.ResolveIntent.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n161 + i += n162 } return i, nil } @@ -13974,11 +14028,11 @@ func (m *RequestUnion_ResolveIntentRange) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.ResolveIntentRange.Size())) - n162, err := m.ResolveIntentRange.MarshalTo(dAtA[i:]) + n163, err := m.ResolveIntentRange.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n162 + i += n163 } return i, nil } @@ -13990,11 +14044,11 @@ func (m *RequestUnion_Merge) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.Merge.Size())) - n163, err := m.Merge.MarshalTo(dAtA[i:]) + n164, err := m.Merge.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n163 + i += n164 } return i, nil } @@ -14006,11 +14060,11 @@ func (m *RequestUnion_TruncateLog) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.TruncateLog.Size())) - n164, err := m.TruncateLog.MarshalTo(dAtA[i:]) + n165, err := m.TruncateLog.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n164 + i += n165 } return i, nil } @@ -14022,11 +14076,11 @@ func (m *RequestUnion_RequestLease) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.RequestLease.Size())) - n165, err := m.RequestLease.MarshalTo(dAtA[i:]) + n166, err := m.RequestLease.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n165 + i += n166 } return i, nil } @@ -14038,11 +14092,11 @@ func (m *RequestUnion_ReverseScan) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.ReverseScan.Size())) - n166, err := m.ReverseScan.MarshalTo(dAtA[i:]) + n167, err := m.ReverseScan.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n166 + i += n167 } return i, nil } @@ -14054,11 +14108,11 @@ func (m *RequestUnion_ComputeChecksum) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.ComputeChecksum.Size())) - n167, err := m.ComputeChecksum.MarshalTo(dAtA[i:]) + n168, err := m.ComputeChecksum.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n167 + i += n168 } return i, nil } @@ -14070,11 +14124,11 @@ func (m *RequestUnion_CheckConsistency) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.CheckConsistency.Size())) - n168, err := m.CheckConsistency.MarshalTo(dAtA[i:]) + n169, err := m.CheckConsistency.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n168 + i += n169 } return i, nil } @@ -14086,11 +14140,11 @@ func (m *RequestUnion_InitPut) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.InitPut.Size())) - n169, err := m.InitPut.MarshalTo(dAtA[i:]) + n170, err := m.InitPut.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n169 + i += n170 } return i, nil } @@ -14102,11 +14156,11 @@ func (m *RequestUnion_TransferLease) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.TransferLease.Size())) - n170, err := m.TransferLease.MarshalTo(dAtA[i:]) + n171, err := m.TransferLease.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n170 + i += n171 } return i, nil } @@ -14118,11 +14172,11 @@ func (m *RequestUnion_AdminTransferLease) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.AdminTransferLease.Size())) - n171, err := m.AdminTransferLease.MarshalTo(dAtA[i:]) + n172, err := m.AdminTransferLease.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n171 + i += n172 } return i, nil } @@ -14134,11 +14188,11 @@ func (m *RequestUnion_LeaseInfo) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.LeaseInfo.Size())) - n172, err := m.LeaseInfo.MarshalTo(dAtA[i:]) + n173, err := m.LeaseInfo.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n172 + i += n173 } return i, nil } @@ -14150,11 +14204,11 @@ func (m *RequestUnion_WriteBatch) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.WriteBatch.Size())) - n173, err := m.WriteBatch.MarshalTo(dAtA[i:]) + n174, err := m.WriteBatch.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n173 + i += n174 } return i, nil } @@ -14166,11 +14220,11 @@ func (m *RequestUnion_Export) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.Export.Size())) - n174, err := m.Export.MarshalTo(dAtA[i:]) + n175, err := m.Export.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n174 + i += n175 } return i, nil } @@ -14182,11 +14236,11 @@ func (m *RequestUnion_QueryTxn) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.QueryTxn.Size())) - n175, err := m.QueryTxn.MarshalTo(dAtA[i:]) + n176, err := m.QueryTxn.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n175 + i += n176 } return i, nil } @@ -14198,11 +14252,11 @@ func (m *RequestUnion_Import) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.Import.Size())) - n176, err := m.Import.MarshalTo(dAtA[i:]) + n177, err := m.Import.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n176 + i += n177 } return i, nil } @@ -14214,11 +14268,11 @@ func (m *RequestUnion_AdminChangeReplicas) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.AdminChangeReplicas.Size())) - n177, err := m.AdminChangeReplicas.MarshalTo(dAtA[i:]) + n178, err := m.AdminChangeReplicas.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n177 + i += n178 } return i, nil } @@ -14230,11 +14284,11 @@ func (m *RequestUnion_AdminScatter) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.AdminScatter.Size())) - n178, err := m.AdminScatter.MarshalTo(dAtA[i:]) + n179, err := m.AdminScatter.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n178 + i += n179 } return i, nil } @@ -14246,11 +14300,11 @@ func (m *RequestUnion_AddSstable) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.AddSstable.Size())) - n179, err := m.AddSstable.MarshalTo(dAtA[i:]) + n180, err := m.AddSstable.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n179 + i += n180 } return i, nil } @@ -14262,11 +14316,11 @@ func (m *RequestUnion_ClearRange) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.ClearRange.Size())) - n180, err := m.ClearRange.MarshalTo(dAtA[i:]) + n181, err := m.ClearRange.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n180 + i += n181 } return i, nil } @@ -14278,11 +14332,11 @@ func (m *RequestUnion_RecomputeStats) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.RecomputeStats.Size())) - n181, err := m.RecomputeStats.MarshalTo(dAtA[i:]) + n182, err := m.RecomputeStats.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n181 + i += n182 } return i, nil } @@ -14294,11 +14348,11 @@ func (m *RequestUnion_Refresh) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.Refresh.Size())) - n182, err := m.Refresh.MarshalTo(dAtA[i:]) + n183, err := m.Refresh.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n182 + i += n183 } return i, nil } @@ -14310,11 +14364,11 @@ func (m *RequestUnion_RefreshRange) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.RefreshRange.Size())) - n183, err := m.RefreshRange.MarshalTo(dAtA[i:]) + n184, err := m.RefreshRange.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n183 + i += n184 } return i, nil } @@ -14326,11 +14380,11 @@ func (m *RequestUnion_QueryIntent) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.QueryIntent.Size())) - n184, err := m.QueryIntent.MarshalTo(dAtA[i:]) + n185, err := m.QueryIntent.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n184 + i += n185 } return i, nil } @@ -14342,11 +14396,11 @@ func (m *RequestUnion_Subsume) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.Subsume.Size())) - n185, err := m.Subsume.MarshalTo(dAtA[i:]) + n186, err := m.Subsume.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n185 + i += n186 } return i, nil } @@ -14358,11 +14412,11 @@ func (m *RequestUnion_RangeStats) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.RangeStats.Size())) - n186, err := m.RangeStats.MarshalTo(dAtA[i:]) + n187, err := m.RangeStats.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n186 + i += n187 } return i, nil } @@ -14374,11 +14428,11 @@ func (m *RequestUnion_AdminRelocateRange) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.AdminRelocateRange.Size())) - n187, err := m.AdminRelocateRange.MarshalTo(dAtA[i:]) + n188, err := m.AdminRelocateRange.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n187 + i += n188 } return i, nil } @@ -14390,11 +14444,11 @@ func (m *RequestUnion_RecoverTxn) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.RecoverTxn.Size())) - n188, err := m.RecoverTxn.MarshalTo(dAtA[i:]) + n189, err := m.RecoverTxn.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n188 + i += n189 } return i, nil } @@ -14414,11 +14468,11 @@ func (m *ResponseUnion) MarshalTo(dAtA []byte) (int, error) { var l int _ = l if m.Value != nil { - nn189, err := m.Value.MarshalTo(dAtA[i:]) + nn190, err := m.Value.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += nn189 + i += nn190 } return i, nil } @@ -14429,11 +14483,11 @@ func (m *ResponseUnion_Get) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.Get.Size())) - n190, err := m.Get.MarshalTo(dAtA[i:]) + n191, err := m.Get.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n190 + i += n191 } return i, nil } @@ -14443,11 +14497,11 @@ func (m *ResponseUnion_Put) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x12 i++ i = encodeVarintApi(dAtA, i, uint64(m.Put.Size())) - n191, err := m.Put.MarshalTo(dAtA[i:]) + n192, err := m.Put.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n191 + i += n192 } return i, nil } @@ -14457,11 +14511,11 @@ func (m *ResponseUnion_ConditionalPut) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1a i++ i = encodeVarintApi(dAtA, i, uint64(m.ConditionalPut.Size())) - n192, err := m.ConditionalPut.MarshalTo(dAtA[i:]) + n193, err := m.ConditionalPut.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n192 + i += n193 } return i, nil } @@ -14471,11 +14525,11 @@ func (m *ResponseUnion_Increment) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x22 i++ i = encodeVarintApi(dAtA, i, uint64(m.Increment.Size())) - n193, err := m.Increment.MarshalTo(dAtA[i:]) + n194, err := m.Increment.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n193 + i += n194 } return i, nil } @@ -14485,11 +14539,11 @@ func (m *ResponseUnion_Delete) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2a i++ i = encodeVarintApi(dAtA, i, uint64(m.Delete.Size())) - n194, err := m.Delete.MarshalTo(dAtA[i:]) + n195, err := m.Delete.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n194 + i += n195 } return i, nil } @@ -14499,11 +14553,11 @@ func (m *ResponseUnion_DeleteRange) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x32 i++ i = encodeVarintApi(dAtA, i, uint64(m.DeleteRange.Size())) - n195, err := m.DeleteRange.MarshalTo(dAtA[i:]) + n196, err := m.DeleteRange.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n195 + i += n196 } return i, nil } @@ -14513,11 +14567,11 @@ func (m *ResponseUnion_Scan) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x3a i++ i = encodeVarintApi(dAtA, i, uint64(m.Scan.Size())) - n196, err := m.Scan.MarshalTo(dAtA[i:]) + n197, err := m.Scan.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n196 + i += n197 } return i, nil } @@ -14527,11 +14581,11 @@ func (m *ResponseUnion_BeginTransaction) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x42 i++ i = encodeVarintApi(dAtA, i, uint64(m.BeginTransaction.Size())) - n197, err := m.BeginTransaction.MarshalTo(dAtA[i:]) + n198, err := m.BeginTransaction.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n197 + i += n198 } return i, nil } @@ -14541,11 +14595,11 @@ func (m *ResponseUnion_EndTransaction) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x4a i++ i = encodeVarintApi(dAtA, i, uint64(m.EndTransaction.Size())) - n198, err := m.EndTransaction.MarshalTo(dAtA[i:]) + n199, err := m.EndTransaction.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n198 + i += n199 } return i, nil } @@ -14555,11 +14609,11 @@ func (m *ResponseUnion_AdminSplit) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x52 i++ i = encodeVarintApi(dAtA, i, uint64(m.AdminSplit.Size())) - n199, err := m.AdminSplit.MarshalTo(dAtA[i:]) + n200, err := m.AdminSplit.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n199 + i += n200 } return i, nil } @@ -14569,11 +14623,11 @@ func (m *ResponseUnion_AdminMerge) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x5a i++ i = encodeVarintApi(dAtA, i, uint64(m.AdminMerge.Size())) - n200, err := m.AdminMerge.MarshalTo(dAtA[i:]) + n201, err := m.AdminMerge.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n200 + i += n201 } return i, nil } @@ -14583,11 +14637,11 @@ func (m *ResponseUnion_HeartbeatTxn) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x62 i++ i = encodeVarintApi(dAtA, i, uint64(m.HeartbeatTxn.Size())) - n201, err := m.HeartbeatTxn.MarshalTo(dAtA[i:]) + n202, err := m.HeartbeatTxn.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n201 + i += n202 } return i, nil } @@ -14597,11 +14651,11 @@ func (m *ResponseUnion_Gc) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x6a i++ i = encodeVarintApi(dAtA, i, uint64(m.Gc.Size())) - n202, err := m.Gc.MarshalTo(dAtA[i:]) + n203, err := m.Gc.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n202 + i += n203 } return i, nil } @@ -14611,11 +14665,11 @@ func (m *ResponseUnion_PushTxn) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x72 i++ i = encodeVarintApi(dAtA, i, uint64(m.PushTxn.Size())) - n203, err := m.PushTxn.MarshalTo(dAtA[i:]) + n204, err := m.PushTxn.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n203 + i += n204 } return i, nil } @@ -14627,11 +14681,11 @@ func (m *ResponseUnion_ResolveIntent) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.ResolveIntent.Size())) - n204, err := m.ResolveIntent.MarshalTo(dAtA[i:]) + n205, err := m.ResolveIntent.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n204 + i += n205 } return i, nil } @@ -14643,11 +14697,11 @@ func (m *ResponseUnion_ResolveIntentRange) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.ResolveIntentRange.Size())) - n205, err := m.ResolveIntentRange.MarshalTo(dAtA[i:]) + n206, err := m.ResolveIntentRange.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n205 + i += n206 } return i, nil } @@ -14659,11 +14713,11 @@ func (m *ResponseUnion_Merge) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.Merge.Size())) - n206, err := m.Merge.MarshalTo(dAtA[i:]) + n207, err := m.Merge.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n206 + i += n207 } return i, nil } @@ -14675,11 +14729,11 @@ func (m *ResponseUnion_TruncateLog) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.TruncateLog.Size())) - n207, err := m.TruncateLog.MarshalTo(dAtA[i:]) + n208, err := m.TruncateLog.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n207 + i += n208 } return i, nil } @@ -14691,11 +14745,11 @@ func (m *ResponseUnion_RequestLease) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.RequestLease.Size())) - n208, err := m.RequestLease.MarshalTo(dAtA[i:]) + n209, err := m.RequestLease.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n208 + i += n209 } return i, nil } @@ -14707,11 +14761,11 @@ func (m *ResponseUnion_ReverseScan) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.ReverseScan.Size())) - n209, err := m.ReverseScan.MarshalTo(dAtA[i:]) + n210, err := m.ReverseScan.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n209 + i += n210 } return i, nil } @@ -14723,11 +14777,11 @@ func (m *ResponseUnion_ComputeChecksum) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.ComputeChecksum.Size())) - n210, err := m.ComputeChecksum.MarshalTo(dAtA[i:]) + n211, err := m.ComputeChecksum.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n210 + i += n211 } return i, nil } @@ -14739,11 +14793,11 @@ func (m *ResponseUnion_CheckConsistency) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.CheckConsistency.Size())) - n211, err := m.CheckConsistency.MarshalTo(dAtA[i:]) + n212, err := m.CheckConsistency.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n211 + i += n212 } return i, nil } @@ -14755,11 +14809,11 @@ func (m *ResponseUnion_InitPut) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.InitPut.Size())) - n212, err := m.InitPut.MarshalTo(dAtA[i:]) + n213, err := m.InitPut.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n212 + i += n213 } return i, nil } @@ -14771,11 +14825,11 @@ func (m *ResponseUnion_AdminTransferLease) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.AdminTransferLease.Size())) - n213, err := m.AdminTransferLease.MarshalTo(dAtA[i:]) + n214, err := m.AdminTransferLease.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n213 + i += n214 } return i, nil } @@ -14787,11 +14841,11 @@ func (m *ResponseUnion_LeaseInfo) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.LeaseInfo.Size())) - n214, err := m.LeaseInfo.MarshalTo(dAtA[i:]) + n215, err := m.LeaseInfo.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n214 + i += n215 } return i, nil } @@ -14803,11 +14857,11 @@ func (m *ResponseUnion_WriteBatch) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.WriteBatch.Size())) - n215, err := m.WriteBatch.MarshalTo(dAtA[i:]) + n216, err := m.WriteBatch.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n215 + i += n216 } return i, nil } @@ -14819,11 +14873,11 @@ func (m *ResponseUnion_Export) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.Export.Size())) - n216, err := m.Export.MarshalTo(dAtA[i:]) + n217, err := m.Export.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n216 + i += n217 } return i, nil } @@ -14835,11 +14889,11 @@ func (m *ResponseUnion_QueryTxn) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.QueryTxn.Size())) - n217, err := m.QueryTxn.MarshalTo(dAtA[i:]) + n218, err := m.QueryTxn.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n217 + i += n218 } return i, nil } @@ -14851,11 +14905,11 @@ func (m *ResponseUnion_Import) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.Import.Size())) - n218, err := m.Import.MarshalTo(dAtA[i:]) + n219, err := m.Import.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n218 + i += n219 } return i, nil } @@ -14867,11 +14921,11 @@ func (m *ResponseUnion_AdminChangeReplicas) MarshalTo(dAtA []byte) (int, error) dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.AdminChangeReplicas.Size())) - n219, err := m.AdminChangeReplicas.MarshalTo(dAtA[i:]) + n220, err := m.AdminChangeReplicas.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n219 + i += n220 } return i, nil } @@ -14883,11 +14937,11 @@ func (m *ResponseUnion_AdminScatter) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.AdminScatter.Size())) - n220, err := m.AdminScatter.MarshalTo(dAtA[i:]) + n221, err := m.AdminScatter.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n220 + i += n221 } return i, nil } @@ -14899,11 +14953,11 @@ func (m *ResponseUnion_AddSstable) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.AddSstable.Size())) - n221, err := m.AddSstable.MarshalTo(dAtA[i:]) + n222, err := m.AddSstable.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n221 + i += n222 } return i, nil } @@ -14915,11 +14969,11 @@ func (m *ResponseUnion_ClearRange) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.ClearRange.Size())) - n222, err := m.ClearRange.MarshalTo(dAtA[i:]) + n223, err := m.ClearRange.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n222 + i += n223 } return i, nil } @@ -14931,11 +14985,11 @@ func (m *ResponseUnion_RecomputeStats) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.RecomputeStats.Size())) - n223, err := m.RecomputeStats.MarshalTo(dAtA[i:]) + n224, err := m.RecomputeStats.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n223 + i += n224 } return i, nil } @@ -14947,11 +15001,11 @@ func (m *ResponseUnion_Refresh) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.Refresh.Size())) - n224, err := m.Refresh.MarshalTo(dAtA[i:]) + n225, err := m.Refresh.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n224 + i += n225 } return i, nil } @@ -14963,11 +15017,11 @@ func (m *ResponseUnion_RefreshRange) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.RefreshRange.Size())) - n225, err := m.RefreshRange.MarshalTo(dAtA[i:]) + n226, err := m.RefreshRange.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n225 + i += n226 } return i, nil } @@ -14979,11 +15033,11 @@ func (m *ResponseUnion_QueryIntent) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.QueryIntent.Size())) - n226, err := m.QueryIntent.MarshalTo(dAtA[i:]) + n227, err := m.QueryIntent.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n226 + i += n227 } return i, nil } @@ -14995,11 +15049,11 @@ func (m *ResponseUnion_Subsume) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.Subsume.Size())) - n227, err := m.Subsume.MarshalTo(dAtA[i:]) + n228, err := m.Subsume.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n227 + i += n228 } return i, nil } @@ -15011,11 +15065,11 @@ func (m *ResponseUnion_RangeStats) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.RangeStats.Size())) - n228, err := m.RangeStats.MarshalTo(dAtA[i:]) + n229, err := m.RangeStats.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n228 + i += n229 } return i, nil } @@ -15027,11 +15081,11 @@ func (m *ResponseUnion_AdminRelocateRange) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.AdminRelocateRange.Size())) - n229, err := m.AdminRelocateRange.MarshalTo(dAtA[i:]) + n230, err := m.AdminRelocateRange.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n229 + i += n230 } return i, nil } @@ -15043,11 +15097,11 @@ func (m *ResponseUnion_RecoverTxn) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.RecoverTxn.Size())) - n230, err := m.RecoverTxn.MarshalTo(dAtA[i:]) + n231, err := m.RecoverTxn.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n230 + i += n231 } return i, nil } @@ -15069,19 +15123,19 @@ func (m *Header) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.Timestamp.Size())) - n231, err := m.Timestamp.MarshalTo(dAtA[i:]) + n232, err := m.Timestamp.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n231 + i += n232 dAtA[i] = 0x12 i++ i = encodeVarintApi(dAtA, i, uint64(m.Replica.Size())) - n232, err := m.Replica.MarshalTo(dAtA[i:]) + n233, err := m.Replica.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n232 + i += n233 if m.RangeID != 0 { dAtA[i] = 0x18 i++ @@ -15097,11 +15151,11 @@ func (m *Header) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2a i++ i = encodeVarintApi(dAtA, i, uint64(m.Txn.Size())) - n233, err := m.Txn.MarshalTo(dAtA[i:]) + n234, err := m.Txn.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n233 + i += n234 } if m.ReadConsistency != 0 { dAtA[i] = 0x30 @@ -15142,11 +15196,11 @@ func (m *Header) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x62 i++ i = encodeVarintApi(dAtA, i, uint64(m.ScanOptions.Size())) - n234, err := m.ScanOptions.MarshalTo(dAtA[i:]) + n235, err := m.ScanOptions.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n234 + i += n235 } if m.AsyncConsensus { dAtA[i] = 0x68 @@ -15179,11 +15233,11 @@ func (m *BatchRequest) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.Header.Size())) - n235, err := m.Header.MarshalTo(dAtA[i:]) + n236, err := m.Header.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n235 + i += n236 if len(m.Requests) > 0 { for _, msg := range m.Requests { dAtA[i] = 0x12 @@ -15217,11 +15271,11 @@ func (m *BatchResponse) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.BatchResponse_Header.Size())) - n236, err := m.BatchResponse_Header.MarshalTo(dAtA[i:]) + n237, err := m.BatchResponse_Header.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n236 + i += n237 if len(m.Responses) > 0 { for _, msg := range m.Responses { dAtA[i] = 0x12 @@ -15256,38 +15310,38 @@ func (m *BatchResponse_Header) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.Error.Size())) - n237, err := m.Error.MarshalTo(dAtA[i:]) + n238, err := m.Error.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n237 + i += n238 } dAtA[i] = 0x12 i++ i = encodeVarintApi(dAtA, i, uint64(m.Timestamp.Size())) - n238, err := m.Timestamp.MarshalTo(dAtA[i:]) + n239, err := m.Timestamp.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n238 + i += n239 if m.Txn != nil { dAtA[i] = 0x1a i++ i = encodeVarintApi(dAtA, i, uint64(m.Txn.Size())) - n239, err := m.Txn.MarshalTo(dAtA[i:]) + n240, err := m.Txn.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n239 + i += n240 } dAtA[i] = 0x2a i++ i = encodeVarintApi(dAtA, i, uint64(m.Now.Size())) - n240, err := m.Now.MarshalTo(dAtA[i:]) + n241, err := m.Now.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n240 + i += n241 if len(m.CollectedSpans) > 0 { for _, msg := range m.CollectedSpans { dAtA[i] = 0x32 @@ -15321,19 +15375,19 @@ func (m *RangeFeedRequest) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.Header.Size())) - n241, err := m.Header.MarshalTo(dAtA[i:]) + n242, err := m.Header.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n241 + i += n242 dAtA[i] = 0x12 i++ i = encodeVarintApi(dAtA, i, uint64(m.Span.Size())) - n242, err := m.Span.MarshalTo(dAtA[i:]) + n243, err := m.Span.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n242 + i += n243 return i, nil } @@ -15361,11 +15415,11 @@ func (m *RangeFeedValue) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x12 i++ i = encodeVarintApi(dAtA, i, uint64(m.Value.Size())) - n243, err := m.Value.MarshalTo(dAtA[i:]) + n244, err := m.Value.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n243 + i += n244 return i, nil } @@ -15387,19 +15441,19 @@ func (m *RangeFeedCheckpoint) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.Span.Size())) - n244, err := m.Span.MarshalTo(dAtA[i:]) + n245, err := m.Span.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n244 + i += n245 dAtA[i] = 0x12 i++ i = encodeVarintApi(dAtA, i, uint64(m.ResolvedTS.Size())) - n245, err := m.ResolvedTS.MarshalTo(dAtA[i:]) + n246, err := m.ResolvedTS.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n245 + i += n246 return i, nil } @@ -15421,11 +15475,11 @@ func (m *RangeFeedError) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.Error.Size())) - n246, err := m.Error.MarshalTo(dAtA[i:]) + n247, err := m.Error.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n246 + i += n247 return i, nil } @@ -15448,31 +15502,31 @@ func (m *RangeFeedEvent) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.Val.Size())) - n247, err := m.Val.MarshalTo(dAtA[i:]) + n248, err := m.Val.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n247 + i += n248 } if m.Checkpoint != nil { dAtA[i] = 0x12 i++ i = encodeVarintApi(dAtA, i, uint64(m.Checkpoint.Size())) - n248, err := m.Checkpoint.MarshalTo(dAtA[i:]) + n249, err := m.Checkpoint.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n248 + i += n249 } if m.Error != nil { dAtA[i] = 0x1a i++ i = encodeVarintApi(dAtA, i, uint64(m.Error.Size())) - n249, err := m.Error.MarshalTo(dAtA[i:]) + n250, err := m.Error.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n249 + i += n250 } return i, nil } @@ -16022,6 +16076,12 @@ func (m *EndTransactionRequest) Size() (n int) { if m.Poison { n += 2 } + if len(m.InFlightWrites) > 0 { + for _, e := range m.InFlightWrites { + l = e.Size() + n += 2 + l + sovApi(uint64(l)) + } + } return n } @@ -16036,6 +16096,8 @@ func (m *EndTransactionResponse) Size() (n int) { if m.OnePhaseCommit { n += 2 } + l = m.StagingTimestamp.Size() + n += 1 + l + sovApi(uint64(l)) return n } @@ -22167,6 +22229,37 @@ func (m *EndTransactionRequest) Unmarshal(dAtA []byte) error { } } m.Poison = bool(v != 0) + case 17: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field InFlightWrites", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowApi + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthApi + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.InFlightWrites = append(m.InFlightWrites, SequencedWrite{}) + if err := m.InFlightWrites[len(m.InFlightWrites)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipApi(dAtA[iNdEx:]) @@ -22267,6 +22360,36 @@ func (m *EndTransactionResponse) Unmarshal(dAtA []byte) error { } } m.OnePhaseCommit = bool(v != 0) + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field StagingTimestamp", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowApi + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthApi + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if err := m.StagingTimestamp.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipApi(dAtA[iNdEx:]) @@ -34970,421 +35093,425 @@ var ( ErrIntOverflowApi = fmt.Errorf("proto: integer overflow") ) -func init() { proto.RegisterFile("roachpb/api.proto", fileDescriptor_api_c24af62f3f876bfa) } +func init() { proto.RegisterFile("roachpb/api.proto", fileDescriptor_api_08e22abf3c79051e) } -var fileDescriptor_api_c24af62f3f876bfa = []byte{ - // 6600 bytes of a gzipped FileDescriptorProto +var fileDescriptor_api_08e22abf3c79051e = []byte{ + // 6659 bytes of a gzipped FileDescriptorProto 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xe4, 0x7d, 0x5b, 0x6c, 0x23, 0xc9, - 0x75, 0xb6, 0x9a, 0x37, 0x91, 0x87, 0x17, 0x51, 0x25, 0xcd, 0x0c, 0x47, 0x3b, 0x3b, 0xd2, 0x70, - 0xae, 0x3b, 0xbb, 0xab, 0xf1, 0x68, 0xbc, 0xb6, 0xff, 0x5d, 0x7b, 0x6d, 0x91, 0xe2, 0x0c, 0x39, - 0x1a, 0x5d, 0xb6, 0x49, 0xcd, 0x7a, 0xd6, 0xff, 0xfe, 0xed, 0x56, 0x77, 0x89, 0x6a, 0x0f, 0xd9, - 0xcd, 0xe9, 0x6e, 0x8e, 0xa4, 0x01, 0x7e, 0x04, 0x48, 0x1e, 0x1c, 0x18, 0x86, 0x11, 0x20, 0x41, - 0x10, 0x38, 0x09, 0x62, 0xc0, 0x01, 0x02, 0x24, 0x88, 0x11, 0x07, 0x06, 0x82, 0x04, 0x09, 0xfc, - 0x90, 0x3c, 0x2c, 0x0c, 0x3f, 0x38, 0x41, 0x1c, 0x18, 0x09, 0x20, 0x24, 0x4a, 0x00, 0x1b, 0x79, - 0xcd, 0x43, 0x80, 0x7d, 0x48, 0x82, 0xba, 0xf4, 0x85, 0x64, 0xf3, 0x22, 0xb9, 0x17, 0xd9, 0x20, - 0x4f, 0x62, 0x9d, 0xaa, 0x73, 0xba, 0x2e, 0xa7, 0x4e, 0x9d, 0xaf, 0xea, 0x54, 0x09, 0x66, 0x4d, - 0x43, 0x56, 0xf6, 0x3b, 0xbb, 0x77, 0xe4, 0x8e, 0xb6, 0xdc, 0x31, 0x0d, 0xdb, 0x40, 0xb3, 0x8a, - 0xa1, 0x3c, 0xa5, 0xe4, 0x65, 0x9e, 0xb9, 0x80, 0x9c, 0x52, 0xaa, 0x6c, 0xcb, 0xac, 0xd8, 0xc2, - 0xbc, 0x43, 0xc3, 0xa6, 0x69, 0x98, 0x16, 0xa7, 0x9e, 0x77, 0xa8, 0x6d, 0x6c, 0xcb, 0xbe, 0xd2, - 0x45, 0xcb, 0x36, 0x4c, 0xb9, 0x89, 0xef, 0x60, 0xbd, 0xa9, 0xe9, 0xce, 0x1f, 0x52, 0xee, 0xb9, - 0xa2, 0xf0, 0x32, 0x57, 0x47, 0x95, 0xb9, 0xc7, 0x0b, 0x15, 0xba, 0xb6, 0xd6, 0xba, 0xb3, 0xdf, - 0x52, 0xee, 0xd8, 0x5a, 0x1b, 0x5b, 0xb6, 0xdc, 0xee, 0xf0, 0x9c, 0x25, 0x9a, 0x63, 0x9b, 0xb2, - 0xa2, 0xe9, 0xcd, 0x3b, 0x26, 0x56, 0x0c, 0x53, 0xc5, 0xaa, 0x64, 0x75, 0x64, 0xdd, 0xa9, 0x72, - 0xd3, 0x68, 0x1a, 0xf4, 0xe7, 0x1d, 0xf2, 0x8b, 0x51, 0x8b, 0xbf, 0x00, 0x29, 0x51, 0xd6, 0x9b, - 0xb8, 0xa6, 0xef, 0x19, 0xe8, 0xb3, 0x10, 0x53, 0xb1, 0xa5, 0x14, 0x84, 0x25, 0xe1, 0x56, 0x7a, - 0xa5, 0xb8, 0x3c, 0xd0, 0x17, 0xcb, 0xb4, 0xec, 0x1a, 0xb6, 0x14, 0x53, 0xeb, 0xd8, 0x86, 0x59, - 0x8a, 0x7d, 0x70, 0xbc, 0x38, 0x25, 0x52, 0x2e, 0xf4, 0x49, 0x88, 0xb7, 0xb0, 0x6c, 0xe1, 0x42, - 0x84, 0xb2, 0x17, 0x02, 0xd8, 0x1f, 0x91, 0x7c, 0xce, 0xc4, 0x0a, 0x17, 0xff, 0x52, 0x80, 0xac, - 0x88, 0x9f, 0x75, 0xb1, 0x65, 0x57, 0xb1, 0xac, 0x62, 0x13, 0x5d, 0x84, 0xe8, 0x53, 0x7c, 0x54, - 0x88, 0x2e, 0x09, 0xb7, 0x32, 0xa5, 0xe9, 0x0f, 0x8f, 0x17, 0xa3, 0xeb, 0xf8, 0x48, 0x24, 0x34, - 0xb4, 0x04, 0xd3, 0x58, 0x57, 0x25, 0x92, 0x1d, 0xeb, 0xcd, 0x4e, 0x60, 0x5d, 0x5d, 0xc7, 0x47, - 0x48, 0x81, 0xa4, 0x45, 0xa4, 0xe9, 0x0a, 0x2e, 0xc4, 0x97, 0x84, 0x5b, 0xf1, 0xd2, 0x83, 0x0f, - 0x8f, 0x17, 0xcb, 0x4d, 0xcd, 0xde, 0xef, 0xee, 0x2e, 0x2b, 0x46, 0xfb, 0x8e, 0x5b, 0x2b, 0x75, - 0xd7, 0xfb, 0x7d, 0xa7, 0xf3, 0xb4, 0x79, 0x67, 0xc8, 0x08, 0x2c, 0x37, 0x0e, 0xf5, 0x3a, 0x7e, - 0x26, 0xba, 0x82, 0xdf, 0x8c, 0xfd, 0xec, 0x5b, 0x8b, 0xc2, 0xc3, 0x58, 0x52, 0xc8, 0x47, 0x1e, - 0xc6, 0x92, 0x91, 0x7c, 0xb4, 0xf8, 0xf5, 0x28, 0xe4, 0x44, 0x6c, 0x75, 0x0c, 0xdd, 0xc2, 0xbc, - 0x19, 0x9f, 0x80, 0xa8, 0x7d, 0xa8, 0xd3, 0x66, 0xa4, 0x57, 0x2e, 0x07, 0x74, 0x46, 0xc3, 0x94, - 0x75, 0x4b, 0x56, 0x6c, 0xcd, 0xd0, 0x45, 0x52, 0x14, 0x7d, 0x06, 0xd2, 0x26, 0xb6, 0xba, 0x6d, - 0x4c, 0x87, 0x8d, 0xb6, 0x30, 0xbd, 0x72, 0x21, 0x80, 0xb3, 0xde, 0x91, 0x75, 0x11, 0x58, 0x59, - 0xf2, 0x1b, 0x5d, 0x84, 0xa4, 0xde, 0x6d, 0x93, 0x7e, 0xb1, 0x68, 0xab, 0xa3, 0xe2, 0xb4, 0xde, - 0x6d, 0xaf, 0xe3, 0x23, 0x0b, 0x95, 0x21, 0x6d, 0x92, 0x41, 0x93, 0x34, 0x7d, 0xcf, 0xb0, 0x0a, - 0x89, 0xa5, 0xe8, 0xad, 0xf4, 0xca, 0xa5, 0x61, 0x43, 0x4b, 0xd4, 0x80, 0x8f, 0x0f, 0x98, 0x0e, - 0xc1, 0x42, 0x75, 0xc8, 0xf2, 0x9a, 0x99, 0x58, 0xb6, 0x0c, 0xbd, 0x30, 0xbd, 0x24, 0xdc, 0xca, - 0xad, 0x2c, 0x07, 0x89, 0xe9, 0xe9, 0x05, 0x92, 0xec, 0xb6, 0xb1, 0x48, 0xb9, 0xc4, 0x8c, 0xe9, - 0x4b, 0x15, 0x9f, 0x40, 0xc6, 0x9f, 0x8b, 0x10, 0xe4, 0xc4, 0x4a, 0x7d, 0x67, 0xa3, 0x22, 0xed, - 0x6c, 0xae, 0x6f, 0x6e, 0xbd, 0xbb, 0x99, 0x9f, 0x42, 0xf3, 0x90, 0xe7, 0xb4, 0xf5, 0xca, 0x13, - 0xe9, 0x51, 0x6d, 0xa3, 0xd6, 0xc8, 0x0b, 0xe8, 0x22, 0x9c, 0xe3, 0x54, 0x71, 0x75, 0xf3, 0x41, - 0x45, 0x2a, 0x6d, 0xed, 0x6c, 0xae, 0xad, 0x8a, 0x4f, 0xf2, 0x91, 0x85, 0xd8, 0x2f, 0x7f, 0xfb, - 0xf2, 0x54, 0xf1, 0x31, 0xc0, 0x03, 0x6c, 0x73, 0xb5, 0x42, 0x25, 0x48, 0xec, 0xd3, 0xda, 0x70, - 0xc5, 0x5e, 0x0a, 0xac, 0xb6, 0x4f, 0x05, 0x4b, 0x49, 0xd2, 0x03, 0x3f, 0x3a, 0x5e, 0x14, 0x44, - 0xce, 0xc9, 0x86, 0xbc, 0xf8, 0x7d, 0x01, 0xd2, 0x54, 0x30, 0x6b, 0x23, 0x2a, 0xf7, 0x49, 0xbe, - 0x32, 0xb6, 0x43, 0x06, 0x45, 0xa3, 0x65, 0x88, 0x3f, 0x97, 0x5b, 0xdd, 0x51, 0xf3, 0xe6, 0x31, - 0xc9, 0x17, 0x59, 0x31, 0xf4, 0x16, 0x64, 0x34, 0xdd, 0xc6, 0xba, 0x2d, 0x31, 0xb6, 0xe8, 0x18, - 0xb6, 0x34, 0x2b, 0x4d, 0x13, 0xc5, 0x3f, 0x13, 0x00, 0xb6, 0xbb, 0x61, 0x76, 0x0d, 0x99, 0xf7, - 0x13, 0xd5, 0xdf, 0x99, 0xf7, 0xac, 0x15, 0xe7, 0x21, 0xa1, 0xe9, 0x2d, 0x4d, 0x67, 0xf5, 0x4f, - 0x8a, 0x3c, 0x85, 0xe6, 0x21, 0xbe, 0xdb, 0xd2, 0x74, 0x95, 0xaa, 0x7f, 0x52, 0x64, 0x09, 0xde, - 0xfd, 0x22, 0xa4, 0x69, 0xdd, 0x43, 0xec, 0xfd, 0xe2, 0x37, 0x23, 0x70, 0xae, 0x6c, 0xe8, 0xaa, - 0x46, 0xe6, 0xa1, 0xdc, 0xfa, 0x58, 0xf4, 0xcd, 0x1b, 0x90, 0xc2, 0x87, 0x9d, 0x09, 0x87, 0x37, - 0x89, 0x0f, 0x3b, 0xf4, 0x57, 0x70, 0xd7, 0xa1, 0x4f, 0xc2, 0x05, 0xb9, 0xd5, 0x32, 0x0e, 0x24, - 0x6d, 0x4f, 0x52, 0x0d, 0x6c, 0x49, 0xba, 0x61, 0x4b, 0xf8, 0x50, 0xb3, 0x6c, 0x6a, 0x2a, 0x92, - 0xe2, 0x1c, 0xcd, 0xae, 0xed, 0xad, 0x19, 0xd8, 0xda, 0x34, 0xec, 0x0a, 0xc9, 0xe2, 0x1d, 0xfe, - 0x3e, 0x9c, 0xef, 0xef, 0x9b, 0x30, 0xfb, 0xfe, 0x6f, 0x04, 0xc8, 0xd5, 0x74, 0xcd, 0xfe, 0x58, - 0x74, 0xba, 0xdb, 0x7b, 0x51, 0x7f, 0xef, 0xdd, 0x86, 0xfc, 0x9e, 0xac, 0xb5, 0xb6, 0xf4, 0x86, - 0xd1, 0xde, 0xb5, 0x6c, 0x43, 0xc7, 0x16, 0xef, 0xde, 0x01, 0x3a, 0xef, 0xb3, 0xc7, 0x30, 0xe3, - 0xb6, 0x29, 0xcc, 0xce, 0x7a, 0x01, 0xf9, 0x9a, 0xae, 0x98, 0xb8, 0x8d, 0xf5, 0x50, 0x7b, 0xeb, - 0x12, 0xa4, 0x34, 0x47, 0x2e, 0xed, 0xb1, 0xa8, 0xe8, 0x11, 0x78, 0x9b, 0xba, 0x30, 0xeb, 0xfb, - 0x76, 0x98, 0xc6, 0xef, 0x25, 0x48, 0xe9, 0xf8, 0x40, 0xf2, 0xc6, 0x2b, 0x2a, 0x26, 0x75, 0x7c, - 0xc0, 0x8c, 0xd5, 0x13, 0xc8, 0xae, 0xe1, 0x16, 0xb6, 0x71, 0xf8, 0x96, 0x7c, 0x07, 0x72, 0x8e, - 0xe8, 0x30, 0x07, 0xe9, 0xb7, 0x05, 0x40, 0x5c, 0x2e, 0x59, 0x3d, 0xc3, 0x1c, 0xa7, 0x45, 0xe2, - 0x1d, 0xd8, 0x5d, 0x53, 0x67, 0xcb, 0x3c, 0xd3, 0x52, 0x60, 0x24, 0xba, 0xd2, 0x7b, 0x16, 0x35, - 0xe6, 0xb7, 0xa8, 0xae, 0xb7, 0x42, 0xfc, 0x94, 0x03, 0x98, 0xeb, 0xa9, 0x5e, 0xb8, 0x43, 0x19, - 0xa3, 0x35, 0x8b, 0x2c, 0x45, 0xfd, 0x9e, 0x19, 0x25, 0x16, 0xdf, 0x87, 0xd9, 0x72, 0x0b, 0xcb, - 0x66, 0xd8, 0xdd, 0xc2, 0x87, 0xf3, 0x09, 0x20, 0xbf, 0xf8, 0x30, 0x87, 0x54, 0x83, 0x74, 0x5d, - 0x91, 0xf5, 0xad, 0x0e, 0x31, 0x82, 0x16, 0xba, 0x07, 0xe7, 0x2d, 0xdb, 0xe8, 0x48, 0xb2, 0x2d, - 0x31, 0xbf, 0x6a, 0xd7, 0xe8, 0xea, 0xaa, 0x6c, 0x1e, 0xd1, 0x6f, 0x24, 0xc5, 0x39, 0x92, 0xbb, - 0x6a, 0xd3, 0x8a, 0x94, 0x78, 0x16, 0x19, 0xbb, 0xb6, 0xa6, 0x4b, 0xc4, 0xfd, 0x69, 0xd9, 0x16, - 0xd7, 0x73, 0x68, 0x6b, 0xba, 0xc8, 0x28, 0xbc, 0x15, 0xdf, 0x16, 0xd8, 0xb7, 0xc2, 0x54, 0x9b, - 0xb7, 0x21, 0x6d, 0x29, 0xb2, 0x2e, 0xed, 0x19, 0x66, 0x5b, 0xb6, 0xa9, 0x6a, 0xe4, 0x56, 0x5e, - 0x0e, 0x72, 0x2a, 0x15, 0x59, 0xbf, 0x4f, 0x0b, 0x89, 0x60, 0xb9, 0xbf, 0xfd, 0xda, 0xf3, 0x30, - 0x96, 0x8c, 0xe6, 0x63, 0xc5, 0x7f, 0x17, 0x20, 0xc3, 0x6a, 0x19, 0xa6, 0xf6, 0xbc, 0x01, 0x31, - 0xd3, 0x38, 0x60, 0xda, 0x93, 0x5e, 0x79, 0x29, 0x40, 0xc4, 0x3a, 0x3e, 0xf2, 0x9b, 0x6d, 0x5a, - 0x1c, 0x95, 0x80, 0xbb, 0x37, 0x12, 0xe5, 0x8e, 0x4e, 0xca, 0x0d, 0x8c, 0x4b, 0x24, 0x32, 0x6e, - 0xc2, 0xcc, 0xae, 0x6c, 0x2b, 0xfb, 0x64, 0x7c, 0x68, 0x25, 0x89, 0x89, 0x8f, 0xde, 0xca, 0x88, - 0x39, 0x4a, 0x76, 0xaa, 0x6e, 0x15, 0x7f, 0x5f, 0x00, 0x24, 0xe2, 0xe7, 0xd8, 0xb4, 0xf0, 0xc7, - 0x7f, 0x98, 0xfe, 0x43, 0x80, 0xb9, 0x9e, 0xca, 0xfe, 0x6f, 0x1b, 0xad, 0x1f, 0x0b, 0x70, 0xa1, - 0xbc, 0x8f, 0x95, 0xa7, 0x65, 0x43, 0xb7, 0x34, 0xcb, 0xc6, 0xba, 0x72, 0x14, 0xe6, 0x90, 0xbd, - 0x04, 0xa9, 0x03, 0xcd, 0xde, 0x97, 0x54, 0x6d, 0x6f, 0x8f, 0x4e, 0xe9, 0xa4, 0x98, 0x24, 0x84, - 0x35, 0x6d, 0x6f, 0x0f, 0xdd, 0x83, 0x58, 0xdb, 0x50, 0x99, 0xf7, 0x96, 0x5b, 0x59, 0x0c, 0x10, - 0x4f, 0xab, 0x66, 0x75, 0xdb, 0x1b, 0x86, 0x8a, 0x45, 0x5a, 0x18, 0x5d, 0x06, 0x50, 0x08, 0xb5, - 0x63, 0x68, 0xba, 0xcd, 0xad, 0xb8, 0x8f, 0xc2, 0xad, 0xc4, 0x37, 0x62, 0x50, 0x18, 0x6c, 0x57, - 0x98, 0xa3, 0xbb, 0x0d, 0x09, 0x66, 0xaa, 0xf8, 0xf8, 0xae, 0x0c, 0xab, 0x7e, 0x40, 0x0d, 0x96, - 0x99, 0x49, 0xe3, 0x03, 0xc7, 0xe5, 0x2c, 0xfc, 0xb9, 0x00, 0x09, 0x96, 0x81, 0xee, 0x42, 0x92, - 0x03, 0x52, 0x95, 0xd6, 0x31, 0x5a, 0x3a, 0x7f, 0x72, 0xbc, 0x38, 0xcd, 0xe0, 0xe7, 0xda, 0x87, - 0xde, 0x4f, 0x71, 0x9a, 0x21, 0x50, 0x95, 0xf4, 0xb4, 0x65, 0xcb, 0xa6, 0x4d, 0x81, 0x3f, 0xe9, - 0xe9, 0x8c, 0x98, 0xa4, 0x04, 0x82, 0xf8, 0x1f, 0x42, 0xc2, 0xb2, 0x65, 0xbb, 0x6b, 0xf1, 0xbe, - 0x3e, 0x55, 0x65, 0xeb, 0x94, 0x53, 0xe4, 0x12, 0xc8, 0x12, 0xaa, 0x62, 0x5b, 0xd6, 0x5a, 0xb4, - 0xf3, 0x53, 0x22, 0x4f, 0x15, 0x7f, 0x53, 0x80, 0x04, 0x2b, 0x8a, 0x2e, 0xc0, 0x1c, 0x03, 0x9d, - 0xb5, 0xcd, 0xb5, 0x4a, 0xa3, 0x22, 0x6e, 0xd4, 0x36, 0x57, 0x1b, 0x95, 0xfc, 0x14, 0x3a, 0x0f, - 0xc8, 0xc9, 0x28, 0x6f, 0x6d, 0xd6, 0x6b, 0xf5, 0x46, 0x65, 0x93, 0x80, 0x55, 0x02, 0x61, 0x29, - 0xdd, 0x47, 0x8d, 0xa0, 0x6b, 0xb0, 0xd4, 0x4f, 0x95, 0xea, 0x8d, 0xd5, 0x46, 0x5d, 0xaa, 0xd4, - 0x1b, 0xb5, 0x8d, 0xd5, 0x46, 0x65, 0x2d, 0x1f, 0x1d, 0x51, 0x8a, 0x7c, 0x44, 0x14, 0x2b, 0xe5, - 0x46, 0x3e, 0x56, 0x7c, 0x01, 0xe7, 0x44, 0xac, 0x18, 0xed, 0x4e, 0xd7, 0xc6, 0xa4, 0x96, 0x56, - 0x98, 0x5a, 0x7e, 0x01, 0xa6, 0x55, 0xf3, 0x48, 0x32, 0xbb, 0x3a, 0xd7, 0xf1, 0x84, 0x6a, 0x1e, - 0x89, 0x5d, 0x9d, 0x2b, 0xe3, 0x9f, 0x08, 0x70, 0xbe, 0xff, 0xe3, 0x61, 0xaa, 0xe2, 0x17, 0x21, - 0x2d, 0xab, 0x2a, 0x56, 0x25, 0x15, 0xb7, 0x6c, 0x99, 0x7b, 0xf4, 0x77, 0x7d, 0x92, 0xf8, 0xa6, - 0xcd, 0x32, 0xdb, 0xad, 0x59, 0x76, 0x37, 0x6d, 0x36, 0x1e, 0x97, 0xcb, 0xb4, 0x3e, 0x6b, 0x84, - 0xd1, 0xb1, 0x23, 0x54, 0x16, 0xa5, 0x14, 0x15, 0xb8, 0x50, 0xc2, 0x4d, 0x4d, 0xf7, 0x6f, 0xc3, - 0x84, 0xee, 0x97, 0x48, 0x50, 0x18, 0xfc, 0x48, 0x98, 0xde, 0xc9, 0x5f, 0x47, 0xe1, 0x5c, 0x45, - 0x57, 0x3f, 0x9a, 0x46, 0x90, 0xf9, 0xa0, 0x18, 0xed, 0xb6, 0x66, 0x3b, 0x63, 0xcf, 0x52, 0xe8, - 0xff, 0x40, 0x52, 0xc5, 0xb2, 0xea, 0xc2, 0xf7, 0x74, 0xcf, 0x52, 0xd5, 0xb5, 0xb5, 0xd6, 0xf2, - 0x7e, 0x4b, 0x59, 0x6e, 0x38, 0x9b, 0x94, 0xa2, 0x5b, 0x1c, 0x7d, 0x19, 0x2e, 0x10, 0x63, 0x6e, - 0xea, 0x72, 0x4b, 0x62, 0xd2, 0x24, 0xdb, 0xd4, 0x9a, 0x4d, 0x6c, 0xf2, 0x0d, 0xaf, 0x5b, 0x01, - 0xf5, 0xac, 0x71, 0x8e, 0x32, 0x65, 0x68, 0xb0, 0xf2, 0xe2, 0x39, 0x2d, 0x88, 0x8c, 0xbe, 0xe0, - 0xee, 0x8f, 0x58, 0x1d, 0x59, 0xb7, 0x0a, 0x71, 0x6a, 0xc3, 0x86, 0xed, 0xa3, 0x71, 0xcd, 0xe0, - 0xeb, 0x12, 0xa1, 0x58, 0xe8, 0x0e, 0x71, 0xb5, 0x9f, 0x75, 0x35, 0x13, 0x4b, 0x77, 0x3b, 0x4a, - 0x21, 0x41, 0xda, 0x5e, 0xca, 0x9d, 0x1c, 0x2f, 0x82, 0xc8, 0xc8, 0x77, 0xb7, 0xcb, 0xc4, 0xf5, - 0x66, 0xbf, 0x3b, 0x0a, 0xba, 0x05, 0x79, 0xdd, 0x90, 0x4c, 0xbc, 0x67, 0x62, 0x6b, 0x9f, 0x7f, - 0x36, 0x49, 0x7b, 0x2c, 0xa7, 0x1b, 0x22, 0x23, 0x33, 0xd1, 0xe7, 0x21, 0xd1, 0x31, 0x34, 0xcb, - 0xd0, 0x0b, 0x29, 0xd6, 0xa3, 0x2c, 0xe5, 0xae, 0xdf, 0xd3, 0xf9, 0x64, 0xf1, 0x57, 0x05, 0x38, - 0xdf, 0x3f, 0xa6, 0x61, 0xce, 0xa9, 0x5b, 0x90, 0x37, 0x74, 0x2c, 0x75, 0xf6, 0x65, 0x0b, 0xf3, - 0x31, 0xe0, 0x8b, 0x4d, 0xce, 0xd0, 0xf1, 0x36, 0x21, 0xb3, 0x1e, 0xed, 0xf1, 0x27, 0x7e, 0x49, - 0x80, 0xd9, 0x55, 0xb5, 0xad, 0xe9, 0xf5, 0x4e, 0x4b, 0x0b, 0x15, 0x81, 0x5e, 0x83, 0x94, 0x45, - 0x64, 0x7a, 0xe6, 0xdd, 0x43, 0x0f, 0x49, 0x9a, 0xb3, 0x8e, 0x8f, 0x3c, 0x47, 0xdf, 0x5f, 0x89, - 0x30, 0xa7, 0xd2, 0xfb, 0xbc, 0x7d, 0x1b, 0xd8, 0xfc, 0x88, 0x20, 0x8a, 0x5f, 0x7c, 0x98, 0x35, - 0xff, 0x9a, 0x00, 0x17, 0xa9, 0x6c, 0xaa, 0x32, 0x7b, 0xd8, 0xa4, 0xfb, 0xec, 0x61, 0x0e, 0xd1, - 0x55, 0x48, 0xd8, 0xb2, 0xd9, 0xc4, 0xcc, 0x10, 0xc4, 0x4b, 0x69, 0xb2, 0x4e, 0xd7, 0x6d, 0xc3, - 0x24, 0xeb, 0x34, 0xcf, 0xe2, 0xed, 0x94, 0x61, 0x21, 0xa8, 0x2e, 0x61, 0xb6, 0xf7, 0x8f, 0x23, - 0xfc, 0x1b, 0xe5, 0x7d, 0x86, 0xf7, 0x3a, 0x2d, 0x4d, 0x91, 0x43, 0x5d, 0xf6, 0x2a, 0x90, 0x56, - 0xa8, 0x70, 0xc9, 0x3e, 0xea, 0xb0, 0x9d, 0x89, 0xdc, 0xca, 0xb5, 0x40, 0x41, 0xf4, 0xe3, 0xac, - 0x26, 0x8d, 0xa3, 0x0e, 0x26, 0x1e, 0x9b, 0xf3, 0x1b, 0xad, 0xc1, 0x34, 0xeb, 0x1c, 0xc7, 0xd9, - 0x1d, 0x21, 0x82, 0x4c, 0xf4, 0x06, 0x2d, 0xcc, 0x6d, 0x92, 0xc3, 0x8a, 0x3e, 0x07, 0x49, 0x7c, - 0xd8, 0x91, 0xe8, 0xd9, 0x4c, 0x6c, 0xd2, 0xb3, 0x19, 0x71, 0x1a, 0x1f, 0x76, 0x48, 0x92, 0x8f, - 0xcb, 0x37, 0x05, 0x78, 0x29, 0xb0, 0xd3, 0xc2, 0x34, 0x2d, 0x9f, 0xe2, 0x27, 0x48, 0x91, 0x89, - 0x6b, 0x49, 0xcb, 0x17, 0xff, 0xc0, 0xd1, 0x60, 0x11, 0xb7, 0x0c, 0x45, 0xfe, 0x08, 0xb6, 0x4f, - 0x7c, 0x23, 0x11, 0x39, 0xf3, 0x48, 0xf4, 0xa9, 0x78, 0x5f, 0x65, 0xc3, 0x54, 0xf1, 0x5f, 0x17, - 0x60, 0xae, 0x8a, 0x65, 0xd3, 0xde, 0xc5, 0xb2, 0xdd, 0x38, 0x0c, 0x75, 0x55, 0x7f, 0x03, 0xa2, - 0xba, 0x71, 0xc0, 0xc7, 0x68, 0xf4, 0xc2, 0xcd, 0xdb, 0x4f, 0xca, 0xf3, 0xb6, 0x7f, 0x09, 0xe6, - 0x7b, 0xeb, 0x15, 0x66, 0xab, 0xbf, 0x1b, 0x85, 0xd4, 0x83, 0x72, 0x98, 0x6d, 0xfd, 0x2c, 0xdf, - 0x94, 0x62, 0xb3, 0x2f, 0x48, 0x21, 0xdd, 0xef, 0x2d, 0x3f, 0x28, 0xaf, 0xe3, 0x23, 0x07, 0xaf, - 0x12, 0x2e, 0xb4, 0x0a, 0x29, 0x7b, 0x9f, 0x2c, 0xde, 0x46, 0x4b, 0xe5, 0x33, 0x6f, 0xa2, 0xfe, - 0xf2, 0xb8, 0x50, 0x0b, 0xce, 0xd9, 0x87, 0x3a, 0xf5, 0x09, 0xa4, 0xa6, 0x22, 0x79, 0xe2, 0xe2, - 0x93, 0x88, 0x5b, 0x20, 0xe2, 0x4e, 0x8e, 0x17, 0x51, 0xe3, 0x50, 0x27, 0x0e, 0xc4, 0x83, 0x72, - 0xc3, 0x11, 0x20, 0x22, 0x9b, 0xd3, 0x14, 0x97, 0xb6, 0xf0, 0x14, 0xe2, 0xb4, 0x15, 0xce, 0x21, - 0xaa, 0x10, 0x70, 0x88, 0x4a, 0x1a, 0xe5, 0x7c, 0xe0, 0x34, 0x4a, 0xe0, 0x71, 0x31, 0x55, 0xe0, - 0x0a, 0xf1, 0x0e, 0x00, 0xe9, 0xc2, 0x30, 0xd5, 0xe0, 0x3f, 0xa3, 0x90, 0xdb, 0xee, 0x5a, 0xfb, - 0x21, 0xeb, 0x7d, 0x19, 0xa0, 0xd3, 0xb5, 0xf6, 0xb1, 0x29, 0xd9, 0x87, 0x3a, 0x6f, 0xf9, 0x98, - 0x83, 0x59, 0xa7, 0xe9, 0x8c, 0xaf, 0x71, 0xa8, 0xa3, 0x2d, 0x2e, 0x04, 0x4b, 0xde, 0xe9, 0xee, - 0xed, 0x09, 0xf0, 0x48, 0xe3, 0x50, 0xdf, 0xc0, 0x2e, 0x10, 0x61, 0x02, 0x31, 0x11, 0xf8, 0x59, - 0x98, 0x26, 0x09, 0xc9, 0x36, 0x4e, 0xa3, 0x61, 0x09, 0xc2, 0xd3, 0x30, 0xd0, 0x43, 0xc8, 0xa9, - 0xb8, 0x63, 0x62, 0x62, 0x86, 0x54, 0x89, 0x4c, 0xeb, 0xf8, 0xe4, 0x42, 0xb2, 0x1e, 0xeb, 0xa6, - 0x71, 0x80, 0xde, 0x82, 0x14, 0xab, 0x09, 0x59, 0xf1, 0x12, 0x74, 0xc5, 0x0b, 0xea, 0x1e, 0x3e, - 0x32, 0x74, 0xad, 0x4b, 0xd2, 0x6a, 0x90, 0x95, 0x6e, 0x1e, 0xe2, 0x7b, 0x86, 0xa9, 0x60, 0x7a, - 0x34, 0x9c, 0x14, 0x59, 0x02, 0xdd, 0x86, 0x59, 0x4d, 0x57, 0x5a, 0x5d, 0x4b, 0x7b, 0x8e, 0x25, - 0xa7, 0x99, 0xcc, 0xf3, 0x9d, 0x71, 0x33, 0xa8, 0x40, 0xc3, 0x75, 0x81, 0x93, 0xf9, 0x14, 0x01, - 0xdc, 0x33, 0xae, 0x06, 0x84, 0xb9, 0x40, 0x95, 0x7b, 0x86, 0xef, 0xf4, 0x3a, 0x40, 0x86, 0xac, - 0xf8, 0xb7, 0x02, 0xcc, 0x12, 0xd0, 0xfb, 0x9c, 0xaa, 0x44, 0x98, 0x2a, 0x5a, 0x62, 0x41, 0x03, - 0x91, 0x33, 0xaa, 0x15, 0x0d, 0x23, 0xb8, 0x0b, 0xf3, 0x5a, 0x9b, 0xac, 0x63, 0x9a, 0xdd, 0x3a, - 0xe2, 0xfe, 0xbd, 0x8d, 0x9d, 0x73, 0xad, 0x39, 0x2f, 0xaf, 0xec, 0x64, 0xf1, 0x99, 0xfc, 0xbb, - 0x74, 0x7b, 0xd3, 0x6b, 0x56, 0x98, 0xfd, 0x5e, 0x83, 0xac, 0xc9, 0x44, 0x63, 0xf5, 0xd4, 0x5d, - 0x9f, 0x71, 0x59, 0x49, 0xef, 0x7f, 0x27, 0x02, 0x33, 0xef, 0x74, 0xb1, 0x79, 0xf4, 0x31, 0xec, - 0xfb, 0x1b, 0x30, 0x73, 0x20, 0x6b, 0xb6, 0xb4, 0x67, 0x98, 0x52, 0xb7, 0xa3, 0xca, 0xb6, 0x73, - 0xbc, 0x9d, 0x25, 0xe4, 0xfb, 0x86, 0xb9, 0x43, 0x89, 0x08, 0x03, 0x7a, 0xaa, 0x1b, 0x07, 0xba, - 0x44, 0xc8, 0x9a, 0xde, 0x24, 0x5d, 0xc2, 0xf7, 0x31, 0x4b, 0x9f, 0xfe, 0xfb, 0xe3, 0xc5, 0x7b, - 0x13, 0x05, 0xac, 0xd0, 0x98, 0x9f, 0x6e, 0x57, 0x53, 0x97, 0x77, 0x76, 0x6a, 0x6b, 0x62, 0x9e, - 0x8a, 0x7c, 0x97, 0x49, 0x6c, 0x1c, 0xea, 0x8e, 0xbb, 0xf2, 0xa1, 0x00, 0x79, 0xaf, 0xc3, 0xc2, - 0x1c, 0xd5, 0x0a, 0xa4, 0x9f, 0x75, 0xb1, 0xa9, 0x9d, 0x61, 0x4c, 0x81, 0x33, 0x12, 0x13, 0xf8, - 0x1e, 0x64, 0x7a, 0xfa, 0x21, 0xfa, 0xf3, 0xf5, 0x43, 0xfa, 0xc0, 0xeb, 0x82, 0xe2, 0x0f, 0x05, - 0x40, 0xb4, 0xf1, 0x35, 0xb6, 0x85, 0xfc, 0x31, 0x53, 0x98, 0x5b, 0x90, 0xa7, 0x21, 0x64, 0x92, - 0xb6, 0x27, 0xb5, 0x35, 0xcb, 0xd2, 0xf4, 0x26, 0xd7, 0x98, 0x1c, 0xa5, 0xd7, 0xf6, 0x36, 0x18, - 0x95, 0x8f, 0xe5, 0xff, 0x87, 0xb9, 0x9e, 0xd6, 0x84, 0x39, 0x9a, 0x57, 0x20, 0xb3, 0x67, 0x74, - 0x75, 0x55, 0x62, 0x7b, 0x21, 0x7c, 0xcf, 0x27, 0x4d, 0x69, 0xec, 0x7b, 0xc5, 0xaf, 0x46, 0x60, - 0x5e, 0xc4, 0x96, 0xd1, 0x7a, 0x8e, 0xc3, 0xef, 0xcf, 0x2d, 0xe0, 0xfb, 0xfc, 0xd2, 0xcf, 0xd3, - 0xad, 0x29, 0x26, 0x83, 0x2d, 0xad, 0xbd, 0x5b, 0xc3, 0xd7, 0x46, 0x6b, 0xe6, 0xe0, 0x66, 0x30, - 0xdf, 0xaa, 0x89, 0x0d, 0x6e, 0xd5, 0x14, 0xff, 0x2f, 0x9c, 0xeb, 0xeb, 0x88, 0x30, 0x3d, 0xa0, - 0xbf, 0x8b, 0xc0, 0xc5, 0x5e, 0xf1, 0x61, 0xe3, 0xa1, 0xff, 0x19, 0x9d, 0x8d, 0xaa, 0x90, 0x6d, - 0x6b, 0xba, 0xe4, 0x39, 0xac, 0xa7, 0x70, 0x6f, 0x32, 0x6d, 0x4d, 0x6f, 0xf4, 0xfa, 0xac, 0x04, - 0xba, 0x05, 0xf5, 0x6b, 0x98, 0x63, 0xf7, 0x0d, 0x01, 0x32, 0x61, 0xef, 0x21, 0x9d, 0x2d, 0xa6, - 0x85, 0xb7, 0xb9, 0x01, 0xd9, 0x8f, 0x60, 0xd3, 0xe9, 0xf7, 0x04, 0x40, 0x0d, 0xb3, 0xab, 0x13, - 0xf7, 0xf1, 0x91, 0xd1, 0x0c, 0xb3, 0xb1, 0xf3, 0x10, 0xd7, 0x74, 0x15, 0x1f, 0xd2, 0xc6, 0xc6, - 0x44, 0x96, 0xe8, 0x39, 0x38, 0x8a, 0x4e, 0x74, 0x70, 0xc4, 0xdb, 0xff, 0x1e, 0xcc, 0xf5, 0x54, - 0x34, 0xcc, 0x5e, 0xf8, 0x4e, 0x04, 0xe6, 0x78, 0x73, 0x42, 0xdf, 0x74, 0x3b, 0x53, 0x40, 0x2d, - 0xfa, 0x1c, 0x40, 0xc7, 0xc4, 0xcf, 0x25, 0xc6, 0x1a, 0x9d, 0x88, 0x35, 0x45, 0x38, 0x28, 0x01, - 0x7d, 0x11, 0x66, 0xc8, 0x84, 0xeb, 0x98, 0x46, 0xc7, 0xb0, 0xc8, 0xba, 0x6e, 0x4d, 0x06, 0x4b, - 0x66, 0x4f, 0x8e, 0x17, 0xb3, 0x1b, 0x9a, 0xbe, 0xcd, 0x19, 0x1b, 0x75, 0x91, 0xcc, 0x5c, 0x37, - 0xe9, 0x38, 0x23, 0x3f, 0x16, 0x60, 0xfe, 0x23, 0xdb, 0xa6, 0xfc, 0xef, 0xe8, 0x31, 0x77, 0x3d, - 0xc8, 0xd3, 0x64, 0x4d, 0xdf, 0x33, 0xc2, 0xdf, 0x3c, 0xfe, 0x86, 0x00, 0xb3, 0x3e, 0xf1, 0x61, - 0xae, 0xfa, 0x67, 0x0b, 0xdb, 0xfe, 0x12, 0xf1, 0x03, 0xfc, 0x6a, 0x1f, 0xe6, 0xa4, 0xfa, 0x37, - 0x01, 0xce, 0x97, 0xd9, 0x91, 0xa2, 0x73, 0x4a, 0x1e, 0xa6, 0x96, 0x14, 0x60, 0xfa, 0x39, 0x36, - 0x2d, 0xcd, 0x60, 0xeb, 0x5e, 0x56, 0x74, 0x92, 0x68, 0x01, 0x92, 0x96, 0x2e, 0x77, 0xac, 0x7d, - 0xc3, 0x39, 0x11, 0x71, 0xd3, 0xee, 0x89, 0x7e, 0xfc, 0xec, 0x27, 0xfa, 0x89, 0xe0, 0x13, 0x7d, - 0x7e, 0xc0, 0xf2, 0x03, 0x01, 0x2e, 0x0c, 0xb4, 0x3a, 0xcc, 0x91, 0xfe, 0x0a, 0xa4, 0x15, 0x2e, - 0x98, 0xd8, 0x50, 0x76, 0xd2, 0x52, 0x23, 0xc5, 0xce, 0xe8, 0x69, 0x9f, 0x1c, 0x2f, 0x82, 0x53, - 0xd5, 0xda, 0x1a, 0x6f, 0x18, 0xf9, 0xad, 0x16, 0x7f, 0x0a, 0x90, 0xad, 0x1c, 0x76, 0x0c, 0xd3, - 0xae, 0x33, 0x47, 0x02, 0xad, 0x41, 0xb2, 0x63, 0x1a, 0xcf, 0x35, 0xa7, 0x11, 0xb9, 0xc0, 0x93, - 0xbe, 0x1e, 0x9e, 0x6d, 0x5e, 0x5e, 0x74, 0x39, 0x91, 0x08, 0xa9, 0x47, 0x86, 0x22, 0xb7, 0xee, - 0x6b, 0x2d, 0x47, 0x63, 0x97, 0xc7, 0x89, 0x59, 0x76, 0x39, 0xb6, 0x65, 0x7b, 0xdf, 0x99, 0xc0, - 0x2e, 0x11, 0x3d, 0x80, 0x64, 0xd5, 0xb6, 0x3b, 0x24, 0x93, 0xcf, 0xfe, 0xeb, 0x63, 0x45, 0x12, - 0x06, 0x2e, 0xc9, 0x65, 0x46, 0x22, 0xcc, 0x3e, 0x30, 0x8c, 0x66, 0x0b, 0x97, 0x5b, 0x46, 0x57, - 0x2d, 0x1b, 0xfa, 0x9e, 0xd6, 0xe4, 0xd6, 0xf3, 0xda, 0x58, 0x89, 0x0f, 0xca, 0x75, 0x71, 0x90, - 0x1d, 0x7d, 0x1e, 0x92, 0xf5, 0x7b, 0x5c, 0x14, 0xf3, 0x7d, 0xae, 0x8e, 0x15, 0x55, 0xbf, 0x27, - 0xba, 0x4c, 0xa8, 0x0a, 0xe9, 0xd5, 0x17, 0x5d, 0x13, 0x73, 0x19, 0x09, 0x2a, 0xe3, 0xc6, 0x58, - 0x19, 0x94, 0x47, 0xf4, 0xb3, 0xa2, 0x77, 0x20, 0xf7, 0xae, 0x61, 0x3e, 0x6d, 0x19, 0xb2, 0xd3, - 0xb6, 0x69, 0x2a, 0xec, 0x95, 0xb1, 0xc2, 0x1c, 0x36, 0xb1, 0x4f, 0xc0, 0xc2, 0x17, 0x21, 0xdb, - 0x33, 0x38, 0x08, 0x41, 0xac, 0x43, 0xc6, 0x41, 0xa0, 0xf1, 0x17, 0xf4, 0x37, 0x7a, 0x1d, 0xa6, - 0x75, 0x43, 0xc5, 0x8e, 0xce, 0x66, 0x4b, 0xf3, 0x27, 0xc7, 0x8b, 0x89, 0x4d, 0x43, 0x65, 0xcb, - 0x3e, 0xff, 0x25, 0x26, 0x48, 0x21, 0x67, 0xd1, 0x5f, 0xb8, 0x01, 0x31, 0x32, 0x2e, 0x64, 0xb2, - 0xef, 0xca, 0x16, 0xde, 0x31, 0x35, 0x2e, 0xd3, 0x49, 0xf2, 0x72, 0x7f, 0x25, 0x40, 0xa4, 0x7e, - 0x8f, 0xf8, 0x9f, 0xbb, 0x5d, 0xe5, 0x29, 0xb6, 0x79, 0x29, 0x9e, 0xa2, 0x7e, 0xa9, 0x89, 0xf7, - 0x34, 0xe6, 0x8b, 0xa4, 0x44, 0x9e, 0x42, 0x2f, 0x03, 0xc8, 0x8a, 0x82, 0x2d, 0x4b, 0x72, 0xee, - 0xaa, 0xa4, 0xc4, 0x14, 0xa3, 0xac, 0xe3, 0x23, 0xc2, 0x66, 0x61, 0xc5, 0xc4, 0xb6, 0x13, 0x48, - 0xc2, 0x52, 0x84, 0xcd, 0xc6, 0xed, 0x8e, 0x64, 0x1b, 0x4f, 0xb1, 0x4e, 0xc7, 0x33, 0x25, 0xa6, - 0x08, 0xa5, 0x41, 0x08, 0xc4, 0xfe, 0x60, 0x5d, 0xf5, 0x8c, 0x45, 0x4a, 0x74, 0xd3, 0x44, 0xa4, - 0x89, 0x9b, 0x1a, 0xbf, 0x7c, 0x91, 0x12, 0x79, 0x8a, 0x37, 0xe3, 0x9b, 0x02, 0x44, 0x1f, 0x94, - 0xeb, 0xa7, 0x6e, 0x07, 0x82, 0x98, 0xdc, 0xe5, 0x7a, 0x9f, 0x12, 0xe9, 0x6f, 0x1a, 0x61, 0xa5, - 0xb5, 0x5a, 0x04, 0x8e, 0x77, 0x4c, 0xe3, 0x2b, 0x58, 0x71, 0x5a, 0x91, 0xe3, 0xe4, 0x6d, 0x46, - 0x45, 0x4b, 0x90, 0x56, 0x4c, 0xac, 0x62, 0xdd, 0xd6, 0xe4, 0x96, 0xc5, 0x9b, 0xe3, 0x27, 0xf1, - 0xca, 0x7d, 0x55, 0x80, 0x38, 0x55, 0x24, 0x74, 0x09, 0x52, 0x8a, 0xa1, 0xdb, 0xb2, 0xa6, 0x73, - 0x2b, 0x90, 0x12, 0x3d, 0xc2, 0xd0, 0x4a, 0x5e, 0x81, 0x8c, 0xac, 0x28, 0x46, 0x57, 0xb7, 0x25, - 0x5d, 0x6e, 0x63, 0x5e, 0xd9, 0x34, 0xa7, 0x6d, 0xca, 0x6d, 0x8c, 0x16, 0xc1, 0x49, 0xba, 0xb7, - 0x83, 0x52, 0x22, 0x70, 0x92, 0x7b, 0x7c, 0xbc, 0xf0, 0x03, 0x01, 0x92, 0x8e, 0x0a, 0x92, 0xca, - 0x34, 0xb1, 0x8e, 0x4d, 0xd9, 0x36, 0xdc, 0xca, 0xb8, 0x84, 0xfe, 0x55, 0x22, 0xe5, 0xad, 0x12, - 0xf3, 0x10, 0xb7, 0xe5, 0xdd, 0x96, 0x53, 0x0f, 0x96, 0xa0, 0x1b, 0xa0, 0x2d, 0xb9, 0xc9, 0x76, - 0x71, 0x52, 0x22, 0x4b, 0x90, 0x26, 0xf1, 0x90, 0x3e, 0xd6, 0x3b, 0x3c, 0x45, 0xea, 0xcb, 0xa2, - 0xd8, 0x76, 0x71, 0x53, 0xd3, 0xe9, 0x60, 0x47, 0x45, 0xa0, 0x24, 0x1a, 0x30, 0x82, 0x5e, 0x82, - 0x14, 0x2b, 0x80, 0x75, 0x95, 0x8e, 0x78, 0x54, 0x4c, 0x52, 0x42, 0xc5, 0xb9, 0x0e, 0xc1, 0x5d, - 0x83, 0x3f, 0x14, 0x60, 0xf6, 0x5d, 0x53, 0xb3, 0x71, 0x89, 0x85, 0xbf, 0x85, 0xb7, 0x4e, 0xbe, - 0x09, 0x29, 0x55, 0xb6, 0x65, 0x76, 0x1b, 0x29, 0x32, 0xf2, 0x36, 0x92, 0x63, 0x0a, 0x49, 0x79, - 0x7a, 0x23, 0x09, 0x41, 0x8c, 0xfc, 0x66, 0xb7, 0xb8, 0x44, 0xfa, 0xdb, 0x3b, 0x07, 0xf7, 0x57, - 0x37, 0x4c, 0xbf, 0xe1, 0x27, 0x51, 0x67, 0xd1, 0x09, 0xb3, 0x1b, 0xbe, 0x00, 0xd3, 0x1c, 0x0c, - 0xf3, 0x4e, 0x58, 0x1a, 0x67, 0xef, 0x9c, 0x53, 0x43, 0xce, 0x86, 0x4a, 0x00, 0x2c, 0x7e, 0x8d, - 0xc0, 0xd8, 0x89, 0x02, 0x66, 0x9c, 0x45, 0x8a, 0xb2, 0x11, 0x2a, 0xda, 0x84, 0x74, 0xfb, 0xb9, - 0xa2, 0x48, 0x7b, 0x5a, 0xcb, 0xe6, 0xb1, 0x32, 0xc1, 0x01, 0xa2, 0x1b, 0x8f, 0xcb, 0xe5, 0xfb, - 0xb4, 0x10, 0x0b, 0x59, 0xf1, 0xd2, 0x22, 0x10, 0x09, 0xec, 0x37, 0x7a, 0x0d, 0x78, 0xec, 0xb8, - 0x64, 0x39, 0x37, 0x41, 0x4a, 0xd9, 0x93, 0xe3, 0xc5, 0x94, 0x48, 0xa9, 0xf5, 0x7a, 0x43, 0x4c, - 0xb1, 0x02, 0x75, 0xcb, 0x46, 0x57, 0x21, 0x6b, 0xb4, 0x35, 0x5b, 0x72, 0x56, 0x78, 0xee, 0xca, - 0x64, 0x08, 0xd1, 0xf1, 0x00, 0x50, 0x03, 0x6e, 0x62, 0x9d, 0xcc, 0x05, 0xda, 0x4e, 0x16, 0x18, - 0x2d, 0x69, 0x36, 0x9b, 0x4f, 0x92, 0xd1, 0xb1, 0xb5, 0xb6, 0xf6, 0x82, 0x1e, 0xae, 0xf2, 0x43, - 0x82, 0xab, 0xac, 0x38, 0x69, 0x1f, 0x0d, 0x95, 0xae, 0xf1, 0xb2, 0x5b, 0xbe, 0xa2, 0x5c, 0x6b, - 0xbe, 0x2e, 0x40, 0xb6, 0xd4, 0x6d, 0x3d, 0xdd, 0xea, 0xd4, 0xbb, 0xed, 0xb6, 0x6c, 0x1e, 0x91, - 0x09, 0xc2, 0xb4, 0x53, 0x7b, 0x81, 0x59, 0x20, 0x21, 0x57, 0x3f, 0xed, 0x05, 0x26, 0xea, 0xc7, - 0xe3, 0x53, 0x09, 0x9d, 0x05, 0x9f, 0x5e, 0x85, 0x2c, 0x05, 0x92, 0x12, 0xd6, 0x6d, 0x53, 0xc3, - 0x6c, 0x9f, 0x22, 0x2a, 0x66, 0x28, 0xb1, 0xc2, 0x68, 0xe8, 0x3a, 0xe4, 0xac, 0x23, 0xcb, 0xc6, - 0x6d, 0x89, 0xdd, 0xa1, 0x64, 0xe8, 0x27, 0x2a, 0x66, 0x19, 0x55, 0x64, 0xc4, 0xe2, 0x1f, 0x45, - 0x21, 0xe7, 0x68, 0x5a, 0x98, 0x2e, 0x5a, 0x09, 0xe2, 0x7b, 0x5a, 0x0b, 0x3b, 0x67, 0xd4, 0xc3, - 0x97, 0x69, 0x37, 0x82, 0x91, 0xac, 0x9c, 0x8e, 0x6b, 0x4e, 0x59, 0xc3, 0xd0, 0xb6, 0x85, 0x1f, - 0x09, 0x10, 0xa3, 0xbe, 0xd1, 0x5d, 0x88, 0xd1, 0xe9, 0x2f, 0x4c, 0x32, 0xfd, 0x69, 0x51, 0x77, - 0x09, 0x8f, 0xf8, 0x96, 0x70, 0xb2, 0x1e, 0xee, 0xcb, 0x6f, 0xdc, 0x5d, 0xa1, 0x9a, 0x96, 0x11, - 0x79, 0x0a, 0x95, 0x68, 0x64, 0x83, 0x61, 0xda, 0x58, 0xe5, 0x9e, 0x49, 0xd0, 0xe4, 0xea, 0x19, - 0x78, 0xc7, 0xd4, 0x38, 0x7c, 0xe8, 0x22, 0x44, 0x89, 0x0a, 0x4f, 0xb3, 0xa3, 0xce, 0x93, 0xe3, - 0xc5, 0x28, 0x51, 0x5e, 0x42, 0x63, 0x8e, 0xf5, 0xc3, 0x58, 0x32, 0x96, 0x8f, 0x17, 0xbf, 0x17, - 0x83, 0x6c, 0xad, 0x1d, 0xb6, 0x71, 0x58, 0xed, 0x1d, 0xb0, 0x20, 0xc7, 0xb1, 0xe7, 0xa3, 0x01, - 0xe3, 0xd5, 0x63, 0x66, 0xa3, 0xa7, 0x33, 0xb3, 0x35, 0xe2, 0x14, 0xf0, 0x6b, 0x9f, 0xe4, 0xfb, - 0xaf, 0x8e, 0xfd, 0x7e, 0x83, 0xcc, 0x38, 0x91, 0xf0, 0x78, 0x21, 0xba, 0xf4, 0xac, 0xfb, 0x6d, - 0xea, 0x7b, 0x30, 0xa5, 0x49, 0x4c, 0xae, 0x34, 0xd3, 0x58, 0x57, 0xa9, 0xca, 0x1c, 0x72, 0x8d, - 0xf9, 0x0c, 0x44, 0x55, 0x6d, 0x54, 0x97, 0x06, 0x99, 0x4a, 0xc2, 0x32, 0x46, 0x71, 0x62, 0x7e, - 0xc5, 0xf1, 0x03, 0xa7, 0x85, 0x2d, 0x00, 0xaf, 0x55, 0x68, 0x09, 0x12, 0x46, 0x4b, 0x75, 0xa2, - 0x8b, 0xb3, 0xa5, 0xd4, 0xc9, 0xf1, 0x62, 0x7c, 0xab, 0xa5, 0xd6, 0xd6, 0xc4, 0xb8, 0xd1, 0x52, - 0x6b, 0x2a, 0xbd, 0x2d, 0x8b, 0x0f, 0x24, 0x37, 0x50, 0x25, 0x23, 0x4e, 0xeb, 0xf8, 0xc0, 0x0b, - 0x95, 0x71, 0x85, 0x13, 0xb5, 0xf9, 0x1d, 0x01, 0x72, 0x4e, 0x0f, 0x86, 0x3b, 0xd3, 0x93, 0x5a, - 0x9b, 0x6b, 0x7e, 0xf4, 0x74, 0x9a, 0xef, 0xf0, 0xf1, 0x3b, 0x3d, 0x5f, 0x13, 0x60, 0x8e, 0xc5, - 0xc4, 0x29, 0xb2, 0x4d, 0xcc, 0x7c, 0x88, 0xea, 0xfd, 0x0a, 0xe4, 0x4d, 0x59, 0x57, 0x8d, 0xb6, - 0xf6, 0x02, 0xb3, 0xfd, 0x11, 0x8b, 0x1f, 0x0b, 0xcc, 0xb8, 0x74, 0xba, 0x01, 0xe0, 0x6c, 0xef, - 0xfc, 0xab, 0x00, 0xf3, 0xbd, 0x95, 0x09, 0xb3, 0xd3, 0xd6, 0x21, 0x41, 0xb7, 0xf6, 0x9c, 0xe9, - 0xf6, 0x7a, 0x80, 0x90, 0xa0, 0xaf, 0xb3, 0xa8, 0x23, 0x57, 0xe1, 0xa9, 0x88, 0x85, 0x2f, 0x40, - 0x9c, 0x92, 0xcf, 0x60, 0xe3, 0x78, 0xcf, 0x3f, 0x83, 0xd9, 0x55, 0x55, 0xad, 0xd7, 0xb9, 0xf6, - 0x85, 0xd7, 0xed, 0x8e, 0xf7, 0x14, 0x09, 0xf2, 0x9e, 0xfc, 0x9f, 0x0c, 0xd3, 0x7b, 0xea, 0x40, - 0x8e, 0x87, 0xaa, 0x86, 0xbc, 0x97, 0x7b, 0x40, 0xdc, 0x3d, 0xae, 0x36, 0x2c, 0xe1, 0x5d, 0x95, - 0x74, 0xbf, 0x18, 0x66, 0x4b, 0xba, 0x30, 0xe7, 0xc8, 0x0d, 0xfb, 0xd8, 0x64, 0x54, 0x73, 0xe8, - 0x9e, 0x98, 0xff, 0xb3, 0x61, 0xb6, 0xe9, 0x5f, 0x04, 0xc8, 0xd5, 0xbb, 0xbb, 0xec, 0xbe, 0x7c, - 0x98, 0x71, 0x8e, 0xa9, 0x16, 0xde, 0xb3, 0xa5, 0xd3, 0x45, 0xed, 0x39, 0x96, 0x88, 0xb0, 0x12, - 0x2a, 0x5a, 0x05, 0x30, 0xb5, 0xe6, 0x3e, 0x97, 0x13, 0x9d, 0x38, 0xfa, 0x2f, 0x45, 0xb9, 0x7c, - 0x51, 0x8a, 0xdf, 0x8b, 0xc0, 0x8c, 0xdb, 0xcc, 0x30, 0x4d, 0xc7, 0xff, 0x03, 0xea, 0xfd, 0x4a, - 0x96, 0x2d, 0xdb, 0x16, 0xaf, 0xe1, 0x6b, 0xa7, 0xb9, 0x47, 0x50, 0x9a, 0xe5, 0xb1, 0x58, 0x29, - 0x97, 0x24, 0xa6, 0x88, 0x48, 0xfa, 0x13, 0x2d, 0xc3, 0x1c, 0xb5, 0x8f, 0x92, 0xdc, 0xe9, 0xb4, - 0x34, 0xac, 0x4a, 0xec, 0x04, 0x23, 0x46, 0x4f, 0x30, 0x66, 0x69, 0xd6, 0x2a, 0xcb, 0xa9, 0xd1, - 0xd3, 0x8c, 0xfb, 0x90, 0xd9, 0x33, 0x31, 0x7e, 0x81, 0x25, 0xea, 0x75, 0x9d, 0xe6, 0x5c, 0x2b, - 0xcd, 0x18, 0xeb, 0x84, 0x8f, 0x5b, 0xa2, 0xf7, 0x61, 0x96, 0x76, 0x6d, 0xd8, 0xd7, 0x3f, 0xf8, - 0xa8, 0xfc, 0x54, 0x00, 0xe4, 0x97, 0xff, 0xd1, 0x0d, 0x4c, 0x24, 0xf4, 0x81, 0x79, 0x0d, 0x10, - 0x0b, 0x35, 0xb0, 0xa4, 0x0e, 0x36, 0x25, 0x0b, 0x2b, 0x06, 0xbf, 0xe4, 0x2d, 0x88, 0x79, 0x9e, - 0xb3, 0x8d, 0xcd, 0x3a, 0xa5, 0x17, 0xff, 0xa1, 0x00, 0x19, 0xde, 0x27, 0x3b, 0x3a, 0x01, 0xfb, - 0x77, 0x21, 0xda, 0xe4, 0xbb, 0x29, 0xe9, 0x40, 0xbc, 0xe5, 0x3d, 0x34, 0x51, 0x9d, 0x12, 0x49, - 0x59, 0xc2, 0xd2, 0xe9, 0xda, 0x01, 0xa1, 0x75, 0x5e, 0x04, 0x95, 0x9f, 0xa5, 0xd3, 0xb5, 0x51, - 0x1d, 0x66, 0x14, 0xef, 0xa2, 0xbd, 0x44, 0xd8, 0xa3, 0x43, 0x6f, 0x43, 0x04, 0x3e, 0x57, 0x50, - 0x9d, 0x12, 0x73, 0x4a, 0x4f, 0x06, 0x2a, 0xfb, 0x6f, 0x76, 0xc7, 0x86, 0xee, 0x1d, 0xf6, 0xdf, - 0x2a, 0xaf, 0x4e, 0xf9, 0x2e, 0x80, 0xa3, 0x37, 0x21, 0xa1, 0xd2, 0x1b, 0xc3, 0x5c, 0x43, 0x83, - 0x94, 0xa8, 0xe7, 0x92, 0x76, 0x75, 0x4a, 0xe4, 0x1c, 0xe8, 0x21, 0x64, 0xd8, 0x2f, 0x76, 0x55, - 0x96, 0xbb, 0x95, 0xd7, 0x87, 0x4b, 0xf0, 0x59, 0xeb, 0xea, 0x94, 0x98, 0x56, 0x3d, 0x2a, 0xfa, - 0x24, 0xc4, 0x2c, 0x45, 0xd6, 0xf9, 0x96, 0xe3, 0xe5, 0x21, 0x37, 0x23, 0x3d, 0x66, 0x5a, 0x1a, - 0x3d, 0x81, 0x59, 0xba, 0xc1, 0x22, 0xd9, 0xde, 0xc9, 0x33, 0xbd, 0x99, 0xd1, 0x7b, 0xd8, 0xed, - 0xba, 0x5b, 0xc1, 0x17, 0x82, 0xaa, 0x53, 0x62, 0x7e, 0xb7, 0x2f, 0x8b, 0x0c, 0x19, 0xf5, 0x97, - 0x7d, 0x82, 0x53, 0x43, 0x87, 0x2c, 0xf0, 0x8a, 0x0e, 0x19, 0x32, 0xdc, 0x93, 0x81, 0x1e, 0x40, - 0x5a, 0x26, 0xfe, 0x8b, 0x44, 0xaf, 0x3d, 0x14, 0x60, 0xe8, 0xde, 0xf1, 0xc0, 0x4d, 0x8c, 0x2a, - 0xbd, 0xdd, 0xe4, 0x10, 0x3d, 0x41, 0x6d, 0x6c, 0x36, 0x71, 0x21, 0x3d, 0x5a, 0x90, 0xff, 0xb8, - 0xda, 0x15, 0x44, 0x89, 0x68, 0x03, 0xb2, 0xfb, 0x4e, 0xbc, 0x2f, 0x0d, 0x15, 0xc8, 0x0c, 0xdd, - 0x40, 0x0e, 0x88, 0x57, 0xae, 0x4e, 0x89, 0x99, 0x7d, 0x1f, 0x19, 0x2d, 0x43, 0xa4, 0xa9, 0x14, - 0xb2, 0x54, 0xc6, 0xa5, 0x51, 0xd1, 0xb8, 0xd5, 0x29, 0x31, 0xd2, 0x54, 0x08, 0x2a, 0x61, 0x61, - 0x83, 0x87, 0x7a, 0x21, 0x37, 0xd4, 0xc8, 0xf4, 0x06, 0x8b, 0x56, 0xa7, 0x44, 0x1a, 0x52, 0x49, - 0xbe, 0xb7, 0x0d, 0x39, 0x93, 0x9d, 0xf7, 0x3b, 0x51, 0x2d, 0x79, 0x2a, 0xe5, 0x66, 0xb0, 0xa9, - 0x1a, 0x08, 0x6c, 0xa9, 0x4e, 0x89, 0x59, 0xd3, 0x4f, 0x47, 0x5f, 0x86, 0xf9, 0x5e, 0x89, 0x5c, - 0xb9, 0x67, 0x07, 0x2c, 0x57, 0xb0, 0xdc, 0x5e, 0x1d, 0x47, 0xe6, 0x40, 0x26, 0xfa, 0x34, 0xc4, - 0xd9, 0xa8, 0x21, 0x2a, 0x32, 0xe8, 0xa8, 0xa9, 0x6f, 0xc0, 0x58, 0x79, 0x32, 0xdf, 0x6c, 0x7e, - 0xd0, 0x2d, 0xb5, 0x8c, 0x66, 0x61, 0x6e, 0xe8, 0x7c, 0x1b, 0x3c, 0xb8, 0x27, 0xf3, 0xcd, 0xf6, - 0xa8, 0x64, 0xdc, 0x4d, 0x96, 0xc3, 0xcf, 0x45, 0xe7, 0x87, 0x8e, 0x7b, 0xc0, 0xf9, 0x77, 0x95, - 0x06, 0xed, 0x79, 0x64, 0x52, 0x35, 0x93, 0x5d, 0x46, 0x96, 0xe8, 0x34, 0x3e, 0x37, 0xb4, 0x6a, - 0x83, 0x17, 0xac, 0x49, 0xd5, 0x4c, 0x8f, 0x8a, 0x1e, 0x43, 0x9e, 0x5f, 0x38, 0xf4, 0xf6, 0xa3, - 0xce, 0x0f, 0x3d, 0x89, 0x08, 0x3e, 0x48, 0xac, 0x4e, 0x89, 0x33, 0x4a, 0x6f, 0x0e, 0x31, 0x16, - 0x54, 0x9e, 0xa4, 0x78, 0x37, 0x45, 0x0b, 0x85, 0xa1, 0xc6, 0x62, 0xc8, 0xdd, 0x62, 0x62, 0x2c, - 0x94, 0xbe, 0x2c, 0xa2, 0xc6, 0x9a, 0xae, 0xd9, 0xd4, 0xb0, 0x2f, 0x0c, 0x55, 0xe3, 0xde, 0xb7, - 0x50, 0x88, 0x1a, 0x6b, 0x8c, 0x42, 0xd4, 0xd8, 0xe6, 0x87, 0xe6, 0x7c, 0x38, 0x2e, 0x0d, 0x55, - 0xe3, 0xa0, 0xd3, 0x75, 0xa2, 0xc6, 0xb6, 0x9f, 0x4e, 0xd4, 0x98, 0x19, 0x88, 0x3e, 0xb9, 0x2f, - 0x0f, 0x55, 0xe3, 0xa1, 0x37, 0x8c, 0x88, 0x1a, 0xcb, 0x03, 0x99, 0x68, 0x0d, 0x80, 0x79, 0x44, - 0x9a, 0xbe, 0x67, 0x14, 0x2e, 0x0f, 0x5d, 0x7f, 0xfa, 0x8f, 0xcd, 0xc9, 0xfa, 0xd3, 0x72, 0x68, - 0xc4, 0x90, 0x51, 0x1f, 0x5b, 0xa2, 0x9b, 0xdf, 0x85, 0xc5, 0xa1, 0x86, 0x6c, 0x60, 0x0f, 0x9c, - 0x18, 0xb2, 0x03, 0x97, 0x48, 0x16, 0x32, 0xb6, 0x65, 0x54, 0x58, 0x1a, 0xb3, 0x35, 0xe1, 0x5b, - 0xc8, 0x18, 0x07, 0x5a, 0x85, 0x14, 0xf1, 0x14, 0x8e, 0xa8, 0x19, 0xba, 0x32, 0xd4, 0xbb, 0xed, - 0x8b, 0x4a, 0xad, 0x4e, 0x89, 0xc9, 0x67, 0x9c, 0x44, 0x3e, 0xcf, 0x70, 0x7b, 0xa1, 0x38, 0xf4, - 0xf3, 0x3d, 0x3b, 0x35, 0xe4, 0xf3, 0x8c, 0x03, 0x29, 0x70, 0x8e, 0x8d, 0x15, 0xbf, 0x92, 0x64, - 0xf2, 0xbb, 0x3b, 0x85, 0xab, 0x54, 0xd4, 0x50, 0x14, 0x1c, 0x78, 0x3d, 0xaa, 0x3a, 0x25, 0xce, - 0xc9, 0x83, 0xb9, 0x64, 0xc2, 0xf3, 0xa5, 0x87, 0x61, 0xe7, 0xc2, 0xb5, 0xa1, 0x13, 0x3e, 0x60, - 0xb7, 0x81, 0x4c, 0x78, 0xd9, 0x47, 0x66, 0x0b, 0x90, 0x2a, 0x59, 0x16, 0x3b, 0x2a, 0xb9, 0x3e, - 0x62, 0x01, 0xea, 0x43, 0xd0, 0x6c, 0x01, 0x52, 0xeb, 0x8c, 0x93, 0x08, 0x52, 0x5a, 0x58, 0x36, - 0xb9, 0x99, 0xbd, 0x31, 0x54, 0xd0, 0xc0, 0xfb, 0x22, 0x44, 0x90, 0xe2, 0x12, 0xc9, 0x82, 0x6d, - 0x3a, 0x37, 0x95, 0xb9, 0xb7, 0x79, 0x73, 0xe8, 0x82, 0x1d, 0x78, 0xa1, 0x9a, 0x2c, 0xd8, 0x66, - 0x4f, 0x06, 0xfa, 0x1c, 0x4c, 0xf3, 0x6b, 0x9f, 0x85, 0x5b, 0x23, 0x7c, 0x60, 0x3f, 0xac, 0x26, - 0xf3, 0x9a, 0xf3, 0x30, 0x2b, 0xcb, 0x6e, 0x8d, 0xb2, 0xe6, 0xbd, 0x32, 0xc2, 0xca, 0x0e, 0x20, - 0x5a, 0x66, 0x65, 0x3d, 0x32, 0xb1, 0xb2, 0x4c, 0x4f, 0xf9, 0x5a, 0x77, 0x7b, 0xa8, 0x95, 0x1d, - 0x0c, 0x89, 0x25, 0x56, 0xf6, 0x99, 0x47, 0x25, 0x2d, 0xb3, 0x18, 0x10, 0x2b, 0xbc, 0x3a, 0xb4, - 0x65, 0xbd, 0x88, 0x94, 0xb4, 0x8c, 0xf3, 0x90, 0x61, 0x63, 0xd1, 0x5a, 0xac, 0xa7, 0x5f, 0x1b, - 0x3a, 0x6c, 0x03, 0xb8, 0xa5, 0xea, 0xbc, 0x3d, 0xc7, 0x7a, 0xd8, 0x35, 0x54, 0x26, 0xbf, 0x6d, - 0xc5, 0x7b, 0xea, 0xf5, 0xd1, 0x86, 0x2a, 0xe8, 0x22, 0x99, 0x6b, 0xa8, 0x7a, 0x32, 0x69, 0x55, - 0x59, 0x80, 0x39, 0x9d, 0xdf, 0xcb, 0xc3, 0xab, 0xda, 0x1f, 0xf3, 0x4f, 0xab, 0xea, 0x12, 0x4b, - 0xd3, 0x3c, 0x48, 0xef, 0x61, 0x2c, 0x39, 0x93, 0xcf, 0x3f, 0x8c, 0x25, 0x2f, 0xe4, 0x0b, 0x0f, - 0x63, 0xc9, 0x8b, 0xf9, 0x85, 0x87, 0xb1, 0xe4, 0x4b, 0xf9, 0x4b, 0xc5, 0x3f, 0x2d, 0x40, 0xd6, - 0x01, 0x44, 0x0c, 0x5e, 0xac, 0xf8, 0xe1, 0xc5, 0xe5, 0x61, 0xf0, 0x82, 0x43, 0x28, 0x8e, 0x2f, - 0x56, 0xfc, 0xf8, 0xe2, 0xf2, 0x30, 0x7c, 0xe1, 0xf1, 0x10, 0x80, 0xd1, 0x18, 0x06, 0x30, 0x5e, - 0x99, 0x00, 0x60, 0xb8, 0xa2, 0xfa, 0x11, 0xc6, 0xda, 0x20, 0xc2, 0xb8, 0x36, 0x1a, 0x61, 0xb8, - 0xa2, 0x7c, 0x10, 0xe3, 0xad, 0x3e, 0x88, 0x71, 0x65, 0x04, 0xc4, 0x70, 0xf9, 0x1d, 0x8c, 0xb1, - 0x1e, 0x88, 0x31, 0x6e, 0x8c, 0xc3, 0x18, 0xae, 0x9c, 0x1e, 0x90, 0xf1, 0x46, 0x0f, 0xc8, 0x58, - 0x1c, 0x0a, 0x32, 0x5c, 0x6e, 0x86, 0x32, 0xde, 0x1b, 0x8e, 0x32, 0x5e, 0x9d, 0x08, 0x65, 0xb8, - 0xf2, 0x06, 0x61, 0x46, 0x63, 0x18, 0xcc, 0x78, 0x65, 0x02, 0x98, 0xe1, 0x0d, 0x5c, 0x1f, 0xce, - 0xa8, 0x06, 0xe1, 0x8c, 0xeb, 0x63, 0x70, 0x86, 0x2b, 0xcd, 0x0f, 0x34, 0xaa, 0x41, 0x40, 0xe3, - 0xfa, 0x18, 0xa0, 0xd1, 0x27, 0x89, 0x21, 0x8d, 0xcd, 0x60, 0xa4, 0x71, 0x73, 0x2c, 0xd2, 0x70, - 0xa5, 0xf5, 0x42, 0x8d, 0x3b, 0x3e, 0xa8, 0xf1, 0xf2, 0x10, 0xa8, 0xe1, 0xb2, 0x12, 0xac, 0xf1, - 0xf9, 0x01, 0xac, 0x51, 0x1c, 0x85, 0x35, 0x5c, 0x5e, 0x17, 0x6c, 0xbc, 0x33, 0x04, 0x6c, 0xdc, - 0x1a, 0x0f, 0x36, 0x5c, 0x61, 0x7d, 0x68, 0x43, 0x1e, 0x89, 0x36, 0x5e, 0x9f, 0x10, 0x6d, 0xb8, - 0xd2, 0x83, 0xe0, 0xc6, 0x67, 0x7a, 0xe1, 0xc6, 0xd2, 0x70, 0xb8, 0xe1, 0x8a, 0xe1, 0x78, 0x63, - 0x3d, 0x10, 0x6f, 0xdc, 0x18, 0x87, 0x37, 0xbc, 0xb9, 0xe7, 0x07, 0x1c, 0x9b, 0xc1, 0x80, 0xe3, - 0xe6, 0x58, 0xc0, 0xe1, 0x0d, 0x7f, 0x0f, 0xe2, 0x58, 0x0f, 0x44, 0x1c, 0x37, 0xc6, 0x21, 0x0e, - 0xaf, 0x72, 0x7e, 0xc8, 0xf1, 0xee, 0x50, 0xc8, 0x71, 0x7b, 0x12, 0xc8, 0xe1, 0x0a, 0x1d, 0xc0, - 0x1c, 0xef, 0x0d, 0xc7, 0x1c, 0xaf, 0x9e, 0xe2, 0x21, 0x9b, 0x40, 0xd0, 0xf1, 0xf9, 0x01, 0xd0, - 0x51, 0x1c, 0x05, 0x3a, 0x3c, 0x7d, 0x76, 0x50, 0x87, 0x3c, 0x12, 0x23, 0xbc, 0x3e, 0x21, 0x46, - 0xf0, 0x94, 0x2f, 0x00, 0x24, 0x54, 0x02, 0x40, 0xc2, 0xb5, 0xd1, 0x20, 0xc1, 0x5b, 0x42, 0x3c, - 0x94, 0x50, 0x0d, 0x42, 0x09, 0xd7, 0xc7, 0xa0, 0x04, 0xcf, 0x0a, 0xf9, 0x60, 0xc2, 0x5b, 0x7d, - 0x30, 0xe1, 0xca, 0xd8, 0x23, 0x78, 0x1f, 0x4e, 0x28, 0x0d, 0xe2, 0x84, 0xab, 0x23, 0x71, 0x82, - 0x2b, 0xc1, 0x03, 0x0a, 0x6f, 0xf5, 0x01, 0x85, 0x2b, 0x23, 0x80, 0x82, 0x57, 0x01, 0x8e, 0x14, - 0xd4, 0xd1, 0x48, 0x61, 0x79, 0x52, 0xa4, 0xe0, 0x0a, 0x0e, 0x84, 0x0a, 0x9b, 0xc1, 0x50, 0xe1, - 0xe6, 0x84, 0xa7, 0x71, 0x03, 0x58, 0xa1, 0x1a, 0x84, 0x15, 0xae, 0x8f, 0xc1, 0x0a, 0xfe, 0x35, - 0xc4, 0x05, 0x0b, 0xd5, 0x20, 0xb0, 0x70, 0x7d, 0x0c, 0x58, 0xf0, 0x24, 0xf9, 0xd0, 0x42, 0x63, - 0x18, 0x5a, 0x78, 0x65, 0x02, 0xb4, 0xe0, 0xad, 0xbb, 0x7d, 0x70, 0xe1, 0xed, 0x7e, 0xb8, 0x50, - 0x1c, 0x05, 0x17, 0xbc, 0x19, 0xe9, 0xe0, 0x85, 0xcd, 0x60, 0xbc, 0x70, 0x73, 0x2c, 0x5e, 0xf0, - 0x1b, 0x49, 0x1f, 0x60, 0x58, 0x0f, 0x04, 0x0c, 0x37, 0xc6, 0x01, 0x06, 0xcf, 0x48, 0xfa, 0x11, - 0xc3, 0xdb, 0xfd, 0x88, 0xa1, 0x38, 0x0a, 0x31, 0x78, 0x8d, 0x73, 0x20, 0x43, 0x35, 0x08, 0x32, - 0x5c, 0x1f, 0x03, 0x19, 0xbc, 0xc1, 0xf3, 0x61, 0x06, 0x79, 0x24, 0x66, 0x78, 0x7d, 0x42, 0xcc, - 0xd0, 0x67, 0xb8, 0x7a, 0x41, 0x43, 0x35, 0x08, 0x34, 0x5c, 0x1f, 0x03, 0x1a, 0x7c, 0x95, 0x9d, - 0x1c, 0x35, 0x3c, 0x8c, 0x25, 0x2f, 0xe5, 0x5f, 0x2e, 0xfe, 0x45, 0x1c, 0x12, 0x55, 0x27, 0xe8, - 0xc4, 0x77, 0x83, 0x5f, 0x38, 0xcb, 0x0d, 0x7e, 0xb4, 0x46, 0x14, 0x91, 0x4e, 0x67, 0x8e, 0x23, - 0x46, 0x3c, 0x87, 0x31, 0x70, 0xee, 0xe7, 0xb0, 0x9e, 0xe1, 0x4a, 0x0e, 0x7a, 0x03, 0xb2, 0x5d, - 0x0b, 0x9b, 0x52, 0xc7, 0xd4, 0x0c, 0x53, 0xb3, 0x59, 0xa8, 0xa6, 0x50, 0xca, 0x7f, 0x78, 0xbc, - 0x98, 0xd9, 0xb1, 0xb0, 0xb9, 0xcd, 0xe9, 0x62, 0xa6, 0xeb, 0x4b, 0x39, 0xaf, 0xa9, 0xc7, 0x27, - 0x7f, 0x4d, 0xfd, 0x1d, 0xc8, 0x9b, 0x58, 0x56, 0x7b, 0x16, 0x56, 0x76, 0xa9, 0x3d, 0xd8, 0x07, - 0xa0, 0x31, 0xc9, 0x4e, 0x49, 0x7a, 0xb9, 0x7d, 0xc6, 0xec, 0x25, 0xa2, 0xbb, 0x70, 0xae, 0x2d, - 0x1f, 0xb2, 0xb7, 0x1c, 0x1c, 0x5f, 0x85, 0x06, 0xdf, 0x24, 0x69, 0x8c, 0x18, 0x6a, 0xcb, 0x87, - 0xf4, 0x69, 0x76, 0x96, 0x45, 0x1f, 0x65, 0xbd, 0x0e, 0x39, 0x55, 0xb3, 0x6c, 0x4d, 0x57, 0x9c, - 0xe7, 0xa8, 0xd8, 0xed, 0xf7, 0xac, 0x43, 0x65, 0xcf, 0x42, 0xdd, 0x86, 0x59, 0x1e, 0x8d, 0xe7, - 0x3d, 0xd6, 0x4e, 0xbd, 0xf2, 0x24, 0xa9, 0x05, 0xc9, 0xf0, 0x5e, 0xe9, 0x2f, 0xc3, 0x4c, 0x53, - 0xb6, 0xf1, 0x81, 0x7c, 0x24, 0x39, 0x61, 0xd1, 0x69, 0xfa, 0x28, 0xcf, 0x4b, 0x27, 0xc7, 0x8b, - 0xd9, 0x07, 0x2c, 0x6b, 0x20, 0x3a, 0x3a, 0xdb, 0xf4, 0x65, 0xa8, 0x68, 0x15, 0x32, 0xf4, 0xbd, - 0x49, 0x83, 0x3d, 0x6b, 0xca, 0x7d, 0xed, 0x61, 0xc7, 0x2a, 0xfc, 0xf1, 0x53, 0x91, 0xbe, 0x51, - 0xe9, 0xbc, 0x84, 0x7a, 0x13, 0x66, 0x64, 0xeb, 0x48, 0x57, 0x68, 0x0f, 0x63, 0xdd, 0xea, 0x5a, - 0xd4, 0xd9, 0x4e, 0x8a, 0x39, 0x4a, 0x2e, 0x3b, 0x54, 0xfe, 0xaa, 0xd5, 0xaf, 0x09, 0x90, 0xe9, - 0x09, 0x51, 0x7d, 0xab, 0xef, 0xf4, 0xf0, 0x62, 0xb0, 0xa3, 0x3f, 0x2c, 0xee, 0x2a, 0xc9, 0x47, - 0xc0, 0x89, 0x05, 0x59, 0x1c, 0xee, 0x28, 0x52, 0xa8, 0xed, 0x9c, 0x59, 0x3b, 0x6c, 0x6f, 0xc6, - 0x7e, 0xe3, 0x5b, 0x8b, 0x53, 0xc5, 0x9f, 0x45, 0x21, 0xdb, 0x1b, 0x8a, 0x5a, 0xeb, 0xab, 0x57, - 0x90, 0x71, 0xed, 0xe1, 0x58, 0x1e, 0xf1, 0xe8, 0x4c, 0xca, 0x7b, 0xa5, 0x92, 0x55, 0x73, 0x69, - 0xc4, 0x19, 0xa9, 0xbf, 0x9e, 0x1e, 0xe3, 0xc2, 0xf7, 0x23, 0xee, 0xcc, 0x5f, 0x86, 0x38, 0xbd, - 0x16, 0xcc, 0xab, 0x16, 0x74, 0x59, 0xa7, 0x42, 0xf2, 0x45, 0x56, 0x8c, 0x58, 0x8a, 0xc6, 0x99, - 0xde, 0xfa, 0x70, 0x09, 0x67, 0xf8, 0x3f, 0x06, 0xfc, 0x7d, 0x99, 0xf8, 0xe9, 0xde, 0x97, 0x61, - 0x67, 0xa0, 0xad, 0x16, 0x56, 0x6c, 0xfe, 0x8f, 0x2b, 0x9c, 0xff, 0x56, 0x70, 0xad, 0x5f, 0x04, - 0xff, 0x37, 0x17, 0xcb, 0x22, 0xff, 0x37, 0x17, 0xbe, 0xf0, 0x9c, 0x9c, 0x2b, 0x82, 0x4e, 0x2c, - 0x16, 0xc4, 0xc5, 0x87, 0xfa, 0x17, 0x05, 0xc8, 0xd3, 0x69, 0x74, 0x1f, 0x63, 0x35, 0x14, 0x2d, - 0x74, 0x22, 0x87, 0x22, 0x13, 0x47, 0x0e, 0x15, 0x65, 0xc8, 0xb9, 0x75, 0x60, 0xcf, 0xb6, 0x8f, - 0x78, 0xaa, 0xe5, 0x4c, 0xb7, 0x3e, 0x8b, 0xbf, 0x25, 0xc0, 0x9c, 0xfb, 0x8d, 0xb2, 0x7b, 0xd9, - 0xe8, 0x2c, 0xb1, 0x9c, 0x22, 0xfd, 0x97, 0x14, 0x04, 0x31, 0xd2, 0x9b, 0x80, 0x13, 0x69, 0x10, - 0xe2, 0xc7, 0xf1, 0xc0, 0x91, 0xa8, 0xda, 0xa8, 0xd3, 0x7f, 0x56, 0xc1, 0x7e, 0x5b, 0xc5, 0xfb, - 0xbe, 0x1e, 0xa0, 0xca, 0x4a, 0x9a, 0x39, 0x91, 0x56, 0x3b, 0xcd, 0xa4, 0x85, 0x8b, 0x3f, 0x14, - 0xfc, 0x82, 0x9e, 0x13, 0x0f, 0xe4, 0x1e, 0x44, 0x9f, 0xcb, 0xad, 0x51, 0xd1, 0x08, 0x3d, 0x5d, - 0x2f, 0x92, 0xd2, 0xe8, 0x7e, 0xcf, 0x1d, 0xad, 0xc8, 0x70, 0x98, 0x38, 0xd8, 0xa5, 0xfe, 0xbb, - 0x5c, 0xe8, 0xd3, 0x4e, 0x2b, 0xa2, 0xe3, 0x3f, 0xef, 0x9f, 0xa4, 0x6f, 0xc6, 0x3e, 0xf8, 0xd6, - 0xa2, 0x70, 0xbb, 0x0e, 0x73, 0x01, 0x8b, 0x10, 0xca, 0x01, 0xf8, 0x1e, 0x0e, 0xe5, 0xff, 0x11, - 0x63, 0x75, 0x4d, 0xda, 0xd9, 0x2c, 0x6f, 0x6d, 0x6c, 0xd4, 0x1a, 0x8d, 0xca, 0x5a, 0x5e, 0x40, - 0x79, 0xc8, 0xf4, 0x3c, 0x3b, 0xca, 0xff, 0x11, 0xc6, 0xed, 0x4f, 0x01, 0x78, 0x0f, 0x06, 0x13, - 0x59, 0xeb, 0x95, 0x27, 0xd2, 0xe3, 0xd5, 0x47, 0x3b, 0x95, 0x7a, 0x7e, 0x0a, 0x21, 0xc8, 0x95, - 0x56, 0x1b, 0xe5, 0xaa, 0x24, 0x56, 0xea, 0xdb, 0x5b, 0x9b, 0xf5, 0x4a, 0x5e, 0xe0, 0x7c, 0x6b, - 0x90, 0xf1, 0xdf, 0x66, 0x43, 0x73, 0x30, 0x53, 0xae, 0x56, 0xca, 0xeb, 0xd2, 0xe3, 0xda, 0xaa, - 0xf4, 0xce, 0x4e, 0x65, 0xa7, 0x92, 0x9f, 0xa2, 0x55, 0xa3, 0xc4, 0xfb, 0x3b, 0x8f, 0x1e, 0xe5, - 0x05, 0x34, 0x03, 0x69, 0x96, 0xa6, 0x4f, 0x94, 0xe6, 0x23, 0xb7, 0x37, 0x20, 0xed, 0x7b, 0x2c, - 0x86, 0x7c, 0x6e, 0x7b, 0xa7, 0x5e, 0x95, 0x1a, 0xb5, 0x8d, 0x4a, 0xbd, 0xb1, 0xba, 0xb1, 0xcd, - 0x64, 0x50, 0xda, 0x6a, 0x69, 0x4b, 0x6c, 0xe4, 0x05, 0x37, 0xdd, 0xd8, 0xda, 0x29, 0x57, 0xdd, - 0xff, 0xe7, 0x11, 0x4b, 0x46, 0xf3, 0xd1, 0xdb, 0x06, 0x9c, 0x0b, 0xbc, 0x1e, 0x86, 0xd2, 0x30, - 0xbd, 0xa3, 0xd3, 0xc7, 0x2f, 0xf2, 0x53, 0x28, 0xeb, 0xbb, 0x21, 0x96, 0x17, 0x50, 0x92, 0xdd, - 0x03, 0xca, 0x47, 0x50, 0x02, 0x22, 0xf5, 0x7b, 0xf9, 0x28, 0xa9, 0xa6, 0xef, 0x9a, 0x55, 0x3e, - 0x86, 0x52, 0xfc, 0x76, 0x4a, 0x3e, 0x8e, 0x32, 0xde, 0xf5, 0x90, 0x7c, 0xe2, 0xf6, 0x15, 0xf0, - 0x45, 0xcf, 0x23, 0x80, 0xc4, 0x23, 0xd9, 0xc6, 0x96, 0x9d, 0x9f, 0x42, 0xd3, 0x10, 0x5d, 0x6d, - 0xb5, 0xf2, 0xc2, 0xca, 0x77, 0x05, 0x48, 0x3a, 0xaf, 0x53, 0xa2, 0x47, 0x10, 0x67, 0x20, 0x72, - 0x71, 0xf8, 0x92, 0x41, 0xad, 0xce, 0xc2, 0xd2, 0xb8, 0x35, 0xa5, 0x38, 0x85, 0xde, 0xe5, 0xff, - 0x9a, 0x87, 0xe8, 0x0b, 0xba, 0x3a, 0x4a, 0x9b, 0x1c, 0xa9, 0xa3, 0x55, 0x8e, 0xcc, 0x90, 0xe2, - 0xd4, 0x27, 0x84, 0xd2, 0x95, 0x0f, 0xfe, 0xe9, 0xf2, 0xd4, 0x07, 0x27, 0x97, 0x85, 0x1f, 0x9d, - 0x5c, 0x16, 0x7e, 0x72, 0x72, 0x59, 0xf8, 0xc7, 0x93, 0xcb, 0xc2, 0xaf, 0xfc, 0xf3, 0xe5, 0xa9, - 0xf7, 0xa6, 0x39, 0xd7, 0x6e, 0x82, 0xfe, 0x77, 0xa0, 0x7b, 0xff, 0x15, 0x00, 0x00, 0xff, 0xff, - 0x5e, 0x07, 0x4d, 0x1d, 0x22, 0x69, 0x00, 0x00, + 0x75, 0xb6, 0x9a, 0xa4, 0x24, 0xf2, 0xf0, 0xa2, 0x56, 0x49, 0x33, 0xc3, 0xd1, 0xee, 0x8e, 0x34, + 0x9c, 0xeb, 0xce, 0xee, 0x6a, 0x3c, 0x1a, 0xaf, 0xed, 0x7f, 0xd7, 0x5e, 0x5b, 0xa4, 0x38, 0x43, + 0x8e, 0x46, 0x97, 0x69, 0x52, 0xb3, 0x9e, 0xf5, 0xbf, 0x7f, 0xbb, 0xd5, 0x5d, 0xa2, 0xda, 0x43, + 0x76, 0x73, 0xba, 0x9b, 0x23, 0x69, 0x80, 0x1f, 0x3f, 0xf0, 0xe7, 0xc1, 0x81, 0x61, 0x18, 0x79, + 0x08, 0x82, 0xc0, 0x4e, 0x10, 0x03, 0x36, 0x10, 0x20, 0x41, 0x8c, 0x38, 0x30, 0x10, 0x24, 0x48, + 0xe0, 0x87, 0xe4, 0x61, 0x61, 0xf8, 0xc1, 0x08, 0xe2, 0xc0, 0x48, 0x00, 0x21, 0x51, 0x12, 0xd8, + 0xc8, 0x6b, 0x1e, 0x02, 0xec, 0x43, 0x12, 0xd4, 0xa5, 0x2f, 0x24, 0x9b, 0x17, 0xc9, 0xbd, 0xc8, + 0x06, 0x79, 0x62, 0xf7, 0xa9, 0x3a, 0xa7, 0xeb, 0x72, 0xea, 0xd4, 0xf9, 0xaa, 0x4e, 0x15, 0x61, + 0xd6, 0x32, 0x15, 0x75, 0xbf, 0xbd, 0x7b, 0x5b, 0x69, 0xeb, 0xcb, 0x6d, 0xcb, 0x74, 0x4c, 0x34, + 0xab, 0x9a, 0xea, 0x53, 0x4a, 0x5e, 0xe6, 0x89, 0x0b, 0xc8, 0xcd, 0xa5, 0x29, 0x8e, 0xc2, 0xb2, + 0x2d, 0xcc, 0xbb, 0x34, 0x6c, 0x59, 0xa6, 0x65, 0x73, 0xea, 0x79, 0x97, 0xda, 0xc2, 0x8e, 0x12, + 0xc8, 0x5d, 0xb0, 0x1d, 0xd3, 0x52, 0x1a, 0xf8, 0x36, 0x36, 0x1a, 0xba, 0xe1, 0xfe, 0x90, 0x7c, + 0xcf, 0x55, 0x95, 0xe7, 0xb9, 0x32, 0x2c, 0xcf, 0x5d, 0x9e, 0x29, 0xdf, 0x71, 0xf4, 0xe6, 0xed, + 0xfd, 0xa6, 0x7a, 0xdb, 0xd1, 0x5b, 0xd8, 0x76, 0x94, 0x56, 0x9b, 0xa7, 0x2c, 0xd1, 0x14, 0xc7, + 0x52, 0x54, 0xdd, 0x68, 0xdc, 0xb6, 0xb0, 0x6a, 0x5a, 0x1a, 0xd6, 0x64, 0xbb, 0xad, 0x18, 0x6e, + 0x91, 0x1b, 0x66, 0xc3, 0xa4, 0x8f, 0xb7, 0xc9, 0x13, 0xa3, 0x16, 0xfe, 0x1f, 0xa4, 0x24, 0xc5, + 0x68, 0xe0, 0xaa, 0xb1, 0x67, 0xa2, 0xcf, 0x42, 0x42, 0xc3, 0xb6, 0x9a, 0x17, 0x96, 0x84, 0x9b, + 0xe9, 0x95, 0xc2, 0x72, 0x5f, 0x5b, 0x2c, 0xd3, 0xbc, 0x6b, 0xd8, 0x56, 0x2d, 0xbd, 0xed, 0x98, + 0x56, 0x31, 0xf1, 0xc1, 0xf1, 0xe2, 0x84, 0x44, 0xb9, 0xd0, 0x27, 0x61, 0xb2, 0x89, 0x15, 0x1b, + 0xe7, 0x63, 0x94, 0x3d, 0x1f, 0xc2, 0xfe, 0x90, 0xa4, 0x73, 0x26, 0x96, 0xb9, 0xf0, 0x17, 0x02, + 0x64, 0x25, 0xfc, 0xac, 0x83, 0x6d, 0xa7, 0x82, 0x15, 0x0d, 0x5b, 0xe8, 0x22, 0xc4, 0x9f, 0xe2, + 0xa3, 0x7c, 0x7c, 0x49, 0xb8, 0x99, 0x29, 0x4e, 0x7f, 0x78, 0xbc, 0x18, 0x5f, 0xc7, 0x47, 0x12, + 0xa1, 0xa1, 0x25, 0x98, 0xc6, 0x86, 0x26, 0x93, 0xe4, 0x44, 0x77, 0xf2, 0x14, 0x36, 0xb4, 0x75, + 0x7c, 0x84, 0x54, 0x48, 0xda, 0x44, 0x9a, 0xa1, 0xe2, 0xfc, 0xe4, 0x92, 0x70, 0x73, 0xb2, 0x78, + 0xff, 0xc3, 0xe3, 0xc5, 0x52, 0x43, 0x77, 0xf6, 0x3b, 0xbb, 0xcb, 0xaa, 0xd9, 0xba, 0xed, 0x95, + 0x4a, 0xdb, 0xf5, 0x9f, 0x6f, 0xb7, 0x9f, 0x36, 0x6e, 0x0f, 0xe8, 0x81, 0xe5, 0xfa, 0xa1, 0x51, + 0xc3, 0xcf, 0x24, 0x4f, 0xf0, 0x5b, 0x89, 0x5f, 0x7c, 0x7b, 0x51, 0x78, 0x90, 0x48, 0x0a, 0x62, + 0xec, 0x41, 0x22, 0x19, 0x13, 0xe3, 0x85, 0xaf, 0xc7, 0x21, 0x27, 0x61, 0xbb, 0x6d, 0x1a, 0x36, + 0xe6, 0xd5, 0xf8, 0x04, 0xc4, 0x9d, 0x43, 0x83, 0x56, 0x23, 0xbd, 0x72, 0x29, 0xa4, 0x31, 0xea, + 0x96, 0x62, 0xd8, 0x8a, 0xea, 0xe8, 0xa6, 0x21, 0x91, 0xac, 0xe8, 0x33, 0x90, 0xb6, 0xb0, 0xdd, + 0x69, 0x61, 0xda, 0x6d, 0xb4, 0x86, 0xe9, 0x95, 0x0b, 0x21, 0x9c, 0xb5, 0xb6, 0x62, 0x48, 0xc0, + 0xf2, 0x92, 0x67, 0x74, 0x11, 0x92, 0x46, 0xa7, 0x45, 0xda, 0xc5, 0xa6, 0xb5, 0x8e, 0x4b, 0xd3, + 0x46, 0xa7, 0xb5, 0x8e, 0x8f, 0x6c, 0x54, 0x82, 0xb4, 0x45, 0x3a, 0x4d, 0xd6, 0x8d, 0x3d, 0xd3, + 0xce, 0x4f, 0x2d, 0xc5, 0x6f, 0xa6, 0x57, 0x5e, 0x1e, 0xd4, 0xb5, 0x44, 0x0d, 0x78, 0xff, 0x80, + 0xe5, 0x12, 0x6c, 0x54, 0x83, 0x2c, 0x2f, 0x99, 0x85, 0x15, 0xdb, 0x34, 0xf2, 0xd3, 0x4b, 0xc2, + 0xcd, 0xdc, 0xca, 0x72, 0x98, 0x98, 0xae, 0x56, 0x20, 0xaf, 0x9d, 0x16, 0x96, 0x28, 0x97, 0x94, + 0xb1, 0x02, 0x6f, 0x85, 0x27, 0x90, 0x09, 0xa6, 0x22, 0x04, 0x39, 0xa9, 0x5c, 0xdb, 0xd9, 0x28, + 0xcb, 0x3b, 0x9b, 0xeb, 0x9b, 0x5b, 0xef, 0x6e, 0x8a, 0x13, 0x68, 0x1e, 0x44, 0x4e, 0x5b, 0x2f, + 0x3f, 0x91, 0x1f, 0x56, 0x37, 0xaa, 0x75, 0x51, 0x40, 0x17, 0xe1, 0x1c, 0xa7, 0x4a, 0xab, 0x9b, + 0xf7, 0xcb, 0x72, 0x71, 0x6b, 0x67, 0x73, 0x6d, 0x55, 0x7a, 0x22, 0xc6, 0x16, 0x12, 0xbf, 0xfa, + 0x9d, 0x4b, 0x13, 0x85, 0xc7, 0x00, 0xf7, 0xb1, 0xc3, 0xd5, 0x0a, 0x15, 0x61, 0x6a, 0x9f, 0x96, + 0x86, 0x2b, 0xf6, 0x52, 0x68, 0xb1, 0x03, 0x2a, 0x58, 0x4c, 0x92, 0x16, 0xf8, 0xc9, 0xf1, 0xa2, + 0x20, 0x71, 0x4e, 0xd6, 0xe5, 0x85, 0x1f, 0x0a, 0x90, 0xa6, 0x82, 0x59, 0x1d, 0x51, 0xa9, 0x47, + 0xf2, 0xe5, 0x91, 0x0d, 0xd2, 0x2f, 0x1a, 0x2d, 0xc3, 0xe4, 0x73, 0xa5, 0xd9, 0x19, 0x36, 0x6e, + 0x1e, 0x93, 0x74, 0x89, 0x65, 0x43, 0x6f, 0x43, 0x46, 0x37, 0x1c, 0x6c, 0x38, 0x32, 0x63, 0x8b, + 0x8f, 0x60, 0x4b, 0xb3, 0xdc, 0xf4, 0xa5, 0xf0, 0xa7, 0x02, 0xc0, 0x76, 0x27, 0xca, 0xa6, 0x21, + 0xe3, 0x7e, 0xac, 0xf2, 0xbb, 0xe3, 0x9e, 0xd5, 0xe2, 0x3c, 0x4c, 0xe9, 0x46, 0x53, 0x37, 0x58, + 0xf9, 0x93, 0x12, 0x7f, 0x43, 0xf3, 0x30, 0xb9, 0xdb, 0xd4, 0x0d, 0x8d, 0xaa, 0x7f, 0x52, 0x62, + 0x2f, 0xbc, 0xf9, 0x25, 0x48, 0xd3, 0xb2, 0x47, 0xd8, 0xfa, 0x85, 0x6f, 0xc6, 0xe0, 0x5c, 0xc9, + 0x34, 0x34, 0x9d, 0x8c, 0x43, 0xa5, 0xf9, 0xb1, 0x68, 0x9b, 0x37, 0x21, 0x85, 0x0f, 0xdb, 0x63, + 0x76, 0x6f, 0x12, 0x1f, 0xb6, 0xe9, 0x53, 0x78, 0xd3, 0xa1, 0x4f, 0xc2, 0x05, 0xa5, 0xd9, 0x34, + 0x0f, 0x64, 0x7d, 0x4f, 0xd6, 0x4c, 0x6c, 0xcb, 0x86, 0xe9, 0xc8, 0xf8, 0x50, 0xb7, 0x1d, 0x6a, + 0x2a, 0x92, 0xd2, 0x1c, 0x4d, 0xae, 0xee, 0xad, 0x99, 0xd8, 0xde, 0x34, 0x9d, 0x32, 0x49, 0xe2, + 0x0d, 0xfe, 0x3e, 0x9c, 0xef, 0x6d, 0x9b, 0x28, 0xdb, 0xfe, 0xaf, 0x04, 0xc8, 0x55, 0x0d, 0xdd, + 0xf9, 0x58, 0x34, 0xba, 0xd7, 0x7a, 0xf1, 0x60, 0xeb, 0xdd, 0x02, 0x71, 0x4f, 0xd1, 0x9b, 0x5b, + 0x46, 0xdd, 0x6c, 0xed, 0xda, 0x8e, 0x69, 0x60, 0x9b, 0x37, 0x6f, 0x1f, 0x9d, 0xb7, 0xd9, 0x63, + 0x98, 0xf1, 0xea, 0x14, 0x65, 0x63, 0xbd, 0x00, 0xb1, 0x6a, 0xa8, 0x16, 0x6e, 0x61, 0x23, 0xd2, + 0xd6, 0x7a, 0x19, 0x52, 0xba, 0x2b, 0x97, 0xb6, 0x58, 0x5c, 0xf2, 0x09, 0xbc, 0x4e, 0x1d, 0x98, + 0x0d, 0x7c, 0x3b, 0x4a, 0xe3, 0xf7, 0x12, 0xa4, 0x0c, 0x7c, 0x20, 0xfb, 0xfd, 0x15, 0x97, 0x92, + 0x06, 0x3e, 0x60, 0xc6, 0xea, 0x09, 0x64, 0xd7, 0x70, 0x13, 0x3b, 0x38, 0x7a, 0x4b, 0xbe, 0x03, + 0x39, 0x57, 0x74, 0x94, 0x9d, 0xf4, 0xdb, 0x02, 0x20, 0x2e, 0x97, 0xcc, 0x9e, 0x51, 0xf6, 0xd3, + 0x22, 0xf1, 0x0e, 0x9c, 0x8e, 0x65, 0xb0, 0x69, 0x9e, 0x69, 0x29, 0x30, 0x12, 0x9d, 0xe9, 0x7d, + 0x8b, 0x9a, 0x08, 0x5a, 0x54, 0xcf, 0x5b, 0x21, 0x7e, 0xca, 0x01, 0xcc, 0x75, 0x15, 0x2f, 0xda, + 0xae, 0x4c, 0xd0, 0x92, 0xc5, 0x96, 0xe2, 0x41, 0xcf, 0x8c, 0x12, 0x0b, 0xef, 0xc3, 0x6c, 0xa9, + 0x89, 0x15, 0x2b, 0xea, 0x66, 0xe1, 0xdd, 0xf9, 0x04, 0x50, 0x50, 0x7c, 0x94, 0x5d, 0xaa, 0x43, + 0xba, 0xa6, 0x2a, 0xc6, 0x56, 0x9b, 0x18, 0x41, 0x1b, 0xdd, 0x85, 0xf3, 0xb6, 0x63, 0xb6, 0x65, + 0xc5, 0x91, 0x99, 0x5f, 0xb5, 0x6b, 0x76, 0x0c, 0x4d, 0xb1, 0x8e, 0xe8, 0x37, 0x92, 0xd2, 0x1c, + 0x49, 0x5d, 0x75, 0x68, 0x41, 0x8a, 0x3c, 0x89, 0xf4, 0x5d, 0x4b, 0x37, 0x64, 0xe2, 0xfe, 0x34, + 0x1d, 0x9b, 0xeb, 0x39, 0xb4, 0x74, 0x43, 0x62, 0x14, 0x5e, 0x8b, 0xef, 0x08, 0xec, 0x5b, 0x51, + 0xaa, 0xcd, 0x3b, 0x90, 0xb6, 0x55, 0xc5, 0x90, 0xf7, 0x4c, 0xab, 0xa5, 0x38, 0x54, 0x35, 0x72, + 0x2b, 0xaf, 0x84, 0x39, 0x95, 0xaa, 0x62, 0xdc, 0xa3, 0x99, 0x24, 0xb0, 0xbd, 0xe7, 0xa0, 0xf6, + 0x3c, 0x48, 0x24, 0xe3, 0x62, 0xa2, 0xf0, 0x6f, 0x02, 0x64, 0x58, 0x29, 0xa3, 0xd4, 0x9e, 0x37, + 0x21, 0x61, 0x99, 0x07, 0x4c, 0x7b, 0xd2, 0x2b, 0x2f, 0x85, 0x88, 0x58, 0xc7, 0x47, 0x41, 0xb3, + 0x4d, 0xb3, 0xa3, 0x22, 0x70, 0xf7, 0x46, 0xa6, 0xdc, 0xf1, 0x71, 0xb9, 0x81, 0x71, 0x49, 0x44, + 0xc6, 0x0d, 0x98, 0xd9, 0x55, 0x1c, 0x75, 0x9f, 0xf4, 0x0f, 0x2d, 0x24, 0x31, 0xf1, 0xf1, 0x9b, + 0x19, 0x29, 0x47, 0xc9, 0x6e, 0xd1, 0xed, 0xc2, 0xef, 0x09, 0x80, 0x24, 0xfc, 0x1c, 0x5b, 0x36, + 0xfe, 0xf8, 0x77, 0xd3, 0xbf, 0x0b, 0x30, 0xd7, 0x55, 0xd8, 0xff, 0x69, 0xbd, 0xf5, 0x53, 0x01, + 0x2e, 0x94, 0xf6, 0xb1, 0xfa, 0xb4, 0x64, 0x1a, 0xb6, 0x6e, 0x3b, 0xd8, 0x50, 0x8f, 0xa2, 0xec, + 0xb2, 0x97, 0x20, 0x75, 0xa0, 0x3b, 0xfb, 0xb2, 0xa6, 0xef, 0xed, 0xd1, 0x21, 0x9d, 0x94, 0x92, + 0x84, 0xb0, 0xa6, 0xef, 0xed, 0xa1, 0xbb, 0x90, 0x68, 0x99, 0x1a, 0xf3, 0xde, 0x72, 0x2b, 0x8b, + 0x21, 0xe2, 0x69, 0xd1, 0xec, 0x4e, 0x6b, 0xc3, 0xd4, 0xb0, 0x44, 0x33, 0xa3, 0x4b, 0x00, 0x2a, + 0xa1, 0xb6, 0x4d, 0xdd, 0x70, 0xb8, 0x15, 0x0f, 0x50, 0xb8, 0x95, 0xf8, 0x46, 0x02, 0xf2, 0xfd, + 0xf5, 0x8a, 0xb2, 0x77, 0xb7, 0x61, 0x8a, 0x99, 0x2a, 0xde, 0xbf, 0x2b, 0x83, 0x8a, 0x1f, 0x52, + 0x82, 0x65, 0x66, 0xd2, 0x78, 0xc7, 0x71, 0x39, 0x0b, 0x7f, 0x26, 0xc0, 0x14, 0x4b, 0x40, 0x77, + 0x20, 0xc9, 0x01, 0xa9, 0x46, 0xcb, 0x18, 0x2f, 0x9e, 0x3f, 0x39, 0x5e, 0x9c, 0x66, 0xf0, 0x73, + 0xed, 0x43, 0xff, 0x51, 0x9a, 0x66, 0x08, 0x54, 0x23, 0x2d, 0x6d, 0x3b, 0x8a, 0xe5, 0x50, 0xe0, + 0x4f, 0x5a, 0x3a, 0x23, 0x25, 0x29, 0x81, 0x20, 0xfe, 0x07, 0x30, 0x65, 0x3b, 0x8a, 0xd3, 0xb1, + 0x79, 0x5b, 0x9f, 0xaa, 0xb0, 0x35, 0xca, 0x29, 0x71, 0x09, 0x64, 0x0a, 0xd5, 0xb0, 0xa3, 0xe8, + 0x4d, 0xda, 0xf8, 0x29, 0x89, 0xbf, 0x15, 0xbe, 0x25, 0xc0, 0x14, 0xcb, 0x8a, 0x2e, 0xc0, 0x1c, + 0x03, 0x9d, 0xd5, 0xcd, 0xb5, 0x72, 0xbd, 0x2c, 0x6d, 0x54, 0x37, 0x57, 0xeb, 0x65, 0x71, 0x02, + 0x9d, 0x07, 0xe4, 0x26, 0x94, 0xb6, 0x36, 0x6b, 0xd5, 0x5a, 0xbd, 0xbc, 0x49, 0xc0, 0x2a, 0x81, + 0xb0, 0x94, 0x1e, 0xa0, 0xc6, 0xd0, 0x55, 0x58, 0xea, 0xa5, 0xca, 0xb5, 0xfa, 0x6a, 0xbd, 0x26, + 0x97, 0x6b, 0xf5, 0xea, 0xc6, 0x6a, 0xbd, 0xbc, 0x26, 0xc6, 0x87, 0xe4, 0x22, 0x1f, 0x91, 0xa4, + 0x72, 0xa9, 0x2e, 0x26, 0x0a, 0x2f, 0xe0, 0x9c, 0x84, 0x55, 0xb3, 0xd5, 0xee, 0x38, 0x98, 0x94, + 0xd2, 0x8e, 0x52, 0xcb, 0x2f, 0xc0, 0xb4, 0x66, 0x1d, 0xc9, 0x56, 0xc7, 0xe0, 0x3a, 0x3e, 0xa5, + 0x59, 0x47, 0x52, 0xc7, 0xe0, 0xca, 0xf8, 0xc7, 0x02, 0x9c, 0xef, 0xfd, 0x78, 0x94, 0xaa, 0xf8, + 0x45, 0x48, 0x2b, 0x9a, 0x86, 0x35, 0x59, 0xc3, 0x4d, 0x47, 0xe1, 0x1e, 0xfd, 0x9d, 0x80, 0x24, + 0xbe, 0x68, 0xb3, 0xcc, 0x56, 0x6b, 0x96, 0xbd, 0x45, 0x9b, 0x8d, 0xc7, 0xa5, 0x12, 0x2d, 0xcf, + 0x1a, 0x61, 0x74, 0xed, 0x08, 0x95, 0x45, 0x29, 0x05, 0x15, 0x2e, 0x14, 0x71, 0x43, 0x37, 0x82, + 0xcb, 0x30, 0x91, 0xfb, 0x25, 0x32, 0xe4, 0xfb, 0x3f, 0x12, 0xa5, 0x77, 0xf2, 0xdd, 0x04, 0x9c, + 0x2b, 0x1b, 0xda, 0x47, 0x53, 0x09, 0x32, 0x1e, 0x54, 0xb3, 0xd5, 0xd2, 0x1d, 0xb7, 0xef, 0xd9, + 0x1b, 0xfa, 0x5f, 0x90, 0xd4, 0xb0, 0xa2, 0x79, 0xf0, 0x3d, 0xdd, 0x35, 0x55, 0x75, 0x1c, 0xbd, + 0xb9, 0xbc, 0xdf, 0x54, 0x97, 0xeb, 0xee, 0x22, 0xa5, 0xe4, 0x65, 0x47, 0x5f, 0x86, 0x0b, 0xc4, + 0x98, 0x5b, 0x86, 0xd2, 0x94, 0x99, 0x34, 0xd9, 0xb1, 0xf4, 0x46, 0x03, 0x5b, 0x7c, 0xc1, 0xeb, + 0x66, 0x48, 0x39, 0xab, 0x9c, 0xa3, 0x44, 0x19, 0xea, 0x2c, 0xbf, 0x74, 0x4e, 0x0f, 0x23, 0xa3, + 0x2f, 0x78, 0xeb, 0x23, 0x76, 0x5b, 0x31, 0xec, 0xfc, 0x24, 0xb5, 0x61, 0x83, 0xd6, 0xd1, 0xb8, + 0x66, 0xf0, 0x79, 0x89, 0x50, 0x6c, 0x74, 0x9b, 0xb8, 0xda, 0xcf, 0x3a, 0xba, 0x85, 0xe5, 0x3b, + 0x6d, 0x35, 0x3f, 0x45, 0xea, 0x5e, 0xcc, 0x9d, 0x1c, 0x2f, 0x82, 0xc4, 0xc8, 0x77, 0xb6, 0x4b, + 0xc4, 0xf5, 0x66, 0xcf, 0x6d, 0x15, 0xdd, 0x04, 0xd1, 0x30, 0x65, 0x0b, 0xef, 0x59, 0xd8, 0xde, + 0xe7, 0x9f, 0x4d, 0xd2, 0x16, 0xcb, 0x19, 0xa6, 0xc4, 0xc8, 0x4c, 0xf4, 0x79, 0x98, 0x6a, 0x9b, + 0xba, 0x6d, 0x1a, 0xf9, 0x14, 0x6b, 0x51, 0xf6, 0x86, 0x1e, 0x81, 0xa8, 0x1b, 0xf2, 0x5e, 0x53, + 0x6f, 0xec, 0x3b, 0xf2, 0x81, 0xa5, 0x3b, 0xd8, 0xce, 0xcf, 0xd2, 0x82, 0x87, 0xa9, 0x45, 0x8d, + 0xaf, 0x44, 0x6a, 0xef, 0x92, 0x9c, 0xbc, 0x0a, 0x39, 0xdd, 0xb8, 0x47, 0xf9, 0x29, 0xd1, 0xf6, + 0x5c, 0x82, 0x69, 0x31, 0x59, 0xf8, 0x67, 0x01, 0xce, 0xf7, 0xaa, 0x49, 0x94, 0xc3, 0xf4, 0x26, + 0x88, 0xa6, 0x81, 0xe5, 0xf6, 0xbe, 0x62, 0x63, 0xde, 0xad, 0x7c, 0xfe, 0xca, 0x99, 0x06, 0xde, + 0x26, 0x64, 0xd6, 0x49, 0x68, 0x1b, 0x66, 0x6d, 0x47, 0x69, 0xe8, 0x46, 0x43, 0xf6, 0xd6, 0xb0, + 0xe9, 0x42, 0xc4, 0x28, 0x1d, 0xe2, 0xb5, 0x14, 0x39, 0xb7, 0x47, 0xef, 0x72, 0x7a, 0x7e, 0x45, + 0x80, 0xd9, 0x55, 0xad, 0xa5, 0x1b, 0xb5, 0x76, 0x53, 0x8f, 0x14, 0x26, 0x5f, 0x85, 0x94, 0x4d, + 0x64, 0xfa, 0x73, 0x90, 0x0f, 0x71, 0x92, 0x34, 0x65, 0x1d, 0x1f, 0xf9, 0x68, 0x24, 0x58, 0x88, + 0x28, 0xc7, 0xfb, 0xfb, 0xbc, 0x7e, 0x1b, 0xd8, 0xfa, 0x88, 0x70, 0x54, 0x50, 0x7c, 0x94, 0x25, + 0xff, 0x9a, 0x00, 0x17, 0xa9, 0x6c, 0xaa, 0x84, 0x7b, 0xd8, 0xa2, 0x9b, 0x01, 0x51, 0x76, 0xd1, + 0x15, 0x98, 0x72, 0x14, 0xab, 0x81, 0x99, 0xb5, 0x9a, 0x2c, 0xa6, 0x89, 0x33, 0x51, 0x73, 0x4c, + 0x8b, 0x38, 0x13, 0x3c, 0x89, 0xd7, 0x53, 0x81, 0x85, 0xb0, 0xb2, 0x44, 0x59, 0xdf, 0x3f, 0x8a, + 0xf1, 0x6f, 0x94, 0xf6, 0x19, 0x28, 0x6d, 0x37, 0x75, 0x55, 0x89, 0x74, 0x6e, 0x2e, 0x43, 0x5a, + 0xa5, 0xc2, 0x65, 0xe7, 0xa8, 0xcd, 0x96, 0x4f, 0x72, 0x2b, 0x57, 0x43, 0x05, 0xd1, 0x8f, 0xb3, + 0x92, 0xd4, 0x8f, 0xda, 0x98, 0xb8, 0x95, 0xee, 0x33, 0x5a, 0x83, 0x69, 0xd6, 0x38, 0xae, 0x47, + 0x3e, 0x44, 0x04, 0x31, 0x1d, 0x75, 0x9a, 0x99, 0x8f, 0x47, 0x97, 0x15, 0x7d, 0x0e, 0x92, 0xf8, + 0xb0, 0x2d, 0xd3, 0x0d, 0xa4, 0xc4, 0xb8, 0x1b, 0x48, 0xd2, 0x34, 0x3e, 0x6c, 0x93, 0x57, 0xde, + 0x2f, 0xdf, 0x14, 0xe0, 0xa5, 0xd0, 0x46, 0x8b, 0xd2, 0x58, 0x7d, 0x8a, 0x6f, 0x73, 0xc5, 0xc6, + 0x2e, 0x25, 0xcd, 0x5f, 0xf8, 0x7d, 0x57, 0x83, 0x25, 0xdc, 0x34, 0x55, 0xe5, 0x23, 0x58, 0xe3, + 0x09, 0xf4, 0x44, 0xec, 0xcc, 0x3d, 0xd1, 0xa3, 0xe2, 0x3d, 0x85, 0x8d, 0x52, 0xc5, 0x7f, 0x43, + 0x80, 0xb9, 0x0a, 0x56, 0x2c, 0x67, 0x17, 0x2b, 0x4e, 0xfd, 0x30, 0x52, 0xd7, 0xe3, 0x4d, 0x88, + 0x1b, 0xe6, 0x01, 0xef, 0xa3, 0xb1, 0x66, 0x06, 0x92, 0x9f, 0xd7, 0xfd, 0x4b, 0x30, 0xdf, 0x5d, + 0xae, 0x28, 0x6b, 0xfd, 0xfd, 0x38, 0xa4, 0xee, 0x97, 0xa2, 0xac, 0xeb, 0x67, 0xf9, 0xca, 0x19, + 0x1b, 0x7d, 0x61, 0x0a, 0xe9, 0x7d, 0x6f, 0xf9, 0x7e, 0x69, 0x1d, 0x1f, 0xb9, 0xa0, 0x9a, 0x70, + 0xa1, 0x55, 0x48, 0x39, 0xfb, 0xc4, 0xc3, 0x30, 0x9b, 0x1a, 0x1f, 0x79, 0x63, 0xb5, 0x97, 0xcf, + 0x85, 0x9a, 0x70, 0xce, 0x39, 0x34, 0xa8, 0xe3, 0x22, 0x37, 0x54, 0xd9, 0x17, 0x37, 0xd6, 0xc4, + 0xbc, 0x40, 0xc4, 0x9d, 0x1c, 0x2f, 0xa2, 0xfa, 0xa1, 0x41, 0xbc, 0x9c, 0xfb, 0xa5, 0xba, 0x2b, + 0x40, 0x42, 0x0e, 0xa7, 0xa9, 0x1e, 0x6d, 0xe1, 0x29, 0x4c, 0xd2, 0x5a, 0xb8, 0x3b, 0xbd, 0x42, + 0xc8, 0x4e, 0x2f, 0xa9, 0x94, 0xe7, 0x1e, 0xc4, 0x4e, 0x53, 0x29, 0x97, 0xc0, 0x54, 0x81, 0x2b, + 0xc4, 0x23, 0x00, 0xd2, 0x84, 0x51, 0xaa, 0xc1, 0x7f, 0xc4, 0x21, 0xb7, 0xdd, 0xb1, 0xf7, 0x23, + 0xd6, 0xfb, 0x12, 0x40, 0xbb, 0x63, 0xef, 0x63, 0x4b, 0x76, 0x0e, 0x0d, 0x5e, 0xf3, 0x11, 0xbb, + 0xc7, 0x6e, 0xd5, 0x19, 0x5f, 0xfd, 0xd0, 0x40, 0x5b, 0x5c, 0x08, 0x96, 0xfd, 0x2d, 0xe8, 0x5b, + 0x63, 0x80, 0xa6, 0xfa, 0xa1, 0xb1, 0x81, 0x3d, 0xb4, 0xc4, 0x04, 0x62, 0x22, 0xf0, 0xb3, 0x30, + 0x4d, 0x5e, 0x64, 0xc7, 0x3c, 0x8d, 0x86, 0x4d, 0x11, 0x9e, 0xba, 0x89, 0x1e, 0x40, 0x4e, 0xc3, + 0x6d, 0x0b, 0x13, 0x33, 0xa4, 0xc9, 0x64, 0x58, 0x9f, 0xc2, 0xe1, 0xcb, 0xfa, 0xac, 0x9b, 0xe6, + 0x01, 0x7a, 0x1b, 0x52, 0xac, 0x24, 0x64, 0xc6, 0x9b, 0xa2, 0x33, 0x5e, 0x58, 0xf3, 0xf0, 0x9e, + 0xa1, 0x73, 0x5d, 0x92, 0x16, 0x83, 0xcc, 0x74, 0xf3, 0x30, 0xb9, 0x67, 0x5a, 0x2a, 0xa6, 0xfb, + 0xd7, 0x49, 0x89, 0xbd, 0xa0, 0x5b, 0x30, 0xab, 0x1b, 0x6a, 0xb3, 0x63, 0xeb, 0xcf, 0xb1, 0xec, + 0x56, 0x93, 0xb9, 0xe7, 0x33, 0x5e, 0x02, 0x15, 0x68, 0x7a, 0x4e, 0x75, 0x52, 0x4c, 0x15, 0xbe, + 0x25, 0xc0, 0x8c, 0xa7, 0x01, 0x51, 0x4e, 0x50, 0xa5, 0xae, 0xee, 0x3b, 0xbd, 0x0e, 0x90, 0x2e, + 0x2b, 0xfc, 0xb5, 0x00, 0xb3, 0x04, 0x99, 0x3f, 0xa7, 0x2a, 0x11, 0xa5, 0x8a, 0x16, 0x59, 0x64, + 0x43, 0xec, 0x8c, 0x6a, 0x45, 0x63, 0x1d, 0xee, 0xc0, 0xbc, 0xde, 0x22, 0xf3, 0x98, 0xee, 0x34, + 0x8f, 0x38, 0x62, 0x70, 0xb0, 0xbb, 0xf9, 0x36, 0xe7, 0xa7, 0x95, 0xdc, 0x24, 0x3e, 0x92, 0xbf, + 0x4b, 0xd7, 0x60, 0xfd, 0x6a, 0x45, 0xd9, 0xee, 0x55, 0xc8, 0x5a, 0x4c, 0x34, 0xd6, 0x4e, 0xdd, + 0xf4, 0x19, 0x8f, 0x95, 0xb4, 0xfe, 0xf7, 0x62, 0x30, 0xf3, 0xa8, 0x83, 0xad, 0xa3, 0x8f, 0x61, + 0xdb, 0x5f, 0x87, 0x99, 0x03, 0x45, 0x77, 0xe4, 0x3d, 0xd3, 0x92, 0x3b, 0x6d, 0x4d, 0x71, 0xdc, + 0x3d, 0xf8, 0x2c, 0x21, 0xdf, 0x33, 0xad, 0x1d, 0x4a, 0x44, 0x18, 0xd0, 0x53, 0xc3, 0x3c, 0x30, + 0x64, 0x42, 0xa6, 0x80, 0xed, 0xd0, 0xe0, 0x8b, 0xad, 0xc5, 0x4f, 0xff, 0xed, 0xf1, 0xe2, 0xdd, + 0xb1, 0xa2, 0x6a, 0x68, 0x60, 0x52, 0xa7, 0xa3, 0x6b, 0xcb, 0x3b, 0x3b, 0xd5, 0x35, 0x49, 0xa4, + 0x22, 0xdf, 0x65, 0x12, 0xeb, 0x87, 0x86, 0xeb, 0xae, 0x7c, 0x28, 0x80, 0xe8, 0x37, 0x58, 0x94, + 0xbd, 0x5a, 0x86, 0xf4, 0xb3, 0x0e, 0xb6, 0xf4, 0x33, 0xf4, 0x29, 0x70, 0x46, 0x62, 0x02, 0xdf, + 0x83, 0x4c, 0x57, 0x3b, 0xc4, 0x7f, 0xb9, 0x76, 0x48, 0x1f, 0xf8, 0x4d, 0x50, 0xf8, 0xb1, 0x00, + 0x88, 0x56, 0xbe, 0xca, 0xd6, 0xb9, 0x3f, 0x66, 0x0a, 0x73, 0x13, 0x44, 0x1a, 0xe7, 0x26, 0xeb, + 0x7b, 0x72, 0x4b, 0xb7, 0x6d, 0xdd, 0x68, 0x70, 0x8d, 0xc9, 0x51, 0x7a, 0x75, 0x6f, 0x83, 0x51, + 0x79, 0x5f, 0xfe, 0x5f, 0x98, 0xeb, 0xaa, 0x4d, 0x94, 0xbd, 0x79, 0x19, 0x32, 0x7b, 0x66, 0xc7, + 0xd0, 0x64, 0xb6, 0x60, 0xc3, 0x17, 0xa6, 0xd2, 0x94, 0xc6, 0xbe, 0x57, 0xf8, 0x6a, 0x0c, 0xe6, + 0x25, 0x6c, 0x9b, 0xcd, 0xe7, 0x38, 0xfa, 0xf6, 0xdc, 0x02, 0xbe, 0x19, 0x21, 0xff, 0x32, 0xcd, + 0x9a, 0x62, 0x32, 0xd8, 0xd4, 0xda, 0xbd, 0x7e, 0x7d, 0x75, 0xb8, 0x66, 0xf6, 0xaf, 0x58, 0xf3, + 0xf5, 0xa4, 0x44, 0x70, 0x3d, 0x89, 0x77, 0xc4, 0xff, 0x86, 0x73, 0x3d, 0x0d, 0x11, 0xa5, 0x07, + 0xf4, 0x37, 0x31, 0xb8, 0xd8, 0x2d, 0x3e, 0x6a, 0x3c, 0xf4, 0xdf, 0xa3, 0xb1, 0x51, 0x05, 0xb2, + 0x2d, 0xdd, 0x38, 0xdb, 0x7a, 0x56, 0xa6, 0xa5, 0x1b, 0xf5, 0x6e, 0x9f, 0x95, 0x40, 0xb7, 0xb0, + 0x76, 0x8d, 0xb2, 0xef, 0xbe, 0x21, 0x40, 0x26, 0xea, 0x35, 0xa4, 0xb3, 0x05, 0xde, 0xf0, 0x3a, + 0xd7, 0x21, 0xfb, 0x11, 0x2c, 0x3a, 0xfd, 0xae, 0x00, 0xa8, 0x6e, 0x75, 0x0c, 0xe2, 0x3e, 0x3e, + 0x34, 0x1b, 0x51, 0x56, 0x76, 0x1e, 0x26, 0x75, 0x43, 0xc3, 0x87, 0xb4, 0xb2, 0x09, 0x89, 0xbd, + 0x74, 0xed, 0x6e, 0xc5, 0xc7, 0xda, 0xdd, 0xe2, 0xf5, 0x7f, 0x0f, 0xe6, 0xba, 0x0a, 0x1a, 0x65, + 0x2b, 0x7c, 0x2f, 0x06, 0x73, 0xbc, 0x3a, 0x91, 0x2f, 0xba, 0x9d, 0x29, 0xea, 0x17, 0x7d, 0x0e, + 0xa0, 0x6d, 0xe1, 0xe7, 0x32, 0x63, 0x8d, 0x8f, 0xc5, 0x9a, 0x22, 0x1c, 0x94, 0x80, 0xbe, 0x08, + 0x33, 0x64, 0xc0, 0xb5, 0x2d, 0xb3, 0x6d, 0xda, 0x64, 0x5e, 0xb7, 0xc7, 0x83, 0x25, 0xb3, 0x27, + 0xc7, 0x8b, 0xd9, 0x0d, 0xdd, 0xd8, 0xe6, 0x8c, 0xf5, 0x9a, 0x44, 0x46, 0xae, 0xf7, 0xea, 0x3a, + 0x23, 0x3f, 0x15, 0x60, 0xfe, 0x23, 0x5b, 0xa6, 0xfc, 0xaf, 0x68, 0x31, 0x6f, 0x3e, 0x10, 0xe9, + 0x6b, 0xd5, 0xd8, 0x33, 0xa3, 0x5f, 0x3c, 0xfe, 0x86, 0x00, 0xb3, 0x01, 0xf1, 0x51, 0xce, 0xfa, + 0x67, 0x8b, 0x2d, 0xff, 0x12, 0xf1, 0x03, 0x82, 0x6a, 0x1f, 0xe5, 0xa0, 0xfa, 0x57, 0x01, 0xce, + 0x97, 0xd8, 0xbe, 0xa7, 0xbb, 0x95, 0x1f, 0xa5, 0x96, 0xe4, 0x61, 0xfa, 0x39, 0xb6, 0x6c, 0xdd, + 0x64, 0xf3, 0x5e, 0x56, 0x72, 0x5f, 0xd1, 0x02, 0x24, 0x6d, 0x43, 0x69, 0xdb, 0xfb, 0xa6, 0xbb, + 0xc7, 0xe2, 0xbd, 0x7b, 0x61, 0x07, 0x93, 0x67, 0x0f, 0x3b, 0x98, 0x0a, 0x0f, 0x3b, 0xe0, 0x1b, + 0x2c, 0x3f, 0x12, 0xe0, 0x42, 0x5f, 0xad, 0xa3, 0xec, 0xe9, 0xaf, 0x40, 0x5a, 0xe5, 0x82, 0x89, + 0x0d, 0x65, 0x3b, 0x2d, 0x55, 0x92, 0xed, 0x8c, 0x9e, 0xf6, 0xc9, 0xf1, 0x22, 0xb8, 0x45, 0xad, + 0xae, 0xf1, 0x8a, 0x91, 0x67, 0xad, 0xf0, 0x73, 0x80, 0x6c, 0xf9, 0xb0, 0x6d, 0x5a, 0x4e, 0x8d, + 0x39, 0x12, 0x68, 0x0d, 0x92, 0x6d, 0xcb, 0x7c, 0xae, 0xbb, 0x95, 0xc8, 0x85, 0x6e, 0x47, 0x76, + 0xf1, 0x6c, 0xf3, 0xfc, 0x92, 0xc7, 0x89, 0x24, 0x48, 0x3d, 0x34, 0x55, 0xa5, 0x79, 0x4f, 0x6f, + 0xba, 0x1a, 0xbb, 0x3c, 0x4a, 0xcc, 0xb2, 0xc7, 0xb1, 0xad, 0x38, 0xfb, 0xee, 0x00, 0xf6, 0x88, + 0xe8, 0x3e, 0x24, 0x2b, 0x8e, 0xd3, 0x26, 0x89, 0x7c, 0xf4, 0x5f, 0x1b, 0x29, 0x92, 0x30, 0x70, + 0x49, 0x1e, 0x33, 0x92, 0x60, 0xf6, 0xbe, 0x69, 0x36, 0x9a, 0xb8, 0xd4, 0x34, 0x3b, 0x5a, 0xc9, + 0x34, 0xf6, 0xf4, 0x06, 0xb7, 0x9e, 0x57, 0x47, 0x4a, 0xbc, 0x5f, 0xaa, 0x49, 0xfd, 0xec, 0xe8, + 0xf3, 0x90, 0xac, 0xdd, 0xe5, 0xa2, 0x98, 0xef, 0x73, 0x65, 0xa4, 0xa8, 0xda, 0x5d, 0xc9, 0x63, + 0x42, 0x15, 0x48, 0xaf, 0xbe, 0xe8, 0x58, 0x98, 0xcb, 0x98, 0xa2, 0x32, 0xae, 0x8f, 0x94, 0x41, + 0x79, 0xa4, 0x20, 0x2b, 0x7a, 0x04, 0xb9, 0x77, 0x4d, 0xeb, 0x69, 0xd3, 0x54, 0xdc, 0xba, 0x4d, + 0x53, 0x61, 0xaf, 0x8e, 0x14, 0xe6, 0xb2, 0x49, 0x3d, 0x02, 0x16, 0xbe, 0x08, 0xd9, 0xae, 0xce, + 0x41, 0x08, 0x12, 0x6d, 0xd2, 0x0f, 0x02, 0x0d, 0x12, 0xa1, 0xcf, 0xe8, 0x0d, 0x98, 0x36, 0x4c, + 0x0d, 0xbb, 0x3a, 0x9b, 0x2d, 0xce, 0x9f, 0x1c, 0x2f, 0x4e, 0x6d, 0x9a, 0x1a, 0x9b, 0xf6, 0xf9, + 0x93, 0x34, 0x45, 0x32, 0xb9, 0x93, 0xfe, 0xc2, 0x75, 0x48, 0x90, 0x7e, 0x21, 0x83, 0x7d, 0x57, + 0xb1, 0xf1, 0x8e, 0xa5, 0x73, 0x99, 0xee, 0x2b, 0xcf, 0xf7, 0x97, 0x02, 0xc4, 0x6a, 0x77, 0x89, + 0xff, 0xb9, 0xdb, 0x51, 0x9f, 0x62, 0x87, 0xe7, 0xe2, 0x6f, 0xd4, 0x2f, 0xb5, 0xf0, 0x9e, 0xce, + 0x7c, 0x91, 0x94, 0xc4, 0xdf, 0xd0, 0x2b, 0x00, 0x8a, 0xaa, 0x62, 0xdb, 0x96, 0xdd, 0x03, 0x35, + 0x29, 0x29, 0xc5, 0x28, 0xeb, 0xf8, 0x88, 0xb0, 0xd9, 0x58, 0xb5, 0xb0, 0xe3, 0x46, 0xbb, 0xb0, + 0x37, 0xc2, 0xe6, 0xe0, 0x56, 0x5b, 0x76, 0xcc, 0xa7, 0xd8, 0xa0, 0xfd, 0x99, 0x92, 0x52, 0x84, + 0x52, 0x27, 0x04, 0x62, 0x7f, 0xb0, 0xa1, 0xf9, 0xc6, 0x22, 0x25, 0x79, 0xef, 0x44, 0xa4, 0x85, + 0x1b, 0x3a, 0x3f, 0x21, 0x92, 0x92, 0xf8, 0x1b, 0xaf, 0xc6, 0x37, 0x05, 0x88, 0xdf, 0x2f, 0xd5, + 0x4e, 0x5d, 0x0f, 0x04, 0x09, 0xa5, 0xc3, 0xf5, 0x3e, 0x25, 0xd1, 0x67, 0x1a, 0x06, 0xa6, 0x37, + 0x9b, 0x04, 0x8e, 0xb7, 0x2d, 0xf3, 0x2b, 0x58, 0x75, 0x6b, 0x91, 0xe3, 0xe4, 0x6d, 0x46, 0x45, + 0x4b, 0x90, 0x56, 0x2d, 0xac, 0x61, 0xc3, 0xd1, 0x95, 0xa6, 0xcd, 0xab, 0x13, 0x24, 0xf1, 0xc2, + 0x7d, 0x55, 0x80, 0x49, 0xaa, 0x48, 0xe8, 0x65, 0x48, 0xa9, 0xa6, 0xe1, 0x28, 0xba, 0xc1, 0xad, + 0x40, 0x4a, 0xf2, 0x09, 0x03, 0x0b, 0x79, 0x19, 0x32, 0x8a, 0xaa, 0x9a, 0x1d, 0xc3, 0x91, 0x0d, + 0xa5, 0x85, 0x79, 0x61, 0xd3, 0x9c, 0xb6, 0xa9, 0xb4, 0x30, 0x5a, 0x04, 0xf7, 0xd5, 0x3b, 0xc2, + 0x94, 0x92, 0x80, 0x93, 0xbc, 0xed, 0xe3, 0x85, 0x1f, 0x09, 0x90, 0x74, 0x55, 0x90, 0x14, 0xa6, + 0x81, 0x0d, 0x6c, 0x29, 0x8e, 0xe9, 0x15, 0xc6, 0x23, 0xf4, 0xce, 0x12, 0x29, 0x7f, 0x96, 0x98, + 0x87, 0x49, 0x47, 0xd9, 0x6d, 0xba, 0xe5, 0x60, 0x2f, 0x74, 0x01, 0xb4, 0xa9, 0x34, 0xd8, 0x2a, + 0x4e, 0x4a, 0x62, 0x2f, 0xa4, 0x4a, 0x3c, 0xee, 0x90, 0xb5, 0x0e, 0x7f, 0x23, 0xe5, 0x65, 0xa1, + 0x76, 0xbb, 0xb8, 0xa1, 0x1b, 0xb4, 0xb3, 0xe3, 0x12, 0x50, 0x12, 0x8d, 0x6a, 0x41, 0x2f, 0x41, + 0x8a, 0x65, 0xc0, 0x86, 0x46, 0x7b, 0x3c, 0x2e, 0x25, 0x29, 0xa1, 0xec, 0x9e, 0xd9, 0xe0, 0xae, + 0xc1, 0x1f, 0x08, 0x30, 0xcb, 0x62, 0x15, 0x58, 0x8c, 0x5e, 0x74, 0xf3, 0xe4, 0x5b, 0x90, 0xd2, + 0x14, 0x47, 0x61, 0x47, 0xa6, 0x62, 0x43, 0x8f, 0x4c, 0xb9, 0xa6, 0x90, 0xe4, 0xa7, 0xc7, 0xa6, + 0x10, 0x24, 0xc8, 0x33, 0x3b, 0x6a, 0x26, 0xd1, 0x67, 0x7f, 0x1f, 0x3c, 0x58, 0xdc, 0x28, 0xfd, + 0x86, 0x9f, 0xc5, 0xdd, 0x49, 0x27, 0xca, 0x66, 0xf8, 0x02, 0x4c, 0x73, 0x30, 0xcc, 0x1b, 0x61, + 0x69, 0x94, 0xbd, 0x73, 0x77, 0x0d, 0x39, 0x1b, 0x2a, 0x02, 0xb0, 0x20, 0x3b, 0x02, 0x63, 0xc7, + 0x8a, 0xea, 0x71, 0x27, 0x29, 0xca, 0x46, 0xa8, 0x68, 0x13, 0xd2, 0xad, 0xe7, 0xaa, 0x2a, 0xef, + 0xe9, 0x4d, 0x87, 0x07, 0xf4, 0x84, 0x47, 0xb1, 0x6e, 0x3c, 0x2e, 0x95, 0xee, 0xd1, 0x4c, 0x2c, + 0xae, 0xc6, 0x7f, 0x97, 0x80, 0x48, 0x60, 0xcf, 0xe8, 0x75, 0xe0, 0x01, 0xee, 0xb2, 0xed, 0x1e, + 0x57, 0x29, 0x66, 0x4f, 0x8e, 0x17, 0x53, 0x12, 0xa5, 0xd6, 0x6a, 0x75, 0x29, 0xc5, 0x32, 0xd4, + 0x6c, 0x07, 0x5d, 0x81, 0xac, 0xd9, 0xd2, 0x1d, 0xd9, 0x9d, 0xe1, 0xb9, 0x2b, 0x93, 0x21, 0x44, + 0xd7, 0x03, 0x40, 0x75, 0xb8, 0x81, 0x0d, 0x32, 0x16, 0x68, 0x3d, 0x59, 0xf4, 0xb6, 0xac, 0x3b, + 0x6c, 0x3c, 0xc9, 0x66, 0xdb, 0xd1, 0x5b, 0xfa, 0x0b, 0xba, 0xb9, 0xca, 0x37, 0x09, 0xae, 0xb0, + 0xec, 0xa4, 0x7e, 0x34, 0x9e, 0xbb, 0xca, 0xf3, 0x6e, 0x05, 0xb2, 0x72, 0xad, 0xf9, 0xba, 0x00, + 0xd9, 0x62, 0xa7, 0xf9, 0x74, 0xab, 0x5d, 0xeb, 0xb4, 0x5a, 0x8a, 0x75, 0x44, 0x06, 0x08, 0xd3, + 0x4e, 0xfd, 0x05, 0x66, 0xd1, 0x8e, 0x5c, 0xfd, 0xf4, 0x17, 0x98, 0xa8, 0x1f, 0x0f, 0xa2, 0x25, + 0x74, 0x16, 0x21, 0x7b, 0x05, 0xb2, 0x14, 0x48, 0xca, 0xd8, 0x70, 0x2c, 0x1d, 0xb3, 0x75, 0x8a, + 0xb8, 0x94, 0xa1, 0xc4, 0x32, 0xa3, 0xa1, 0x6b, 0x90, 0xb3, 0x8f, 0x6c, 0x07, 0xb7, 0x64, 0x76, + 0xd0, 0x93, 0xa1, 0x9f, 0xb8, 0x94, 0x65, 0x54, 0x89, 0x11, 0x0b, 0x7f, 0x18, 0x87, 0x9c, 0xab, + 0x69, 0x51, 0xba, 0x68, 0x45, 0x98, 0xdc, 0xd3, 0x9b, 0xd8, 0xdd, 0xa3, 0x1e, 0x3c, 0x4d, 0x7b, + 0x61, 0x96, 0x64, 0xe6, 0x74, 0x5d, 0x73, 0xca, 0x1a, 0x85, 0xb6, 0x2d, 0xfc, 0x44, 0x80, 0x04, + 0xf5, 0x8d, 0xee, 0x40, 0x82, 0x0e, 0x7f, 0x61, 0x9c, 0xe1, 0x4f, 0xb3, 0x7a, 0x53, 0x78, 0x2c, + 0x30, 0x85, 0x93, 0xf9, 0x70, 0x5f, 0x79, 0xf3, 0xce, 0x0a, 0xd5, 0xb4, 0x8c, 0xc4, 0xdf, 0x50, + 0x91, 0x46, 0x36, 0x98, 0x96, 0x83, 0x35, 0xee, 0x99, 0x84, 0x0d, 0xae, 0xae, 0x8e, 0x77, 0x4d, + 0x8d, 0xcb, 0x87, 0x2e, 0x42, 0x9c, 0xa8, 0xf0, 0x34, 0xdb, 0xea, 0x3c, 0x39, 0x5e, 0x8c, 0x13, + 0xe5, 0x25, 0x34, 0xe6, 0x58, 0x3f, 0x48, 0x24, 0x13, 0xe2, 0x64, 0xe1, 0x07, 0x09, 0xc8, 0x56, + 0x5b, 0x51, 0x1b, 0x87, 0xd5, 0xee, 0x0e, 0x0b, 0x73, 0x1c, 0xbb, 0x3e, 0x1a, 0xd2, 0x5f, 0x5d, + 0x66, 0x36, 0x7e, 0x3a, 0x33, 0x5b, 0x25, 0x4e, 0x01, 0x3f, 0x9b, 0x4a, 0xbe, 0xff, 0xda, 0xc8, + 0xef, 0xd7, 0xc9, 0x88, 0x93, 0x08, 0x8f, 0x1f, 0x47, 0x4c, 0xf7, 0xba, 0xdf, 0xa1, 0xbe, 0x07, + 0x53, 0x9a, 0xa9, 0xf1, 0x95, 0x66, 0x1a, 0x1b, 0x1a, 0x55, 0x99, 0x43, 0xae, 0x31, 0x9f, 0x81, + 0xb8, 0xa6, 0x0f, 0x6b, 0xd2, 0x30, 0x53, 0x49, 0x58, 0x46, 0x28, 0x4e, 0x22, 0xa8, 0x38, 0x41, + 0xe0, 0xb4, 0xb0, 0x05, 0xe0, 0xd7, 0x0a, 0x2d, 0xc1, 0x94, 0xd9, 0xd4, 0xdc, 0x10, 0xe8, 0x6c, + 0x31, 0x75, 0x72, 0xbc, 0x38, 0xb9, 0xd5, 0xd4, 0xaa, 0x6b, 0xd2, 0xa4, 0xd9, 0xd4, 0xaa, 0x1a, + 0x3d, 0xd2, 0x8b, 0x0f, 0x64, 0x2f, 0x50, 0x25, 0x23, 0x4d, 0x1b, 0xf8, 0xc0, 0x0f, 0x95, 0xf1, + 0x84, 0x13, 0xb5, 0xf9, 0x1d, 0x01, 0x72, 0x6e, 0x0b, 0x46, 0x3b, 0xd2, 0x93, 0x7a, 0x8b, 0x6b, + 0x7e, 0xfc, 0x74, 0x9a, 0xef, 0xf2, 0xf1, 0x83, 0x47, 0x5f, 0x13, 0x60, 0x8e, 0xc5, 0xc4, 0xa9, + 0x8a, 0x43, 0xcc, 0x7c, 0x84, 0xea, 0xfd, 0x2a, 0x88, 0x96, 0x62, 0x68, 0x66, 0x4b, 0x7f, 0x81, + 0xd9, 0xfa, 0x88, 0xcd, 0xb7, 0x05, 0x66, 0x3c, 0x3a, 0x5d, 0x00, 0x70, 0x97, 0x77, 0xfe, 0x45, + 0x80, 0xf9, 0xee, 0xc2, 0x44, 0xd9, 0x68, 0xeb, 0x30, 0x45, 0x97, 0xf6, 0xdc, 0xe1, 0xf6, 0x46, + 0x88, 0x90, 0xb0, 0xaf, 0xb3, 0xa8, 0x23, 0x4f, 0xe1, 0xa9, 0x88, 0x85, 0x2f, 0xc0, 0x24, 0x25, + 0x9f, 0xc1, 0xc6, 0xf1, 0x96, 0x7f, 0x06, 0xb3, 0xab, 0x9a, 0x56, 0xab, 0x71, 0xed, 0x8b, 0xae, + 0xd9, 0x5d, 0xef, 0x29, 0x16, 0xe6, 0x3d, 0x05, 0x3f, 0x19, 0xa5, 0xf7, 0xd4, 0x86, 0x1c, 0x8f, + 0xa7, 0x8d, 0x78, 0x2d, 0x97, 0xc6, 0xdc, 0x72, 0xb5, 0x61, 0x2f, 0xfe, 0x79, 0x4e, 0xef, 0x8b, + 0x51, 0xd6, 0xa4, 0x03, 0x73, 0xae, 0xdc, 0xa8, 0xb7, 0x4d, 0x86, 0x55, 0x87, 0xae, 0x89, 0x05, + 0x3f, 0x1b, 0x65, 0x9d, 0xfe, 0x49, 0x80, 0x5c, 0xad, 0xb3, 0xcb, 0x0e, 0xf5, 0x47, 0x19, 0xe7, + 0x98, 0x6a, 0xe2, 0x3d, 0x47, 0x3e, 0x5d, 0xd4, 0x9e, 0x6b, 0x89, 0x08, 0x2b, 0xa1, 0xa2, 0x55, + 0x00, 0x8b, 0xc6, 0x57, 0x53, 0x39, 0xf1, 0xb1, 0xa3, 0xff, 0x52, 0x94, 0x2b, 0x10, 0xa5, 0xf8, + 0x83, 0x18, 0xcc, 0x78, 0xd5, 0x8c, 0xd2, 0x74, 0xfc, 0x1f, 0xa0, 0xde, 0xaf, 0x6c, 0x3b, 0x8a, + 0x63, 0xf3, 0x12, 0xbe, 0x7e, 0x9a, 0xc3, 0x0e, 0xc5, 0x59, 0x1e, 0x8b, 0x95, 0xf2, 0x48, 0x52, + 0x8a, 0x88, 0xa4, 0x8f, 0x68, 0x19, 0xe6, 0xa8, 0x7d, 0x94, 0x95, 0x76, 0xbb, 0xa9, 0x63, 0x4d, + 0x66, 0x3b, 0x18, 0x09, 0xba, 0x83, 0x31, 0x4b, 0x93, 0x56, 0x59, 0x4a, 0x95, 0xee, 0x66, 0xdc, + 0x83, 0xcc, 0x9e, 0x85, 0xf1, 0x0b, 0x2c, 0x53, 0xaf, 0xeb, 0x34, 0xfb, 0x5a, 0x69, 0xc6, 0x58, + 0x23, 0x7c, 0xdc, 0x12, 0xbd, 0x0f, 0xb3, 0xb4, 0x69, 0xa3, 0x3e, 0xa3, 0xc2, 0x7b, 0xe5, 0xe7, + 0x02, 0xa0, 0xa0, 0xfc, 0x8f, 0xae, 0x63, 0x62, 0x91, 0x77, 0xcc, 0xeb, 0x80, 0x58, 0xa8, 0x81, + 0x2d, 0xb7, 0xb1, 0x25, 0xdb, 0x58, 0x35, 0xf9, 0x49, 0x74, 0x41, 0x12, 0x79, 0xca, 0x36, 0xb6, + 0x6a, 0x94, 0x5e, 0xf8, 0xbb, 0x3c, 0x64, 0x78, 0x9b, 0xec, 0x18, 0x04, 0xec, 0xdf, 0x81, 0x78, + 0x83, 0xaf, 0xa6, 0xa4, 0x43, 0xf1, 0x96, 0x7f, 0x1b, 0x46, 0x65, 0x42, 0x22, 0x79, 0x09, 0x4b, + 0xbb, 0xe3, 0x84, 0x84, 0xd6, 0xf9, 0x11, 0x54, 0x41, 0x96, 0x76, 0xc7, 0x41, 0x35, 0x98, 0x51, + 0xfd, 0xdb, 0x00, 0x64, 0xc2, 0x1e, 0x1f, 0x78, 0x64, 0x23, 0xf4, 0x4e, 0x85, 0xca, 0x84, 0x94, + 0x53, 0xbb, 0x12, 0x50, 0x29, 0x78, 0xfc, 0x3c, 0x31, 0x70, 0xed, 0xb0, 0xf7, 0xe8, 0x7b, 0x65, + 0x22, 0x70, 0x4a, 0x1d, 0xbd, 0x05, 0x53, 0x1a, 0x3d, 0xd6, 0xcc, 0x35, 0x34, 0x4c, 0x89, 0xba, + 0x4e, 0x92, 0x57, 0x26, 0x24, 0xce, 0x81, 0x1e, 0x40, 0x86, 0x3d, 0xb1, 0xf3, 0xbc, 0xdc, 0xad, + 0xbc, 0x36, 0x58, 0x42, 0xc0, 0x5a, 0x57, 0x26, 0xa4, 0xb4, 0xe6, 0x53, 0xd1, 0x27, 0x21, 0x61, + 0xab, 0x8a, 0xc1, 0x97, 0x1c, 0x2f, 0x0d, 0x38, 0xbe, 0xe9, 0x33, 0xd3, 0xdc, 0xe8, 0x09, 0xcc, + 0xd2, 0x05, 0x16, 0xd9, 0xf1, 0x77, 0x9e, 0xe9, 0xf1, 0x91, 0xee, 0xcd, 0x6e, 0xcf, 0xdd, 0x0a, + 0x3f, 0xb5, 0x54, 0x99, 0x90, 0xc4, 0xdd, 0x9e, 0x24, 0xd2, 0x65, 0xd4, 0x5f, 0x0e, 0x08, 0x4e, + 0x0d, 0xec, 0xb2, 0xd0, 0x73, 0x44, 0xa4, 0xcb, 0x70, 0x57, 0x02, 0xba, 0x0f, 0x69, 0x85, 0xf8, + 0x2f, 0x32, 0x3d, 0xf6, 0x90, 0x87, 0x81, 0x6b, 0xc7, 0x7d, 0x27, 0x31, 0x2a, 0xf4, 0x08, 0x96, + 0x4b, 0xf4, 0x05, 0xb5, 0xb0, 0xd5, 0xc0, 0xf9, 0xf4, 0x70, 0x41, 0xc1, 0xed, 0x6a, 0x4f, 0x10, + 0x25, 0xa2, 0x0d, 0xc8, 0xee, 0xbb, 0xf1, 0xbe, 0x34, 0x54, 0x20, 0x33, 0x70, 0x01, 0x39, 0x24, + 0x5e, 0xb9, 0x32, 0x21, 0x65, 0xf6, 0x03, 0x64, 0xb4, 0x0c, 0xb1, 0x86, 0x9a, 0xcf, 0x52, 0x19, + 0x2f, 0x0f, 0x8b, 0xc6, 0xad, 0x4c, 0x48, 0xb1, 0x86, 0x4a, 0x50, 0x09, 0x0b, 0x1b, 0x3c, 0x34, + 0xf2, 0xb9, 0x81, 0x46, 0xa6, 0x3b, 0x58, 0xb4, 0x32, 0x21, 0xd1, 0x90, 0x4a, 0xf2, 0xbd, 0x6d, + 0xc8, 0x59, 0x6c, 0xbf, 0xdf, 0x8d, 0x6a, 0x11, 0xa9, 0x94, 0x1b, 0xe1, 0xa6, 0xaa, 0x2f, 0xb0, + 0xa5, 0x32, 0x21, 0x65, 0xad, 0x20, 0x1d, 0x7d, 0x19, 0xe6, 0xbb, 0x25, 0x72, 0xe5, 0x9e, 0xed, + 0xb3, 0x5c, 0xe1, 0x72, 0xbb, 0x75, 0x1c, 0x59, 0x7d, 0x89, 0xe8, 0xd3, 0x30, 0xc9, 0x7a, 0x0d, + 0x51, 0x91, 0x61, 0x5b, 0x4d, 0x3d, 0x1d, 0xc6, 0xf2, 0x93, 0xf1, 0xe6, 0xf0, 0x8d, 0x6e, 0xb9, + 0x69, 0x36, 0xf2, 0x73, 0x03, 0xc7, 0x5b, 0xff, 0xc6, 0x3d, 0x19, 0x6f, 0x8e, 0x4f, 0x25, 0xfd, + 0x6e, 0xb1, 0x14, 0xbe, 0x2f, 0x3a, 0x3f, 0xb0, 0xdf, 0x43, 0xf6, 0xbf, 0x2b, 0x34, 0x68, 0xcf, + 0x27, 0x93, 0xa2, 0x59, 0xec, 0xc4, 0xb4, 0x4c, 0x87, 0xf1, 0xb9, 0x81, 0x45, 0xeb, 0x3f, 0x05, + 0x4e, 0x8a, 0x66, 0xf9, 0x54, 0xf4, 0x18, 0x44, 0x7e, 0x2a, 0xd2, 0x5f, 0x8f, 0x3a, 0x3f, 0x70, + 0x27, 0x22, 0x7c, 0x23, 0xb1, 0x32, 0x21, 0xcd, 0xa8, 0xdd, 0x29, 0xc4, 0x58, 0x50, 0x79, 0xb2, + 0xea, 0x1f, 0x67, 0xcd, 0xe7, 0x07, 0x1a, 0x8b, 0x01, 0x07, 0xa0, 0x89, 0xb1, 0x50, 0x7b, 0x92, + 0x88, 0x1a, 0xeb, 0x86, 0xee, 0x50, 0xc3, 0xbe, 0x30, 0x50, 0x8d, 0xbb, 0x2f, 0x6c, 0x21, 0x6a, + 0xac, 0x33, 0x0a, 0x51, 0x63, 0x87, 0x6f, 0x9a, 0xf3, 0xee, 0x78, 0x79, 0xa0, 0x1a, 0x87, 0xed, + 0xae, 0x13, 0x35, 0x76, 0x82, 0x74, 0xa2, 0xc6, 0xcc, 0x40, 0xf4, 0xc8, 0x7d, 0x65, 0xa0, 0x1a, + 0x0f, 0x3c, 0x61, 0x44, 0xd4, 0x58, 0xe9, 0x4b, 0x44, 0x6b, 0x00, 0xcc, 0x23, 0xd2, 0x8d, 0x3d, + 0x33, 0x7f, 0x69, 0xe0, 0xfc, 0xd3, 0xbb, 0x6d, 0x4e, 0xe6, 0x9f, 0xa6, 0x4b, 0x23, 0x86, 0x8c, + 0xfa, 0xd8, 0x32, 0x5d, 0xfc, 0xce, 0x2f, 0x0e, 0x34, 0x64, 0x7d, 0x6b, 0xe0, 0xc4, 0x90, 0x1d, + 0x78, 0x44, 0x32, 0x91, 0xb1, 0x25, 0xa3, 0xfc, 0xd2, 0x88, 0xa5, 0x89, 0xc0, 0x44, 0xc6, 0x38, + 0xd0, 0x2a, 0xa4, 0x88, 0xa7, 0x70, 0x44, 0xcd, 0xd0, 0xe5, 0x81, 0xde, 0x6d, 0x4f, 0x54, 0x6a, + 0x65, 0x42, 0x4a, 0x3e, 0xe3, 0x24, 0xf2, 0x79, 0x86, 0xdb, 0xf3, 0x85, 0x81, 0x9f, 0xef, 0x5a, + 0xa9, 0x21, 0x9f, 0x67, 0x1c, 0x48, 0x85, 0x73, 0xac, 0xaf, 0xf8, 0x91, 0x24, 0x8b, 0x9f, 0xdd, + 0xc9, 0x5f, 0xa1, 0xa2, 0x06, 0xa2, 0xe0, 0xd0, 0xe3, 0x51, 0x95, 0x09, 0x69, 0x4e, 0xe9, 0x4f, + 0x25, 0x03, 0x9e, 0x4f, 0x3d, 0x0c, 0x3b, 0xe7, 0xaf, 0x0e, 0x1c, 0xf0, 0x21, 0xab, 0x0d, 0x64, + 0xc0, 0x2b, 0x01, 0x32, 0x9b, 0x80, 0x34, 0xd9, 0xb6, 0xd9, 0x56, 0xc9, 0xb5, 0x21, 0x13, 0x50, + 0x0f, 0x82, 0x66, 0x13, 0x90, 0x56, 0x63, 0x9c, 0x44, 0x90, 0xda, 0xc4, 0x8a, 0xc5, 0xcd, 0xec, + 0xf5, 0x81, 0x82, 0xfa, 0x2e, 0x41, 0x21, 0x82, 0x54, 0x8f, 0x48, 0x26, 0x6c, 0xcb, 0x3d, 0x4e, + 0xcd, 0xbd, 0xcd, 0x1b, 0x03, 0x27, 0xec, 0xd0, 0x53, 0xdf, 0x64, 0xc2, 0xb6, 0xba, 0x12, 0xd0, + 0xe7, 0x60, 0x9a, 0x9f, 0x4d, 0xcd, 0xdf, 0x1c, 0xe2, 0x03, 0x07, 0x61, 0x35, 0x19, 0xd7, 0x9c, + 0x87, 0x59, 0x59, 0x76, 0xb4, 0x95, 0x55, 0xef, 0xd5, 0x21, 0x56, 0xb6, 0x0f, 0xd1, 0x32, 0x2b, + 0xeb, 0x93, 0x89, 0x95, 0x65, 0x7a, 0xca, 0xe7, 0xba, 0x5b, 0x03, 0xad, 0x6c, 0x7f, 0x48, 0x2c, + 0xb1, 0xb2, 0xcf, 0x7c, 0x2a, 0xa9, 0x99, 0xcd, 0x80, 0x58, 0xfe, 0xb5, 0x81, 0x35, 0xeb, 0x46, + 0xa4, 0xa4, 0x66, 0x9c, 0x87, 0x74, 0x1b, 0x8b, 0xd6, 0x62, 0x2d, 0xfd, 0xfa, 0xc0, 0x6e, 0xeb, + 0xc3, 0x2d, 0x15, 0xf7, 0x82, 0x3c, 0xd6, 0xc2, 0x9e, 0xa1, 0xb2, 0xf8, 0x69, 0x2b, 0xde, 0x52, + 0x6f, 0x0c, 0x37, 0x54, 0x61, 0x07, 0xc9, 0x3c, 0x43, 0xd5, 0x95, 0x48, 0x8b, 0xca, 0x02, 0xcc, + 0xe9, 0xf8, 0x5e, 0x1e, 0x5c, 0xd4, 0xde, 0x98, 0x7f, 0x5a, 0x54, 0x8f, 0x58, 0x9c, 0xe6, 0x41, + 0x7a, 0x0f, 0x12, 0xc9, 0x19, 0x51, 0x7c, 0x90, 0x48, 0x5e, 0x10, 0xf3, 0x0f, 0x12, 0xc9, 0x8b, + 0xe2, 0xc2, 0x83, 0x44, 0xf2, 0x25, 0xf1, 0xe5, 0xc2, 0x9f, 0xe4, 0x21, 0xeb, 0x02, 0x22, 0x06, + 0x2f, 0x56, 0x82, 0xf0, 0xe2, 0xd2, 0x20, 0x78, 0xc1, 0x21, 0x14, 0xc7, 0x17, 0x2b, 0x41, 0x7c, + 0x71, 0x69, 0x10, 0xbe, 0xf0, 0x79, 0x08, 0xc0, 0xa8, 0x0f, 0x02, 0x18, 0xaf, 0x8e, 0x01, 0x30, + 0x3c, 0x51, 0xbd, 0x08, 0x63, 0xad, 0x1f, 0x61, 0x5c, 0x1d, 0x8e, 0x30, 0x3c, 0x51, 0x01, 0x88, + 0xf1, 0x76, 0x0f, 0xc4, 0xb8, 0x3c, 0x04, 0x62, 0x78, 0xfc, 0x2e, 0xc6, 0x58, 0x0f, 0xc5, 0x18, + 0xd7, 0x47, 0x61, 0x0c, 0x4f, 0x4e, 0x17, 0xc8, 0x78, 0xb3, 0x0b, 0x64, 0x2c, 0x0e, 0x04, 0x19, + 0x1e, 0x37, 0x43, 0x19, 0xef, 0x0d, 0x46, 0x19, 0xaf, 0x8d, 0x85, 0x32, 0x3c, 0x79, 0xfd, 0x30, + 0xa3, 0x3e, 0x08, 0x66, 0xbc, 0x3a, 0x06, 0xcc, 0xf0, 0x3b, 0xae, 0x07, 0x67, 0x54, 0xc2, 0x70, + 0xc6, 0xb5, 0x11, 0x38, 0xc3, 0x93, 0x16, 0x04, 0x1a, 0x95, 0x30, 0xa0, 0x71, 0x6d, 0x04, 0xd0, + 0xe8, 0x91, 0xc4, 0x90, 0xc6, 0x66, 0x38, 0xd2, 0xb8, 0x31, 0x12, 0x69, 0x78, 0xd2, 0xba, 0xa1, + 0xc6, 0xed, 0x00, 0xd4, 0x78, 0x65, 0x00, 0xd4, 0xf0, 0x58, 0x09, 0xd6, 0xf8, 0x7c, 0x1f, 0xd6, + 0x28, 0x0c, 0xc3, 0x1a, 0x1e, 0xaf, 0x07, 0x36, 0x1e, 0x0d, 0x00, 0x1b, 0x37, 0x47, 0x83, 0x0d, + 0x4f, 0x58, 0x0f, 0xda, 0x50, 0x86, 0xa2, 0x8d, 0x37, 0xc6, 0x44, 0x1b, 0x9e, 0xf4, 0x30, 0xb8, + 0xf1, 0x99, 0x6e, 0xb8, 0xb1, 0x34, 0x18, 0x6e, 0x78, 0x62, 0x38, 0xde, 0x58, 0x0f, 0xc5, 0x1b, + 0xd7, 0x47, 0xe1, 0x0d, 0x7f, 0xec, 0x05, 0x01, 0xc7, 0x66, 0x38, 0xe0, 0xb8, 0x31, 0x12, 0x70, + 0xf8, 0xdd, 0xdf, 0x85, 0x38, 0xd6, 0x43, 0x11, 0xc7, 0xf5, 0x51, 0x88, 0xc3, 0x2f, 0x5c, 0x10, + 0x72, 0xbc, 0x3b, 0x10, 0x72, 0xdc, 0x1a, 0x07, 0x72, 0x78, 0x42, 0xfb, 0x30, 0xc7, 0x7b, 0x83, + 0x31, 0xc7, 0x6b, 0xa7, 0xb8, 0x6d, 0x27, 0x14, 0x74, 0x7c, 0xbe, 0x0f, 0x74, 0x14, 0x86, 0x81, + 0x0e, 0x5f, 0x9f, 0x5d, 0xd4, 0xa1, 0x0c, 0xc5, 0x08, 0x6f, 0x8c, 0x89, 0x11, 0x7c, 0xe5, 0x0b, + 0x01, 0x09, 0xe5, 0x10, 0x90, 0x70, 0x75, 0x38, 0x48, 0xf0, 0xa7, 0x10, 0x1f, 0x25, 0x54, 0xc2, + 0x50, 0xc2, 0xb5, 0x11, 0x28, 0xc1, 0xb7, 0x42, 0x01, 0x98, 0xf0, 0x76, 0x0f, 0x4c, 0xb8, 0x3c, + 0x72, 0x0b, 0x3e, 0x80, 0x13, 0x8a, 0xfd, 0x38, 0xe1, 0xca, 0x50, 0x9c, 0xe0, 0x49, 0xf0, 0x81, + 0xc2, 0xdb, 0x3d, 0x40, 0xe1, 0xf2, 0x10, 0xa0, 0xe0, 0x17, 0x80, 0x23, 0x05, 0x6d, 0x38, 0x52, + 0x58, 0x1e, 0x17, 0x29, 0x78, 0x82, 0x43, 0xa1, 0xc2, 0x66, 0x38, 0x54, 0xb8, 0x31, 0xe6, 0x6e, + 0x5c, 0x1f, 0x56, 0xa8, 0x84, 0x61, 0x85, 0x6b, 0x23, 0xb0, 0x42, 0x70, 0x0e, 0xf1, 0xc0, 0x42, + 0x25, 0x0c, 0x2c, 0x5c, 0x1b, 0x01, 0x16, 0x7c, 0x49, 0x01, 0xb4, 0x50, 0x1f, 0x84, 0x16, 0x5e, + 0x1d, 0x03, 0x2d, 0xf8, 0xf3, 0x6e, 0x0f, 0x5c, 0x78, 0xa7, 0x17, 0x2e, 0x14, 0x86, 0xc1, 0x05, + 0x7f, 0x44, 0xba, 0x78, 0x61, 0x33, 0x1c, 0x2f, 0xdc, 0x18, 0x89, 0x17, 0x82, 0x46, 0x32, 0x00, + 0x18, 0xd6, 0x43, 0x01, 0xc3, 0xf5, 0x51, 0x80, 0xc1, 0x37, 0x92, 0x41, 0xc4, 0xf0, 0x4e, 0x2f, + 0x62, 0x28, 0x0c, 0x43, 0x0c, 0x7e, 0xe5, 0x5c, 0xc8, 0x50, 0x09, 0x83, 0x0c, 0xd7, 0x46, 0x40, + 0x06, 0xbf, 0xf3, 0x02, 0x98, 0x41, 0x19, 0x8a, 0x19, 0xde, 0x18, 0x13, 0x33, 0xf4, 0x18, 0xae, + 0x6e, 0xd0, 0x50, 0x09, 0x03, 0x0d, 0xd7, 0x46, 0x80, 0x86, 0x40, 0x61, 0xc7, 0x47, 0x0d, 0x0f, + 0x12, 0xc9, 0x97, 0xc5, 0x57, 0x0a, 0x7f, 0x3e, 0x09, 0x53, 0x15, 0x37, 0xe8, 0x24, 0x70, 0x82, + 0x5f, 0x38, 0xcb, 0x09, 0x7e, 0xb4, 0x46, 0x14, 0x91, 0x0e, 0x67, 0x8e, 0x23, 0x86, 0x5c, 0x87, + 0xd1, 0xb7, 0xef, 0xe7, 0xb2, 0x9e, 0xe1, 0x48, 0x0e, 0x7a, 0x13, 0xb2, 0x1d, 0x1b, 0x5b, 0x72, + 0xdb, 0xd2, 0x4d, 0x4b, 0x77, 0x58, 0xa8, 0xa6, 0x50, 0x14, 0x3f, 0x3c, 0x5e, 0xcc, 0xec, 0xd8, + 0xd8, 0xda, 0xe6, 0x74, 0x29, 0xd3, 0x09, 0xbc, 0xb9, 0x57, 0xbe, 0x4f, 0x8e, 0x7f, 0xe5, 0xfb, + 0x23, 0x10, 0x2d, 0xac, 0x68, 0x5d, 0x13, 0x2b, 0x3b, 0xd4, 0x1e, 0xee, 0x03, 0xd0, 0x98, 0x64, + 0x37, 0x27, 0x3d, 0xdc, 0x3e, 0x63, 0x75, 0x13, 0xd1, 0x1d, 0x38, 0xd7, 0x52, 0x0e, 0xd9, 0x5d, + 0x0e, 0xae, 0xaf, 0x42, 0x83, 0x6f, 0x92, 0x34, 0x46, 0x0c, 0xb5, 0x94, 0x43, 0x7a, 0x7f, 0x3c, + 0x4b, 0xa2, 0x37, 0xc7, 0x5e, 0x83, 0x9c, 0xa6, 0xdb, 0x8e, 0x6e, 0xa8, 0xee, 0x9d, 0x59, 0xec, + 0xf4, 0x7b, 0xd6, 0xa5, 0xb2, 0xbb, 0xab, 0x6e, 0xc1, 0x2c, 0x8f, 0xc6, 0xf3, 0x6f, 0x94, 0xa7, + 0x5e, 0x79, 0x92, 0x94, 0x82, 0x24, 0xf8, 0x7f, 0x25, 0x50, 0x82, 0x99, 0x86, 0xe2, 0xe0, 0x03, + 0xe5, 0x48, 0x76, 0xc3, 0xa2, 0xd3, 0xf4, 0x52, 0x9e, 0x97, 0x4e, 0x8e, 0x17, 0xb3, 0xf7, 0x59, + 0x52, 0x5f, 0x74, 0x74, 0xb6, 0x11, 0x48, 0xd0, 0xd0, 0x2a, 0x64, 0xe8, 0xa5, 0x98, 0x26, 0xbb, + 0x7b, 0x95, 0xfb, 0xda, 0x83, 0xb6, 0x55, 0xf8, 0x0d, 0xad, 0x12, 0xbd, 0x48, 0xd3, 0xbd, 0xae, + 0xf5, 0x06, 0xcc, 0x28, 0xf6, 0x91, 0xa1, 0xd2, 0x16, 0xc6, 0x86, 0xdd, 0xb1, 0xa9, 0xb3, 0x9d, + 0x94, 0x72, 0x94, 0x5c, 0x72, 0xa9, 0xfc, 0x9e, 0xac, 0x5f, 0x17, 0x20, 0xd3, 0x15, 0xa2, 0xfa, + 0x76, 0xcf, 0xee, 0xe1, 0xc5, 0x70, 0x47, 0x7f, 0x50, 0xdc, 0x55, 0x92, 0xf7, 0x80, 0x1b, 0x0b, + 0xb2, 0x38, 0xd8, 0x51, 0xa4, 0x50, 0xdb, 0xdd, 0xb3, 0x76, 0xd9, 0xde, 0x4a, 0xfc, 0xe6, 0xb7, + 0x17, 0x27, 0x0a, 0xbf, 0x88, 0x43, 0xb6, 0x3b, 0x14, 0xb5, 0xda, 0x53, 0xae, 0x30, 0xe3, 0xda, + 0xc5, 0xb1, 0x3c, 0xe4, 0xd2, 0x99, 0x94, 0x7f, 0x95, 0x26, 0x2b, 0xe6, 0xd2, 0x90, 0x3d, 0xd2, + 0x60, 0x39, 0x7d, 0xc6, 0x85, 0x1f, 0xc6, 0xbc, 0x91, 0xbf, 0x0c, 0x93, 0xf4, 0x58, 0x30, 0x2f, + 0x5a, 0xd8, 0x61, 0x9d, 0x32, 0x49, 0x97, 0x58, 0x36, 0x62, 0x29, 0xea, 0x67, 0xba, 0xeb, 0xc3, + 0x23, 0x9c, 0xe1, 0xcf, 0x16, 0xf8, 0xfd, 0x32, 0x93, 0xa7, 0xbb, 0x5f, 0x86, 0xed, 0x81, 0x36, + 0x9b, 0x58, 0x75, 0xf8, 0xbf, 0x6b, 0xb8, 0x7f, 0xa9, 0x70, 0xb5, 0x57, 0x04, 0xff, 0x2f, 0x8e, + 0x65, 0x89, 0xff, 0x17, 0x47, 0x20, 0x3c, 0x27, 0xe7, 0x89, 0xa0, 0x03, 0x8b, 0x05, 0x71, 0xf1, + 0xae, 0xfe, 0xff, 0x02, 0x88, 0x74, 0x18, 0xdd, 0xc3, 0x58, 0x8b, 0x44, 0x0b, 0xdd, 0xc8, 0xa1, + 0xd8, 0xd8, 0x91, 0x43, 0x05, 0x05, 0x72, 0x5e, 0x19, 0xd8, 0xdd, 0xf2, 0x43, 0xae, 0x6a, 0x39, + 0xd3, 0xa9, 0xcf, 0xc2, 0x6f, 0x09, 0x30, 0xe7, 0x7d, 0xa3, 0xe4, 0x1d, 0x36, 0x3a, 0x4b, 0x2c, + 0xa7, 0x44, 0xff, 0x37, 0x83, 0x20, 0x46, 0x7a, 0x12, 0x70, 0x2c, 0x0d, 0x42, 0x7c, 0x3b, 0x1e, + 0x38, 0x12, 0xd5, 0xea, 0x35, 0xfa, 0x8f, 0x1a, 0xec, 0xd9, 0x2e, 0xdc, 0x0b, 0xb4, 0x00, 0x55, + 0x56, 0x52, 0xcd, 0xb1, 0xb4, 0xda, 0xad, 0x26, 0xcd, 0x5c, 0xf8, 0xb1, 0x10, 0x14, 0xf4, 0x9c, + 0x78, 0x20, 0x77, 0x21, 0xfe, 0x5c, 0x69, 0x0e, 0x8b, 0x46, 0xe8, 0x6a, 0x7a, 0x89, 0xe4, 0x46, + 0xf7, 0xba, 0xce, 0x68, 0xc5, 0x06, 0xc3, 0xc4, 0xfe, 0x26, 0x0d, 0x9e, 0xe5, 0x42, 0x9f, 0x76, + 0x6b, 0x11, 0x1f, 0xfd, 0xf9, 0xe0, 0x20, 0x7d, 0x2b, 0xf1, 0xc1, 0xb7, 0x17, 0x85, 0x5b, 0x35, + 0x98, 0x0b, 0x99, 0x84, 0x50, 0x0e, 0x20, 0x70, 0xbb, 0x29, 0xff, 0xdb, 0x8e, 0xd5, 0x35, 0x79, + 0x67, 0xb3, 0xb4, 0xb5, 0xb1, 0x51, 0xad, 0xd7, 0xcb, 0x6b, 0xa2, 0x80, 0x44, 0xc8, 0x74, 0xdd, + 0x8d, 0xca, 0xff, 0xad, 0xe3, 0xd6, 0xa7, 0x00, 0xfc, 0x5b, 0x8d, 0x89, 0xac, 0xf5, 0xf2, 0x13, + 0xf9, 0xf1, 0xea, 0xc3, 0x9d, 0x72, 0x4d, 0x9c, 0x40, 0x08, 0x72, 0xc5, 0xd5, 0x7a, 0xa9, 0x22, + 0x4b, 0xe5, 0xda, 0xf6, 0xd6, 0x66, 0xad, 0x2c, 0x0a, 0x9c, 0x6f, 0x0d, 0x32, 0xc1, 0xd3, 0x6c, + 0x68, 0x0e, 0x66, 0x4a, 0x95, 0x72, 0x69, 0x5d, 0x7e, 0x5c, 0x5d, 0x95, 0x1f, 0xed, 0x94, 0x77, + 0xca, 0xe2, 0x04, 0x2d, 0x1a, 0x25, 0xde, 0xdb, 0x79, 0xf8, 0x50, 0x14, 0xd0, 0x0c, 0xa4, 0xd9, + 0x3b, 0xbd, 0x47, 0x55, 0x8c, 0xdd, 0xda, 0x80, 0x74, 0xe0, 0xb2, 0x18, 0xf2, 0xb9, 0xed, 0x9d, + 0x5a, 0x45, 0xae, 0x57, 0x37, 0xca, 0xb5, 0xfa, 0xea, 0xc6, 0x36, 0x93, 0x41, 0x69, 0xab, 0xc5, + 0x2d, 0xa9, 0x2e, 0x0a, 0xde, 0x7b, 0x7d, 0x6b, 0xa7, 0x54, 0xf1, 0xfe, 0x74, 0x24, 0x91, 0x8c, + 0x8b, 0xf1, 0x5b, 0x26, 0x9c, 0x0b, 0x3d, 0x1e, 0x86, 0xd2, 0x30, 0xbd, 0x63, 0xd0, 0xcb, 0x2f, + 0xc4, 0x09, 0x94, 0x0d, 0x9c, 0x10, 0x13, 0x05, 0x94, 0x64, 0xe7, 0x80, 0xc4, 0x18, 0x9a, 0x82, + 0x58, 0xed, 0xae, 0x18, 0x27, 0xc5, 0x0c, 0x1c, 0xb3, 0x12, 0x13, 0x28, 0xc5, 0x4f, 0xa7, 0x88, + 0x93, 0x28, 0xe3, 0x1f, 0x0f, 0x11, 0xa7, 0x6e, 0x5d, 0x86, 0x40, 0xf4, 0x3c, 0x02, 0x98, 0x7a, + 0xa8, 0x38, 0xd8, 0x76, 0xc4, 0x09, 0x34, 0x0d, 0xf1, 0xd5, 0x66, 0x53, 0x14, 0x56, 0xbe, 0x2f, + 0x40, 0xd2, 0xbd, 0x42, 0x13, 0x3d, 0x84, 0x49, 0x06, 0x22, 0x17, 0x07, 0x4f, 0x19, 0xd4, 0xea, + 0x2c, 0x2c, 0x8d, 0x9a, 0x53, 0x0a, 0x13, 0xe8, 0x5d, 0xfe, 0xff, 0x41, 0x44, 0x5f, 0xd0, 0x95, + 0x61, 0xda, 0xe4, 0x4a, 0x1d, 0xae, 0x72, 0x64, 0x84, 0x14, 0x26, 0x3e, 0x21, 0x14, 0x2f, 0x7f, + 0xf0, 0x0f, 0x97, 0x26, 0x3e, 0x38, 0xb9, 0x24, 0xfc, 0xe4, 0xe4, 0x92, 0xf0, 0xb3, 0x93, 0x4b, + 0xc2, 0xdf, 0x9f, 0x5c, 0x12, 0x7e, 0xed, 0x1f, 0x2f, 0x4d, 0xbc, 0x37, 0xcd, 0xb9, 0x76, 0xa7, + 0xe8, 0x5f, 0x18, 0xdd, 0xfd, 0xcf, 0x00, 0x00, 0x00, 0xff, 0xff, 0x00, 0x8d, 0x0e, 0xf9, 0xc7, + 0x69, 0x00, 0x00, } diff --git a/pkg/roachpb/api.proto b/pkg/roachpb/api.proto index e2d28b30ad93..c891d6e6acdf 100644 --- a/pkg/roachpb/api.proto +++ b/pkg/roachpb/api.proto @@ -576,7 +576,28 @@ message EndTransactionRequest { // are spans which must be resolved on txn completion. Note that these // spans may be condensed to cover aggregate spans if the keys written // by the transaction exceeded a size threshold. + // + // The set logically extends to include the keys of all writes in the + // in-flight write set. However, those keys are not stored in this set + // to avoid duplication. This means that elements that are removed from + // that set should be merged into this one. + // + // The slice is maintained in sorted order and all spans are maximally + // merged such that no two spans here overlap each other. repeated Span intent_spans = 5 [(gogoproto.nullable) = false]; + // Set of in-flight intent writes that have been issued by the transaction but + // which may not have succeeded yet. If any promised writes are provided, a + // committing EndTransaction request will move a PENDING transaction to the + // STAGING status instead of the COMMITTED status. These in-flight writes must + // then all be confirmed as successful before the transaction can be moved + // from STAGING to COMMITTED. For more, see txnCommitter. + // + // The slice is maintained in sorted order by sequence number. This provides + // O(log n) access to individual writes in this set based on their sequence + // number. See SequencedWriteBySeq.Find and its uses. The set can contain + // multiple SequencedWrites with the same key, but all sequence numbers are + // unique. + repeated SequencedWrite in_flight_writes = 17 [(gogoproto.nullable) = false]; // Requires that the transaction completes as a 1 phase commit. This // guarantees that all writes are to the same range and that no // intents are left in the event of an error. @@ -609,6 +630,9 @@ message EndTransactionResponse { // This means that all writes which were part of the transaction // were written as a single, atomic write batch to just one range. bool one_phase_commit = 4; + // The commit timestamp of the STAGING transaction record written + // by the request. Only set if the transaction record was staged. + util.hlc.Timestamp staging_timestamp = 5 [(gogoproto.nullable) = false]; } // An AdminSplitRequest is the argument to the AdminSplit() method. The diff --git a/pkg/roachpb/data.go b/pkg/roachpb/data.go index 15c447506987..71ca986651a4 100644 --- a/pkg/roachpb/data.go +++ b/pkg/roachpb/data.go @@ -31,6 +31,7 @@ import ( "github.com/cockroachdb/apd" "github.com/cockroachdb/cockroach/pkg/storage/engine/enginepb" + "github.com/cockroachdb/cockroach/pkg/util" "github.com/cockroachdb/cockroach/pkg/util/bitarray" "github.com/cockroachdb/cockroach/pkg/util/duration" "github.com/cockroachdb/cockroach/pkg/util/encoding" @@ -1787,3 +1788,37 @@ func (s observedTimestampSlice) update( copy(cpy[i+1:], s[i:]) return cpy } + +// SequencedWriteBySeq implements sorting of a slice of SequencedWrites +// by sequence number. +type SequencedWriteBySeq []SequencedWrite + +// Len implements sort.Interface. +func (s SequencedWriteBySeq) Len() int { return len(s) } + +// Less implements sort.Interface. +func (s SequencedWriteBySeq) Less(i, j int) bool { return s[i].Sequence < s[j].Sequence } + +// Swap implements sort.Interface. +func (s SequencedWriteBySeq) Swap(i, j int) { s[i], s[j] = s[j], s[i] } + +var _ sort.Interface = SequencedWriteBySeq{} + +// Find searches for the index of the SequencedWrite with the provided +// sequence number. Returns -1 if no corresponding write is found. +func (s SequencedWriteBySeq) Find(seq enginepb.TxnSeq) int { + if util.RaceEnabled { + if !sort.IsSorted(s) { + panic("SequencedWriteBySeq must be sorted") + } + } + if i := sort.Search(len(s), func(i int) bool { + return s[i].Sequence >= seq + }); i < len(s) && s[i].Sequence == seq { + return i + } + return -1 +} + +// Silence unused warning. +var _ = (SequencedWriteBySeq{}).Find diff --git a/pkg/sql/row/fetcher_mvcc_test.go b/pkg/sql/row/fetcher_mvcc_test.go index 5f011c8779e2..9dc33710c387 100644 --- a/pkg/sql/row/fetcher_mvcc_test.go +++ b/pkg/sql/row/fetcher_mvcc_test.go @@ -28,32 +28,44 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/sqlbase" "github.com/cockroachdb/cockroach/pkg/storage" "github.com/cockroachdb/cockroach/pkg/storage/engine" + "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" "github.com/cockroachdb/cockroach/pkg/testutils/sqlutils" "github.com/cockroachdb/cockroach/pkg/util" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/pkg/errors" ) func slurpUserDataKVs(t testing.TB, e engine.Engine) []roachpb.KeyValue { t.Helper() + // Scan meta keys directly from engine. We put this in a retry loop + // because the application of all of a transactions committed writes + // is not always synchronous with it committing. var kvs []roachpb.KeyValue - it := e.NewIterator(engine.IterOptions{UpperBound: roachpb.KeyMax}) - defer it.Close() - for it.Seek(engine.MVCCKey{Key: keys.UserTableDataMin}); ; it.NextKey() { - ok, err := it.Valid() - if err != nil { - t.Fatal(err) - } - if !ok { - break + testutils.SucceedsSoon(t, func() error { + kvs = nil + it := e.NewIterator(engine.IterOptions{UpperBound: roachpb.KeyMax}) + defer it.Close() + for it.Seek(engine.MVCCKey{Key: keys.UserTableDataMin}); ; it.NextKey() { + ok, err := it.Valid() + if err != nil { + t.Fatal(err) + } + if !ok { + break + } + if !it.UnsafeKey().IsValue() { + return errors.Errorf("found intent key %v", it.UnsafeKey()) + } + kvs = append(kvs, roachpb.KeyValue{ + Key: it.Key().Key, + Value: roachpb.Value{RawBytes: it.Value(), Timestamp: it.UnsafeKey().Timestamp}, + }) } - kvs = append(kvs, roachpb.KeyValue{ - Key: it.Key().Key, - Value: roachpb.Value{RawBytes: it.Value(), Timestamp: it.UnsafeKey().Timestamp}, - }) - } + return nil + }) return kvs } @@ -110,6 +122,7 @@ func TestRowFetcherMVCCMetadata(t *testing.T) { RowLastModified string } kvsToRows := func(kvs []roachpb.KeyValue) []rowWithMVCCMetadata { + t.Helper() for _, kv := range kvs { log.Info(ctx, kv.Key, kv.Value.Timestamp, kv.Value.PrettyPrint()) } diff --git a/pkg/sql/show_test.go b/pkg/sql/show_test.go index f9e86ed002c7..d7c300b0d5f7 100644 --- a/pkg/sql/show_test.go +++ b/pkg/sql/show_test.go @@ -1254,6 +1254,7 @@ func TestLintClusterSettingNames(t *testing.T) { "kv.raft_log.disable_synchronization_unsafe": `kv.raft_log.disable_synchronization_unsafe: use .enabled for booleans`, "kv.range_merge.queue_enabled": `kv.range_merge.queue_enabled: use ".enabled" instead of "_enabled"`, "kv.range_split.by_load_enabled": `kv.range_split.by_load_enabled: use ".enabled" instead of "_enabled"`, + "kv.transaction.parallel_commits_enabled": `kv.transaction.parallel_commits_enabled: use ".enabled" instead of "_enabled"`, "kv.transaction.write_pipelining_enabled": `kv.transaction.write_pipelining_enabled: use ".enabled" instead of "_enabled"`, "server.clock.forward_jump_check_enabled": `server.clock.forward_jump_check_enabled: use ".enabled" instead of "_enabled"`, "sql.defaults.experimental_optimizer_mutations": `sql.defaults.experimental_optimizer_mutations: use .enabled for booleans`, diff --git a/pkg/sql/upsert_test.go b/pkg/sql/upsert_test.go index 535945674e4d..0c2f27272d4c 100644 --- a/pkg/sql/upsert_test.go +++ b/pkg/sql/upsert_test.go @@ -44,6 +44,10 @@ func TestUpsertFastPath(t *testing.T) { case roachpb.Scan: atomic.AddUint64(&scans, 1) case roachpb.EndTransaction: + if filterArgs.Hdr.Txn.Status == roachpb.STAGING { + // Ignore async explicit commits. + return nil + } atomic.AddUint64(&endTxn, 1) } } diff --git a/pkg/storage/abortspan/abortspan.go b/pkg/storage/abortspan/abortspan.go index cdddbf3868c9..29016601eff7 100644 --- a/pkg/storage/abortspan/abortspan.go +++ b/pkg/storage/abortspan/abortspan.go @@ -97,7 +97,6 @@ func (sc *AbortSpan) ClearData(e engine.Engine) error { func (sc *AbortSpan) Get( ctx context.Context, e engine.Reader, txnID uuid.UUID, entry *roachpb.AbortSpanEntry, ) (bool, error) { - // Pull response from disk and read into reply if available. key := keys.AbortSpanKey(sc.rangeID, txnID) ok, err := engine.MVCCGetProto(ctx, e, key, hlc.Timestamp{}, entry, engine.MVCCGetOptions{}) diff --git a/pkg/storage/addressing_test.go b/pkg/storage/addressing_test.go index dc34c4308101..ae5153a73afc 100644 --- a/pkg/storage/addressing_test.go +++ b/pkg/storage/addressing_test.go @@ -160,11 +160,31 @@ func TestUpdateRangeAddressing(t *testing.T) { if err := txn.Commit(ctx); err != nil { t.Fatal(err) } - // Scan meta keys directly from engine. - kvs, _, _, err := engine.MVCCScan(ctx, store.Engine(), keys.MetaMin, keys.MetaMax, math.MaxInt64, hlc.MaxTimestamp, engine.MVCCScanOptions{}) - if err != nil { - t.Fatal(err) - } + // Scan meta keys directly from engine. We put this in a retry loop + // because the application of all of a transactions committed writes + // is not always synchronous with it committing. Cases where the + // application of a write is asynchronous are: + // - the write is on a different range than the transaction's record. + // Intent resolution will be asynchronous. + // - the transaction performed a parallel commit. Explicitly committing + // the transaction will be asynchronous. + // - [not yet implemented] the corresponding Raft log entry is committed + // but not applied before acknowledging the write. Applying the write + // to RocksDB will be asynchronous. + var kvs []roachpb.KeyValue + testutils.SucceedsSoon(t, func() error { + var err error + kvs, _, _, err = engine.MVCCScan(ctx, store.Engine(), keys.MetaMin, keys.MetaMax, + math.MaxInt64, hlc.MaxTimestamp, engine.MVCCScanOptions{}) + if err != nil { + // Wait for the intent to be resolved. + if _, ok := err.(*roachpb.WriteIntentError); ok { + return err + } + t.Fatal(err) + } + return nil + }) metas := metaSlice{} for _, kv := range kvs { scannedDesc := &roachpb.RangeDescriptor{} diff --git a/pkg/storage/batcheval/cmd_begin_transaction.go b/pkg/storage/batcheval/cmd_begin_transaction.go index a88345b97d82..54ac18359a6d 100644 --- a/pkg/storage/batcheval/cmd_begin_transaction.go +++ b/pkg/storage/batcheval/cmd_begin_transaction.go @@ -65,7 +65,7 @@ func BeginTransaction( h := cArgs.Header reply := resp.(*roachpb.BeginTransactionResponse) - if err := VerifyTransaction(h, args); err != nil { + if err := VerifyTransaction(h, args, roachpb.PENDING); err != nil { return result.Result{}, err } key := keys.TransactionKey(h.Txn.Key, h.Txn.ID) @@ -88,7 +88,9 @@ func BeginTransaction( case roachpb.ABORTED: // Check whether someone has come in ahead and already aborted the // txn. - return result.Result{}, roachpb.NewTransactionAbortedError(roachpb.ABORT_REASON_ABORTED_RECORD_FOUND) + return result.Result{}, roachpb.NewTransactionAbortedError( + roachpb.ABORT_REASON_ABORTED_RECORD_FOUND, + ) case roachpb.PENDING: if h.Txn.Epoch > existingTxn.Epoch { @@ -106,6 +108,12 @@ func BeginTransaction( return result.Result{}, nil } + case roachpb.STAGING: + // NB: we could support this case, but there isn't a reason to. No + // cluster that's performing parallel commits should still be sending + // BeginTransaction requests. + fallthrough + case roachpb.COMMITTED: return result.Result{}, roachpb.NewTransactionStatusError( fmt.Sprintf("BeginTransaction can't overwrite %s", existingTxn), diff --git a/pkg/storage/batcheval/cmd_end_transaction.go b/pkg/storage/batcheval/cmd_end_transaction.go index ea327113ce78..7b9d0725f3ef 100644 --- a/pkg/storage/batcheval/cmd_end_transaction.go +++ b/pkg/storage/batcheval/cmd_end_transaction.go @@ -53,14 +53,8 @@ func init() { func declareKeysEndTransaction( desc roachpb.RangeDescriptor, header roachpb.Header, req roachpb.Request, spans *spanset.SpanSet, ) { - declareKeysWriteTransaction(desc, header, req, spans) et := req.(*roachpb.EndTransactionRequest) - // The spans may extend beyond this Range, but it's ok for the - // purpose of acquiring latches. The parts in our Range will - // be resolved eagerly. - for _, span := range et.IntentSpans { - spans.Add(spanset.SpanReadWrite, span) - } + declareKeysWriteTransaction(desc, header, req, spans) if header.Txn != nil { header.Txn.AssertInitialized(context.TODO()) abortSpanAccess := spanset.SpanReadOnly @@ -72,78 +66,90 @@ func declareKeysEndTransaction( }) } - // All transactions depend on the range descriptor because they need - // to determine which intents are within the local range. - spans.Add(spanset.SpanReadOnly, roachpb.Span{Key: keys.RangeDescriptorKey(desc.StartKey)}) - - if et.InternalCommitTrigger != nil { - if st := et.InternalCommitTrigger.SplitTrigger; st != nil { - // Splits may read from the entire pre-split range (they read - // from the LHS in all cases, and the RHS only when the existing - // stats contain estimates), but they need to declare a write - // access to block all other concurrent writes. We block writes - // to the RHS because they will fail if applied after the split, - // and writes to the LHS because their stat deltas will - // interfere with the non-delta stats computed as a part of the - // split. (see - // https://github.com/cockroachdb/cockroach/issues/14881) - spans.Add(spanset.SpanReadWrite, roachpb.Span{ - Key: st.LeftDesc.StartKey.AsRawKey(), - EndKey: st.RightDesc.EndKey.AsRawKey(), - }) - spans.Add(spanset.SpanReadWrite, roachpb.Span{ - Key: keys.MakeRangeKeyPrefix(st.LeftDesc.StartKey), - EndKey: keys.MakeRangeKeyPrefix(st.RightDesc.EndKey).PrefixEnd(), - }) - leftRangeIDPrefix := keys.MakeRangeIDReplicatedPrefix(header.RangeID) - spans.Add(spanset.SpanReadOnly, roachpb.Span{ - Key: leftRangeIDPrefix, - EndKey: leftRangeIDPrefix.PrefixEnd(), - }) - - rightRangeIDPrefix := keys.MakeRangeIDReplicatedPrefix(st.RightDesc.RangeID) - spans.Add(spanset.SpanReadWrite, roachpb.Span{ - Key: rightRangeIDPrefix, - EndKey: rightRangeIDPrefix.PrefixEnd(), - }) - rightRangeIDUnreplicatedPrefix := keys.MakeRangeIDUnreplicatedPrefix(st.RightDesc.RangeID) - spans.Add(spanset.SpanReadWrite, roachpb.Span{ - Key: rightRangeIDUnreplicatedPrefix, - EndKey: rightRangeIDUnreplicatedPrefix.PrefixEnd(), - }) - - spans.Add(spanset.SpanReadOnly, roachpb.Span{ - Key: keys.RangeLastReplicaGCTimestampKey(st.LeftDesc.RangeID), - }) - spans.Add(spanset.SpanReadWrite, roachpb.Span{ - Key: keys.RangeLastReplicaGCTimestampKey(st.RightDesc.RangeID), - }) - - spans.Add(spanset.SpanReadOnly, roachpb.Span{ - Key: abortspan.MinKey(header.RangeID), - EndKey: abortspan.MaxKey(header.RangeID)}) + // If the request is intending to finalize the transaction record then it + // needs to declare a few extra keys. + if !needsStaging(et) { + // All requests that intent on resolving local intents need to depend on + // the range descriptor because they need to determine which intents are + // within the local range. + spans.Add(spanset.SpanReadOnly, roachpb.Span{Key: keys.RangeDescriptorKey(desc.StartKey)}) + + // The spans may extend beyond this Range, but it's ok for the + // purpose of acquiring latches. The parts in our Range will + // be resolved eagerly. + for _, span := range et.IntentSpans { + spans.Add(spanset.SpanReadWrite, span) } - if mt := et.InternalCommitTrigger.MergeTrigger; mt != nil { - // Merges write to the left side's abort span and the right side's data - // and range-local spans. They also read from the right side's range ID - // span. - leftRangeIDPrefix := keys.MakeRangeIDReplicatedPrefix(header.RangeID) - spans.Add(spanset.SpanReadWrite, roachpb.Span{ - Key: leftRangeIDPrefix, - EndKey: leftRangeIDPrefix.PrefixEnd(), - }) - spans.Add(spanset.SpanReadWrite, roachpb.Span{ - Key: mt.RightDesc.StartKey.AsRawKey(), - EndKey: mt.RightDesc.EndKey.AsRawKey(), - }) - spans.Add(spanset.SpanReadWrite, roachpb.Span{ - Key: keys.MakeRangeKeyPrefix(mt.RightDesc.StartKey), - EndKey: keys.MakeRangeKeyPrefix(mt.RightDesc.EndKey), - }) - spans.Add(spanset.SpanReadOnly, roachpb.Span{ - Key: keys.MakeRangeIDReplicatedPrefix(mt.RightDesc.RangeID), - EndKey: keys.MakeRangeIDReplicatedPrefix(mt.RightDesc.RangeID).PrefixEnd(), - }) + + if et.InternalCommitTrigger != nil { + if st := et.InternalCommitTrigger.SplitTrigger; st != nil { + // Splits may read from the entire pre-split range (they read + // from the LHS in all cases, and the RHS only when the existing + // stats contain estimates), but they need to declare a write + // access to block all other concurrent writes. We block writes + // to the RHS because they will fail if applied after the split, + // and writes to the LHS because their stat deltas will + // interfere with the non-delta stats computed as a part of the + // split. (see + // https://github.com/cockroachdb/cockroach/issues/14881) + spans.Add(spanset.SpanReadWrite, roachpb.Span{ + Key: st.LeftDesc.StartKey.AsRawKey(), + EndKey: st.RightDesc.EndKey.AsRawKey(), + }) + spans.Add(spanset.SpanReadWrite, roachpb.Span{ + Key: keys.MakeRangeKeyPrefix(st.LeftDesc.StartKey), + EndKey: keys.MakeRangeKeyPrefix(st.RightDesc.EndKey).PrefixEnd(), + }) + leftRangeIDPrefix := keys.MakeRangeIDReplicatedPrefix(header.RangeID) + spans.Add(spanset.SpanReadOnly, roachpb.Span{ + Key: leftRangeIDPrefix, + EndKey: leftRangeIDPrefix.PrefixEnd(), + }) + + rightRangeIDPrefix := keys.MakeRangeIDReplicatedPrefix(st.RightDesc.RangeID) + spans.Add(spanset.SpanReadWrite, roachpb.Span{ + Key: rightRangeIDPrefix, + EndKey: rightRangeIDPrefix.PrefixEnd(), + }) + rightRangeIDUnreplicatedPrefix := keys.MakeRangeIDUnreplicatedPrefix(st.RightDesc.RangeID) + spans.Add(spanset.SpanReadWrite, roachpb.Span{ + Key: rightRangeIDUnreplicatedPrefix, + EndKey: rightRangeIDUnreplicatedPrefix.PrefixEnd(), + }) + + spans.Add(spanset.SpanReadOnly, roachpb.Span{ + Key: keys.RangeLastReplicaGCTimestampKey(st.LeftDesc.RangeID), + }) + spans.Add(spanset.SpanReadWrite, roachpb.Span{ + Key: keys.RangeLastReplicaGCTimestampKey(st.RightDesc.RangeID), + }) + + spans.Add(spanset.SpanReadOnly, roachpb.Span{ + Key: abortspan.MinKey(header.RangeID), + EndKey: abortspan.MaxKey(header.RangeID)}) + } + if mt := et.InternalCommitTrigger.MergeTrigger; mt != nil { + // Merges write to the left side's abort span and the right side's data + // and range-local spans. They also read from the right side's range ID + // span. + leftRangeIDPrefix := keys.MakeRangeIDReplicatedPrefix(header.RangeID) + spans.Add(spanset.SpanReadWrite, roachpb.Span{ + Key: leftRangeIDPrefix, + EndKey: leftRangeIDPrefix.PrefixEnd(), + }) + spans.Add(spanset.SpanReadWrite, roachpb.Span{ + Key: mt.RightDesc.StartKey.AsRawKey(), + EndKey: mt.RightDesc.EndKey.AsRawKey(), + }) + spans.Add(spanset.SpanReadWrite, roachpb.Span{ + Key: keys.MakeRangeKeyPrefix(mt.RightDesc.StartKey), + EndKey: keys.MakeRangeKeyPrefix(mt.RightDesc.EndKey), + }) + spans.Add(spanset.SpanReadOnly, roachpb.Span{ + Key: keys.MakeRangeIDReplicatedPrefix(mt.RightDesc.RangeID), + EndKey: keys.MakeRangeIDReplicatedPrefix(mt.RightDesc.RangeID).PrefixEnd(), + }) + } } } } @@ -159,7 +165,7 @@ func evalEndTransaction( ms := cArgs.Stats reply := resp.(*roachpb.EndTransactionResponse) - if err := VerifyTransaction(h, args); err != nil { + if err := VerifyTransaction(h, args, roachpb.PENDING, roachpb.STAGING, roachpb.ABORTED); err != nil { return result.Result{}, err } @@ -178,7 +184,7 @@ func evalEndTransaction( return result.Result{}, err } else if !ok { // No existing transaction record was found - create one by writing it - // below in updateTxnWithExternalIntents. + // below in updateFinalizedTxn. reply.Txn = h.Txn.Clone() // Verify that it is safe to create the transaction record. We only need @@ -214,7 +220,7 @@ func evalEndTransaction( if err != nil { return result.Result{}, err } - if err := updateTxnWithExternalIntents( + if err := updateFinalizedTxn( ctx, batch, ms, args, reply.Txn, externalIntents, ); err != nil { return result.Result{}, err @@ -235,24 +241,18 @@ func evalEndTransaction( return result.FromEndTxn(reply.Txn, true /* alwaysReturn */, args.Poison), roachpb.NewTransactionAbortedError(roachpb.ABORT_REASON_ABORTED_RECORD_FOUND) - case roachpb.PENDING: + case roachpb.PENDING, roachpb.STAGING: if h.Txn.Epoch < reply.Txn.Epoch { - // TODO(tschottdorf): this leaves the Txn record (and more - // importantly, intents) dangling; we can't currently write on - // error. Would panic, but that makes TestEndTransactionWithErrors - // awkward. - return result.Result{}, roachpb.NewTransactionStatusError( - fmt.Sprintf("epoch regression: %d", h.Txn.Epoch), - ) + return result.Result{}, roachpb.NewTransactionStatusError(fmt.Sprintf( + "programming error: epoch regression: %d", h.Txn.Epoch, + )) } else if h.Txn.Epoch == reply.Txn.Epoch && reply.Txn.Timestamp.Less(h.Txn.OrigTimestamp) { // The transaction record can only ever be pushed forward, so it's an // error if somehow the transaction record has an earlier timestamp // than the original transaction timestamp. - - // TODO(tschottdorf): see above comment on epoch regression. - return result.Result{}, roachpb.NewTransactionStatusError( - fmt.Sprintf("timestamp regression: %s", h.Txn.OrigTimestamp), - ) + return result.Result{}, roachpb.NewTransactionStatusError(fmt.Sprintf( + "programming error: timestamp regression: %s", h.Txn.OrigTimestamp, + )) } default: @@ -267,13 +267,38 @@ func evalEndTransaction( var pd result.Result - // Set transaction status to COMMITTED or ABORTED as per the - // args.Commit parameter. + // Attempt to commit or abort the transaction per the args.Commit parameter. if args.Commit { - if retry, reason, extraMsg := IsEndTransactionTriggeringRetryError(reply.Txn, args); retry { - return result.Result{}, roachpb.NewTransactionRetryError(reason, extraMsg) + // If the transaction is still PENDING, determine whether the commit + // should be rejected. + if reply.Txn.Status == roachpb.PENDING { + if retry, reason, extraMsg := IsEndTransactionTriggeringRetryError(reply.Txn, args); retry { + return result.Result{}, roachpb.NewTransactionRetryError(reason, extraMsg) + } + } + + // If the transaction needs to be staged as part of an implicit commit + // before being explicitly committed, write the staged transaction + // record and return without running commit triggers or resolving local + // intents. + if needsStaging(args) { + // It's not clear how to combine transaction recovery with commit + // triggers, so for now we don't allow them to mix. This shouldn't + // cause any issues and the txn coordinator knows not to mix them. + if ct := args.InternalCommitTrigger; ct != nil { + err := errors.Errorf("cannot stage transaction with a commit trigger: %+v", ct) + return result.Result{}, err + } + + reply.Txn.Status = roachpb.STAGING + reply.StagingTimestamp = reply.Txn.Timestamp + if err := updateStagingTxn(ctx, batch, ms, args, reply.Txn); err != nil { + return result.Result{}, err + } + return result.Result{}, nil } + // Else, the transaction can be explicitly committed. reply.Txn.Status = roachpb.COMMITTED // Merge triggers must run before intent resolution as the merge trigger @@ -300,16 +325,21 @@ func evalEndTransaction( reply.Txn.Status = roachpb.ABORTED } + // Resolve intents on the local range synchronously so that their resolution + // ends up in the same Raft entry. There should always be at least one because + // we position the transaction record next to the first write of a transaction. + // This avoids the need for the intentResolver to have to return to this range + // to resolve intents for this transaction in the future. desc := cArgs.EvalCtx.Desc() externalIntents, err := resolveLocalIntents(ctx, desc, batch, ms, args, reply.Txn, cArgs.EvalCtx) if err != nil { return result.Result{}, err } - if err := updateTxnWithExternalIntents(ctx, batch, ms, args, reply.Txn, externalIntents); err != nil { + if err := updateFinalizedTxn(ctx, batch, ms, args, reply.Txn, externalIntents); err != nil { return result.Result{}, err } - // Run triggers if successfully committed. + // Run the rest of the commit triggers if successfully committed. if reply.Txn.Status == roachpb.COMMITTED { triggerResult, err := RunCommitTrigger(ctx, cArgs.EvalCtx, batch.(engine.Batch), ms, args, reply.Txn) @@ -410,6 +440,16 @@ func canForwardSerializableTimestamp(txn *roachpb.Transaction, noRefreshSpans bo return !txn.OrigTimestampWasObserved && noRefreshSpans } +// needsStaging determines whether the EndTransaction request requires +// that a transaction move to the STAGING state before committing or +// not. +func needsStaging(args *roachpb.EndTransactionRequest) bool { + if args.Commit { + return len(args.InFlightWrites) > 0 + } + return false +} + const intentResolutionBatchSize = 500 // resolveLocalIntents synchronously resolves any intents that are @@ -505,11 +545,29 @@ func resolveLocalIntents( return externalIntents, nil } -// updateTxnWithExternalIntents persists the transaction record with -// updated status (& possibly timestamp). If we've already resolved -// all intents locally, we actually delete the record right away - no -// use in keeping it around. -func updateTxnWithExternalIntents( +// updateStagingTxn persists the STAGING transaction record with updated status +// (and possibly timestamp). It persists the record with the EndTransaction +// request's declared in-flight writes along with all of the transaction's +// (local and remote) intents. +func updateStagingTxn( + ctx context.Context, + batch engine.ReadWriter, + ms *enginepb.MVCCStats, + args *roachpb.EndTransactionRequest, + txn *roachpb.Transaction, +) error { + key := keys.TransactionKey(txn.Key, txn.ID) + txn.IntentSpans = args.IntentSpans + txn.InFlightWrites = args.InFlightWrites + txnRecord := txn.AsRecord() + return engine.MVCCPutProto(ctx, batch, ms, key, hlc.Timestamp{}, nil /* txn */, &txnRecord) +} + +// updateFinalizedTxn persists the COMMITTED or ABORTED transaction record with +// updated status (and possibly timestamp). If we've already resolved all +// intents locally, we actually delete the record right away - no use in keeping +// it around. +func updateFinalizedTxn( ctx context.Context, batch engine.ReadWriter, ms *enginepb.MVCCStats, @@ -525,6 +583,7 @@ func updateTxnWithExternalIntents( return engine.MVCCDelete(ctx, batch, ms, key, hlc.Timestamp{}, nil /* txn */) } txn.IntentSpans = externalIntents + txn.InFlightWrites = nil txnRecord := txn.AsRecord() return engine.MVCCPutProto(ctx, batch, ms, key, hlc.Timestamp{}, nil /* txn */, &txnRecord) } diff --git a/pkg/storage/batcheval/cmd_heartbeat_txn.go b/pkg/storage/batcheval/cmd_heartbeat_txn.go index fed69e2c7d07..81d2234ab432 100644 --- a/pkg/storage/batcheval/cmd_heartbeat_txn.go +++ b/pkg/storage/batcheval/cmd_heartbeat_txn.go @@ -46,7 +46,7 @@ func HeartbeatTxn( h := cArgs.Header reply := resp.(*roachpb.HeartbeatTxnResponse) - if err := VerifyTransaction(h, args); err != nil { + if err := VerifyTransaction(h, args, roachpb.PENDING, roachpb.STAGING); err != nil { return result.Result{}, err } @@ -65,11 +65,6 @@ func HeartbeatTxn( // No existing transaction record was found - create one by writing // it below. txn = *h.Txn - if txn.Status != roachpb.PENDING { - return result.Result{}, roachpb.NewTransactionStatusError( - fmt.Sprintf("cannot heartbeat txn with status %v: %s", txn.Status, txn), - ) - } // Verify that it is safe to create the transaction record. if err := CanCreateTxnRecord(cArgs.EvalCtx, &txn); err != nil { @@ -77,7 +72,7 @@ func HeartbeatTxn( } } - if txn.Status == roachpb.PENDING { + if !txn.Status.IsFinalized() { txn.LastHeartbeat.Forward(args.Now) txnRecord := txn.AsRecord() if err := engine.MVCCPutProto(ctx, batch, cArgs.Stats, key, hlc.Timestamp{}, nil, &txnRecord); err != nil { diff --git a/pkg/storage/batcheval/cmd_push_txn.go b/pkg/storage/batcheval/cmd_push_txn.go index 58a287f851ba..62b34b11a30c 100644 --- a/pkg/storage/batcheval/cmd_push_txn.go +++ b/pkg/storage/batcheval/cmd_push_txn.go @@ -240,19 +240,20 @@ func PushTxn( reason, reply.PusheeTxn.LastActive()) } - if !pusherWins { - err := roachpb.NewTransactionPushError(reply.PusheeTxn) + // If the pushed transaction is in the staging state, we can't change its + // record without first going through the transaction recovery process and + // attempting to finalize it. + recoverOnFailedPush := cArgs.EvalCtx.EvalKnobs().RecoverIndeterminateCommitsOnFailedPushes + if reply.PusheeTxn.Status == roachpb.STAGING && (pusherWins || recoverOnFailedPush) { + err := roachpb.NewIndeterminateCommitError(reply.PusheeTxn) if log.V(1) { log.Infof(ctx, "%v", err) } return result.Result{}, err } - // If the pushed transaction is in the staging state, we can't change its - // record without first going through the transaction recovery process and - // attempting to finalize it. - if reply.PusheeTxn.Status == roachpb.STAGING { - err := roachpb.NewIndeterminateCommitError(reply.PusheeTxn) + if !pusherWins { + err := roachpb.NewTransactionPushError(reply.PusheeTxn) if log.V(1) { log.Infof(ctx, "%v", err) } diff --git a/pkg/storage/batcheval/cmd_recover_txn.go b/pkg/storage/batcheval/cmd_recover_txn.go index 91741c370da6..16d88bf53344 100644 --- a/pkg/storage/batcheval/cmd_recover_txn.go +++ b/pkg/storage/batcheval/cmd_recover_txn.go @@ -210,6 +210,10 @@ func RecoverTxn( return result.Result{}, err } + // TODO(nvanbenschoten): This could use result.FromEndTxn to trigger + // intent resolution for the recovered transaction's intents. To do + // that, we might need to plumb in a "poison" flag on the RecoverTxn + // request. result := result.Result{} result.Local.UpdatedTxns = &[]*roachpb.Transaction{&reply.RecoveredTxn} return result, nil diff --git a/pkg/storage/batcheval/transaction.go b/pkg/storage/batcheval/transaction.go index 022f2d055c6b..6b21ff4b7a9b 100644 --- a/pkg/storage/batcheval/transaction.go +++ b/pkg/storage/batcheval/transaction.go @@ -17,6 +17,7 @@ package batcheval import ( "bytes" "context" + "fmt" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/storage/engine" @@ -30,13 +31,27 @@ var ErrTransactionUnsupported = errors.New("not supported within a transaction") // VerifyTransaction runs sanity checks verifying that the transaction in the // header and the request are compatible. -func VerifyTransaction(h roachpb.Header, args roachpb.Request) error { +func VerifyTransaction( + h roachpb.Header, args roachpb.Request, permittedStatuses ...roachpb.TransactionStatus, +) error { if h.Txn == nil { return errors.Errorf("no transaction specified to %s", args.Method()) } if !bytes.Equal(args.Header().Key, h.Txn.Key) { return errors.Errorf("request key %s should match txn key %s", args.Header().Key, h.Txn.Key) } + statusPermitted := false + for _, s := range permittedStatuses { + if h.Txn.Status == s { + statusPermitted = true + break + } + } + if !statusPermitted { + return roachpb.NewTransactionStatusError( + fmt.Sprintf("cannot perform %s with txn status %v", args.Method(), h.Txn.Status), + ) + } return nil } @@ -82,7 +97,10 @@ func SetAbortSpan( if exists && curEntry.Equal(entry) { return nil } - return rec.AbortSpan().Put(ctx, batch, ms, txn.ID, &entry) + // curEntry already escapes, so assign entry to curEntry and pass + // that to Put instead of allowing entry to escape as well. + curEntry = entry + return rec.AbortSpan().Put(ctx, batch, ms, txn.ID, &curEntry) } // CanPushWithPriority returns true if the given pusher can push the pushee diff --git a/pkg/storage/replica_test.go b/pkg/storage/replica_test.go index 14c9379274ad..5458ffb4d7ad 100644 --- a/pkg/storage/replica_test.go +++ b/pkg/storage/replica_test.go @@ -411,6 +411,101 @@ func createReplicaSets(replicaNumbers []roachpb.StoreID) []roachpb.ReplicaDescri return result } +// TestMaybeStripInFlightWrites verifies that in-flight writes declared +// on an EndTransaction request are stripped if the corresponding write +// or query intent is in the same batch as the EndTransaction. +func TestMaybeStripInFlightWrites(t *testing.T) { + defer leaktest.AfterTest(t)() + + keyA, keyB, keyC := roachpb.Key("a"), roachpb.Key("b"), roachpb.Key("c") + qi1 := &roachpb.QueryIntentRequest{RequestHeader: roachpb.RequestHeader{Key: keyA}} + qi1.Txn.Sequence = 1 + put2 := &roachpb.PutRequest{RequestHeader: roachpb.RequestHeader{Key: keyB}} + put2.Sequence = 2 + put3 := &roachpb.PutRequest{RequestHeader: roachpb.RequestHeader{Key: keyC}} + put3.Sequence = 3 + delRng3 := &roachpb.DeleteRangeRequest{RequestHeader: roachpb.RequestHeader{Key: keyC}} + delRng3.Sequence = 3 + scan3 := &roachpb.ScanRequest{RequestHeader: roachpb.RequestHeader{Key: keyC}} + scan3.Sequence = 3 + et := &roachpb.EndTransactionRequest{RequestHeader: roachpb.RequestHeader{Key: keyA}, Commit: true} + et.Sequence = 4 + et.IntentSpans = []roachpb.Span{{Key: keyC}} + et.InFlightWrites = []roachpb.SequencedWrite{{Key: keyA, Sequence: 1}, {Key: keyB, Sequence: 2}} + testCases := []struct { + reqs []roachpb.Request + expIFW []roachpb.SequencedWrite + expIntentSpans []roachpb.Span + expErr string + }{ + { + reqs: []roachpb.Request{et}, + expIFW: []roachpb.SequencedWrite{{Key: keyA, Sequence: 1}, {Key: keyB, Sequence: 2}}, + expIntentSpans: []roachpb.Span{{Key: keyC}}, + }, + // QueryIntents aren't stripped from the in-flight writes set on the + // slow-path of maybeStripInFlightWrites. This is intentional. + { + reqs: []roachpb.Request{qi1, et}, + expIFW: []roachpb.SequencedWrite{{Key: keyA, Sequence: 1}, {Key: keyB, Sequence: 2}}, + expIntentSpans: []roachpb.Span{{Key: keyC}}, + }, + { + reqs: []roachpb.Request{put2, et}, + expIFW: []roachpb.SequencedWrite{{Key: keyA, Sequence: 1}}, + expIntentSpans: []roachpb.Span{{Key: keyB}, {Key: keyC}}, + }, + { + reqs: []roachpb.Request{put3, et}, + expErr: "write in batch with EndTransaction missing from in-flight writes", + }, + { + reqs: []roachpb.Request{qi1, put2, et}, + expIFW: nil, + expIntentSpans: []roachpb.Span{{Key: keyA}, {Key: keyB}, {Key: keyC}}, + }, + { + reqs: []roachpb.Request{qi1, put2, delRng3, et}, + expIFW: nil, + expIntentSpans: []roachpb.Span{{Key: keyA}, {Key: keyB}, {Key: keyC}}, + }, + { + reqs: []roachpb.Request{qi1, put2, scan3, et}, + expIFW: nil, + expIntentSpans: []roachpb.Span{{Key: keyA}, {Key: keyB}, {Key: keyC}}, + }, + { + reqs: []roachpb.Request{qi1, put2, delRng3, scan3, et}, + expIFW: nil, + expIntentSpans: []roachpb.Span{{Key: keyA}, {Key: keyB}, {Key: keyC}}, + }, + } + for _, c := range testCases { + var ba roachpb.BatchRequest + ba.Add(c.reqs...) + t.Run(fmt.Sprint(ba), func(t *testing.T) { + resBa, err := maybeStripInFlightWrites(ba) + if c.expErr == "" { + if err != nil { + t.Errorf("expected no error, got %v", err) + } + resArgs, _ := resBa.GetArg(roachpb.EndTransaction) + resEt := resArgs.(*roachpb.EndTransactionRequest) + if !reflect.DeepEqual(resEt.InFlightWrites, c.expIFW) { + t.Errorf("expected in-flight writes %v, got %v", c.expIFW, resEt.InFlightWrites) + } + if !reflect.DeepEqual(resEt.IntentSpans, c.expIntentSpans) { + t.Errorf("expected intent spans %v, got %v", c.expIntentSpans, resEt.IntentSpans) + } + } else { + if !testutils.IsError(err, c.expErr) { + t.Errorf("expected error %q, got %v", c.expErr, err) + } + } + }) + } +} + // TestIsOnePhaseCommit verifies the circumstances where a // transactional batch can be committed as an atomic write. func TestIsOnePhaseCommit(t *testing.T) { @@ -9986,6 +10081,10 @@ func TestTxnRecordLifecycleTransitions(t *testing.T) { return pErr.GoError() } + intents := []roachpb.Span{{Key: roachpb.Key("a")}} + inFlightWrites := []roachpb.SequencedWrite{{Key: roachpb.Key("a"), Sequence: 1}} + otherInFlightWrites := []roachpb.SequencedWrite{{Key: roachpb.Key("b"), Sequence: 2}} + type verifyFunc func(*roachpb.Transaction, hlc.Timestamp) roachpb.TransactionRecord noTxnRecord := verifyFunc(nil) txnWithoutChanges := func(txn *roachpb.Transaction, _ hlc.Timestamp) roachpb.TransactionRecord { @@ -9998,6 +10097,11 @@ func TestTxnRecordLifecycleTransitions(t *testing.T) { return record } } + txnWithStagingStatusAndInFlightWrites := func(txn *roachpb.Transaction, now hlc.Timestamp) roachpb.TransactionRecord { + record := txnWithStatus(roachpb.STAGING)(txn, now) + record.InFlightWrites = inFlightWrites + return record + } testCases := []struct { name string @@ -10027,6 +10131,15 @@ func TestTxnRecordLifecycleTransitions(t *testing.T) { return record }, }, + { + name: "end transaction (stage)", + run: func(txn *roachpb.Transaction, _ hlc.Timestamp) error { + et, etH := endTxnArgs(txn, true /* commit */) + et.InFlightWrites = inFlightWrites + return sendWrappedWithErr(etH, &et) + }, + expTxn: txnWithStagingStatusAndInFlightWrites, + }, { name: "end transaction (abort)", run: func(txn *roachpb.Transaction, _ hlc.Timestamp) error { @@ -10185,6 +10298,19 @@ func TestTxnRecordLifecycleTransitions(t *testing.T) { return record }, }, + { + name: "end transaction (stage) after begin transaction", + setup: func(txn *roachpb.Transaction, _ hlc.Timestamp) error { + bt, btH := beginTxnArgs(txn.Key, txn) + return sendWrappedWithErr(btH, &bt) + }, + run: func(txn *roachpb.Transaction, _ hlc.Timestamp) error { + et, etH := endTxnArgs(txn, true /* commit */) + et.InFlightWrites = inFlightWrites + return sendWrappedWithErr(etH, &et) + }, + expTxn: txnWithStagingStatusAndInFlightWrites, + }, { name: "end transaction (abort) after begin transaction", setup: func(txn *roachpb.Transaction, _ hlc.Timestamp) error { @@ -10305,6 +10431,198 @@ func TestTxnRecordLifecycleTransitions(t *testing.T) { return record }, }, + { + name: "begin transaction after end transaction (stage)", + setup: func(txn *roachpb.Transaction, _ hlc.Timestamp) error { + et, etH := endTxnArgs(txn, true /* commit */) + et.InFlightWrites = inFlightWrites + return sendWrappedWithErr(etH, &et) + }, + run: func(txn *roachpb.Transaction, _ hlc.Timestamp) error { + bt, btH := beginTxnArgs(txn.Key, txn) + return sendWrappedWithErr(btH, &bt) + }, + expError: "TransactionStatusError: BeginTransaction can't overwrite", + expTxn: txnWithStagingStatusAndInFlightWrites, + }, + { + // Staging transaction records can still be heartbeat. + name: "heartbeat transaction after end transaction (stage)", + setup: func(txn *roachpb.Transaction, _ hlc.Timestamp) error { + et, etH := endTxnArgs(txn, true /* commit */) + et.InFlightWrites = inFlightWrites + return sendWrappedWithErr(etH, &et) + }, + run: func(txn *roachpb.Transaction, now hlc.Timestamp) error { + hb, hbH := heartbeatArgs(txn, now) + return sendWrappedWithErr(hbH, &hb) + }, + expTxn: func(txn *roachpb.Transaction, hbTs hlc.Timestamp) roachpb.TransactionRecord { + record := txnWithStagingStatusAndInFlightWrites(txn, hbTs) + record.LastHeartbeat.Forward(hbTs) + return record + }, + }, + { + // Should not be possible outside of replays or re-issues of the + // same request, but also not prevented. If not a re-issue, the + // second stage will always either bump the commit timestamp or + // bump the epoch. + name: "end transaction (stage) after end transaction (stage)", + setup: func(txn *roachpb.Transaction, _ hlc.Timestamp) error { + et, etH := endTxnArgs(txn, true /* commit */) + et.InFlightWrites = inFlightWrites + return sendWrappedWithErr(etH, &et) + }, + run: func(txn *roachpb.Transaction, _ hlc.Timestamp) error { + et, etH := endTxnArgs(txn, true /* commit */) + et.InFlightWrites = inFlightWrites + return sendWrappedWithErr(etH, &et) + }, + expTxn: txnWithStagingStatusAndInFlightWrites, + }, + { + // Case of a transaction that refreshed after an unsuccessful + // implicit commit. If the refresh is successful then the + // transaction coordinator can attempt the implicit commit again. + name: "end transaction (stage) with timestamp increase after end transaction (stage)", + setup: func(txn *roachpb.Transaction, _ hlc.Timestamp) error { + et, etH := endTxnArgs(txn, true /* commit */) + et.InFlightWrites = inFlightWrites + return sendWrappedWithErr(etH, &et) + }, + run: func(txn *roachpb.Transaction, now hlc.Timestamp) error { + clone := txn.Clone() + clone.RefreshedTimestamp.Forward(now) + clone.Timestamp.Forward(now) + et, etH := endTxnArgs(clone, true /* commit */) + // Add different in-flight writes to test whether they are + // replaced by the second EndTransaction request. + et.InFlightWrites = otherInFlightWrites + return sendWrappedWithErr(etH, &et) + }, + expTxn: func(txn *roachpb.Transaction, now hlc.Timestamp) roachpb.TransactionRecord { + record := txnWithStagingStatusAndInFlightWrites(txn, now) + record.InFlightWrites = otherInFlightWrites + record.Timestamp.Forward(now) + return record + }, + }, + { + // Case of a transaction that restarted after an unsuccessful + // implicit commit. The transaction coordinator can attempt an + // implicit commit in the next epoch. + name: "end transaction (stage) with epoch bump after end transaction (stage)", + setup: func(txn *roachpb.Transaction, _ hlc.Timestamp) error { + et, etH := endTxnArgs(txn, true /* commit */) + et.InFlightWrites = inFlightWrites + return sendWrappedWithErr(etH, &et) + }, + run: func(txn *roachpb.Transaction, now hlc.Timestamp) error { + clone := txn.Clone() + clone.Restart(-1, 0, now) + et, etH := endTxnArgs(clone, true /* commit */) + // Add different in-flight writes to test whether they are + // replaced by the second EndTransaction request. + et.InFlightWrites = otherInFlightWrites + return sendWrappedWithErr(etH, &et) + }, + expTxn: func(txn *roachpb.Transaction, now hlc.Timestamp) roachpb.TransactionRecord { + record := txnWithStagingStatusAndInFlightWrites(txn, now) + record.InFlightWrites = otherInFlightWrites + record.Epoch = txn.Epoch + 1 + record.Timestamp.Forward(now) + record.OrigTimestamp.Forward(now) + return record + }, + }, + { + // Case of a rollback after an unsuccessful implicit commit. + name: "end transaction (abort) after end transaction (stage)", + setup: func(txn *roachpb.Transaction, _ hlc.Timestamp) error { + et, etH := endTxnArgs(txn, true /* commit */) + et.InFlightWrites = inFlightWrites + return sendWrappedWithErr(etH, &et) + }, + run: func(txn *roachpb.Transaction, _ hlc.Timestamp) error { + et, etH := endTxnArgs(txn, false /* commit */) + return sendWrappedWithErr(etH, &et) + }, + // The transaction record will be eagerly GC-ed. + expTxn: noTxnRecord, + }, + { + // Case of making a commit "explicit" after a successful implicit commit. + name: "end transaction (commit) after end transaction (stage)", + setup: func(txn *roachpb.Transaction, _ hlc.Timestamp) error { + et, etH := endTxnArgs(txn, true /* commit */) + et.InFlightWrites = inFlightWrites + return sendWrappedWithErr(etH, &et) + }, + run: func(txn *roachpb.Transaction, _ hlc.Timestamp) error { + et, etH := endTxnArgs(txn, true /* commit */) + return sendWrappedWithErr(etH, &et) + }, + // The transaction record will be eagerly GC-ed. + expTxn: noTxnRecord, + }, + { + name: "end transaction (abort) without eager gc after end transaction (stage)", + setup: func(txn *roachpb.Transaction, _ hlc.Timestamp) error { + et, etH := endTxnArgs(txn, true /* commit */) + et.InFlightWrites = inFlightWrites + return sendWrappedWithErr(etH, &et) + }, + run: func(txn *roachpb.Transaction, _ hlc.Timestamp) error { + et, etH := endTxnArgs(txn, false /* commit */) + return sendWrappedWithErr(etH, &et) + }, + expTxn: txnWithStatus(roachpb.ABORTED), + disableTxnAutoGC: true, + }, + { + name: "end transaction (commit) without eager gc after end transaction (stage)", + setup: func(txn *roachpb.Transaction, _ hlc.Timestamp) error { + et, etH := endTxnArgs(txn, true /* commit */) + et.InFlightWrites = inFlightWrites + return sendWrappedWithErr(etH, &et) + }, + run: func(txn *roachpb.Transaction, _ hlc.Timestamp) error { + et, etH := endTxnArgs(txn, true /* commit */) + return sendWrappedWithErr(etH, &et) + }, + expTxn: txnWithStatus(roachpb.COMMITTED), + disableTxnAutoGC: true, + }, + { + name: "push transaction (timestamp) after end transaction (stage)", + setup: func(txn *roachpb.Transaction, _ hlc.Timestamp) error { + et, etH := endTxnArgs(txn, true /* commit */) + et.InFlightWrites = inFlightWrites + return sendWrappedWithErr(etH, &et) + }, + run: func(txn *roachpb.Transaction, now hlc.Timestamp) error { + pt := pushTxnArgs(pusher, txn, roachpb.PUSH_TIMESTAMP) + pt.PushTo = now + return sendWrappedWithErr(roachpb.Header{}, &pt) + }, + expError: "found txn in indeterminate STAGING state", + expTxn: txnWithStagingStatusAndInFlightWrites, + }, + { + name: "push transaction (abort) after end transaction (stage)", + setup: func(txn *roachpb.Transaction, _ hlc.Timestamp) error { + et, etH := endTxnArgs(txn, true /* commit */) + et.InFlightWrites = inFlightWrites + return sendWrappedWithErr(etH, &et) + }, + run: func(txn *roachpb.Transaction, _ hlc.Timestamp) error { + pt := pushTxnArgs(pusher, txn, roachpb.PUSH_ABORT) + return sendWrappedWithErr(roachpb.Header{}, &pt) + }, + expError: "found txn in indeterminate STAGING state", + expTxn: txnWithStagingStatusAndInFlightWrites, + }, { name: "begin transaction after end transaction (abort)", setup: func(txn *roachpb.Transaction, _ hlc.Timestamp) error { @@ -10350,6 +10668,21 @@ func TestTxnRecordLifecycleTransitions(t *testing.T) { expError: "TransactionAbortedError(ABORT_REASON_ALREADY_COMMITTED_OR_ROLLED_BACK_POSSIBLE_REPLAY)", expTxn: noTxnRecord, }, + { + // Could be a replay or a retry. + name: "end transaction (stage) after end transaction (abort)", + setup: func(txn *roachpb.Transaction, _ hlc.Timestamp) error { + et, etH := endTxnArgs(txn, false /* commit */) + return sendWrappedWithErr(etH, &et) + }, + run: func(txn *roachpb.Transaction, _ hlc.Timestamp) error { + et, etH := endTxnArgs(txn, true /* commit */) + et.InFlightWrites = inFlightWrites + return sendWrappedWithErr(etH, &et) + }, + expError: "TransactionAbortedError(ABORT_REASON_ALREADY_COMMITTED_OR_ROLLED_BACK_POSSIBLE_REPLAY)", + expTxn: noTxnRecord, + }, { // Could be a replay or a retry. name: "end transaction (abort) after end transaction (abort)", @@ -10429,6 +10762,21 @@ func TestTxnRecordLifecycleTransitions(t *testing.T) { expError: "TransactionAbortedError(ABORT_REASON_ALREADY_COMMITTED_OR_ROLLED_BACK_POSSIBLE_REPLAY)", expTxn: noTxnRecord, }, + { + // Could be a replay or a retry. + name: "end transaction (stage) after end transaction (commit)", + setup: func(txn *roachpb.Transaction, _ hlc.Timestamp) error { + et, etH := endTxnArgs(txn, true /* commit */) + return sendWrappedWithErr(etH, &et) + }, + run: func(txn *roachpb.Transaction, _ hlc.Timestamp) error { + et, etH := endTxnArgs(txn, true /* commit */) + et.InFlightWrites = inFlightWrites + return sendWrappedWithErr(etH, &et) + }, + expError: "TransactionAbortedError(ABORT_REASON_ALREADY_COMMITTED_OR_ROLLED_BACK_POSSIBLE_REPLAY)", + expTxn: noTxnRecord, + }, { // This case shouldn't happen in practice given a well-functioning // transaction coordinator, but is handled correctly nevertheless. @@ -10513,6 +10861,22 @@ func TestTxnRecordLifecycleTransitions(t *testing.T) { expTxn: txnWithStatus(roachpb.ABORTED), disableTxnAutoGC: true, }, + { + // Could be a replay or a retry. + name: "end transaction (stage) after end transaction (abort) without eager gc", + setup: func(txn *roachpb.Transaction, _ hlc.Timestamp) error { + et, etH := endTxnArgs(txn, false /* commit */) + return sendWrappedWithErr(etH, &et) + }, + run: func(txn *roachpb.Transaction, _ hlc.Timestamp) error { + et, etH := endTxnArgs(txn, true /* commit */) + et.InFlightWrites = inFlightWrites + return sendWrappedWithErr(etH, &et) + }, + expError: "TransactionAbortedError(ABORT_REASON_ABORTED_RECORD_FOUND)", + expTxn: txnWithStatus(roachpb.ABORTED), + disableTxnAutoGC: true, + }, { // Could be a replay or a retry. name: "end transaction (abort) after end transaction (abort) without eager gc", @@ -10601,6 +10965,22 @@ func TestTxnRecordLifecycleTransitions(t *testing.T) { expTxn: txnWithStatus(roachpb.COMMITTED), disableTxnAutoGC: true, }, + { + // Could be a replay or a retry. + name: "end transaction (stage) after end transaction (commit) without eager gc", + setup: func(txn *roachpb.Transaction, _ hlc.Timestamp) error { + et, etH := endTxnArgs(txn, true /* commit */) + return sendWrappedWithErr(etH, &et) + }, + run: func(txn *roachpb.Transaction, _ hlc.Timestamp) error { + et, etH := endTxnArgs(txn, true /* commit */) + et.InFlightWrites = inFlightWrites + return sendWrappedWithErr(etH, &et) + }, + expError: "TransactionStatusError: already committed (REASON_TXN_COMMITTED)", + expTxn: txnWithStatus(roachpb.COMMITTED), + disableTxnAutoGC: true, + }, { // This case shouldn't happen in practice given a well-functioning // transaction coordinator, but is handled correctly nevertheless. @@ -10694,6 +11074,23 @@ func TestTxnRecordLifecycleTransitions(t *testing.T) { return record }, }, + { + name: "end transaction (stage) after push transaction (timestamp)", + setup: func(txn *roachpb.Transaction, now hlc.Timestamp) error { + pt := pushTxnArgs(pusher, txn, roachpb.PUSH_TIMESTAMP) + pt.PushTo = now + return sendWrappedWithErr(roachpb.Header{}, &pt) + }, + run: func(txn *roachpb.Transaction, _ hlc.Timestamp) error { + et, etH := endTxnArgs(txn, true /* commit */) + et.InFlightWrites = inFlightWrites + return sendWrappedWithErr(etH, &et) + }, + expError: "TransactionRetryError: retry txn (RETRY_SERIALIZABLE)", + // The end transaction (stage) does not write a transaction record + // if it hits a serializable retry error. + expTxn: noTxnRecord, + }, { name: "end transaction (abort) after push transaction (timestamp)", setup: func(txn *roachpb.Transaction, now hlc.Timestamp) error { @@ -10770,6 +11167,20 @@ func TestTxnRecordLifecycleTransitions(t *testing.T) { expError: "TransactionAbortedError(ABORT_REASON_ABORTED_RECORD_FOUND)", expTxn: noTxnRecord, }, + { + name: "end transaction (stage) after push transaction (abort)", + setup: func(txn *roachpb.Transaction, now hlc.Timestamp) error { + pt := pushTxnArgs(pusher, txn, roachpb.PUSH_ABORT) + return sendWrappedWithErr(roachpb.Header{}, &pt) + }, + run: func(txn *roachpb.Transaction, _ hlc.Timestamp) error { + et, etH := endTxnArgs(txn, true /* commit */) + et.InFlightWrites = inFlightWrites + return sendWrappedWithErr(etH, &et) + }, + expError: "TransactionAbortedError(ABORT_REASON_ABORTED_RECORD_FOUND)", + expTxn: noTxnRecord, + }, { name: "end transaction (abort) after push transaction (abort)", setup: func(txn *roachpb.Transaction, now hlc.Timestamp) error { @@ -10798,6 +11209,7 @@ func TestTxnRecordLifecycleTransitions(t *testing.T) { expTxn: noTxnRecord, }, { + // Should not be possible. name: "recover transaction (implicitly committed) after begin transaction", setup: func(txn *roachpb.Transaction, _ hlc.Timestamp) error { bt, btH := beginTxnArgs(txn.Key, txn) @@ -10811,6 +11223,72 @@ func TestTxnRecordLifecycleTransitions(t *testing.T) { expTxn: txnWithoutChanges, }, { + // Typical case of transaction recovery from a STAGING status after + // a successful implicit commit. + name: "recover transaction (implicitly committed) after end transaction (stage)", + setup: func(txn *roachpb.Transaction, _ hlc.Timestamp) error { + et, etH := endTxnArgs(txn, true /* commit */) + et.InFlightWrites = inFlightWrites + return sendWrappedWithErr(etH, &et) + }, + run: func(txn *roachpb.Transaction, _ hlc.Timestamp) error { + rt := recoverTxnArgs(txn, true /* implicitlyCommitted */) + return sendWrappedWithErr(roachpb.Header{}, &rt) + }, + expTxn: func(txn *roachpb.Transaction, now hlc.Timestamp) roachpb.TransactionRecord { + record := txnWithStatus(roachpb.COMMITTED)(txn, now) + // RecoverTxn does not synchronously resolve local intents. + record.IntentSpans = intents + return record + }, + }, + { + // Should not be possible. + name: "recover transaction (implicitly committed) after end transaction (stage) with timestamp increase", + setup: func(txn *roachpb.Transaction, now hlc.Timestamp) error { + clone := txn.Clone() + clone.RefreshedTimestamp.Forward(now) + clone.Timestamp.Forward(now) + et, etH := endTxnArgs(clone, true /* commit */) + et.InFlightWrites = inFlightWrites + return sendWrappedWithErr(etH, &et) + }, + run: func(txn *roachpb.Transaction, _ hlc.Timestamp) error { + rt := recoverTxnArgs(txn, true /* implicitlyCommitted */) + return sendWrappedWithErr(roachpb.Header{}, &rt) + }, + expError: "timestamp change by implicitly committed transaction", + expTxn: func(txn *roachpb.Transaction, now hlc.Timestamp) roachpb.TransactionRecord { + record := txnWithStagingStatusAndInFlightWrites(txn, now) + record.Timestamp.Forward(now) + return record + }, + }, + { + // Should not be possible. + name: "recover transaction (implicitly committed) after end transaction (stage) with epoch bump", + setup: func(txn *roachpb.Transaction, now hlc.Timestamp) error { + clone := txn.Clone() + clone.Restart(-1, 0, now) + et, etH := endTxnArgs(clone, true /* commit */) + et.InFlightWrites = inFlightWrites + return sendWrappedWithErr(etH, &et) + }, + run: func(txn *roachpb.Transaction, _ hlc.Timestamp) error { + rt := recoverTxnArgs(txn, true /* implicitlyCommitted */) + return sendWrappedWithErr(roachpb.Header{}, &rt) + }, + expError: "epoch change by implicitly committed transaction", + expTxn: func(txn *roachpb.Transaction, now hlc.Timestamp) roachpb.TransactionRecord { + record := txnWithStagingStatusAndInFlightWrites(txn, now) + record.Epoch = txn.Epoch + 1 + record.Timestamp.Forward(now) + record.OrigTimestamp.Forward(now) + return record + }, + }, + { + // Should not be possible. name: "recover transaction (implicitly committed) after end transaction (abort)", setup: func(txn *roachpb.Transaction, _ hlc.Timestamp) error { et, etH := endTxnArgs(txn, false /* commit */) @@ -10825,6 +11303,8 @@ func TestTxnRecordLifecycleTransitions(t *testing.T) { expTxn: noTxnRecord, }, { + // A concurrent recovery process completed or the transaction + // coordinator made its commit explicit. name: "recover transaction (implicitly committed) after end transaction (commit)", setup: func(txn *roachpb.Transaction, _ hlc.Timestamp) error { et, etH := endTxnArgs(txn, true /* commit */) @@ -10837,6 +11317,7 @@ func TestTxnRecordLifecycleTransitions(t *testing.T) { expTxn: noTxnRecord, }, { + // Should not be possible. name: "recover transaction (implicitly committed) after end transaction (abort) without eager gc", setup: func(txn *roachpb.Transaction, _ hlc.Timestamp) error { et, etH := endTxnArgs(txn, false /* commit */) @@ -10851,6 +11332,8 @@ func TestTxnRecordLifecycleTransitions(t *testing.T) { disableTxnAutoGC: true, }, { + // A concurrent recovery process completed or the transaction + // coordinator made its commit explicit. name: "recover transaction (implicitly committed) after end transaction (commit) without eager gc", setup: func(txn *roachpb.Transaction, _ hlc.Timestamp) error { et, etH := endTxnArgs(txn, true /* commit */) @@ -10864,6 +11347,7 @@ func TestTxnRecordLifecycleTransitions(t *testing.T) { disableTxnAutoGC: true, }, { + // Should not be possible. name: "recover transaction (not implicitly committed) after begin transaction", setup: func(txn *roachpb.Transaction, _ hlc.Timestamp) error { bt, btH := beginTxnArgs(txn.Key, txn) @@ -10877,6 +11361,8 @@ func TestTxnRecordLifecycleTransitions(t *testing.T) { expTxn: txnWithoutChanges, }, { + // Transaction coordinator restarted after failing to perform a + // implicit commit. Common case. name: "recover transaction (not implicitly committed) after begin transaction with epoch bump", setup: func(txn *roachpb.Transaction, now hlc.Timestamp) error { clone := txn.Clone() @@ -10897,6 +11383,79 @@ func TestTxnRecordLifecycleTransitions(t *testing.T) { }, }, { + // Typical case of transaction recovery from a STAGING status after + // an unsuccessful implicit commit. + name: "recover transaction (not implicitly committed) after end transaction (stage)", + setup: func(txn *roachpb.Transaction, _ hlc.Timestamp) error { + et, etH := endTxnArgs(txn, true /* commit */) + et.InFlightWrites = inFlightWrites + return sendWrappedWithErr(etH, &et) + }, + run: func(txn *roachpb.Transaction, _ hlc.Timestamp) error { + rt := recoverTxnArgs(txn, false /* implicitlyCommitted */) + return sendWrappedWithErr(roachpb.Header{}, &rt) + }, + expTxn: func(txn *roachpb.Transaction, now hlc.Timestamp) roachpb.TransactionRecord { + record := txnWithStatus(roachpb.ABORTED)(txn, now) + // RecoverTxn does not synchronously resolve local intents. + record.IntentSpans = intents + return record + }, + }, + { + // Typical case of transaction recovery from a STAGING status after + // an unsuccessful implicit commit. Transaction coordinator bumped + // timestamp in same epoch to attempt implicit commit again. The + // RecoverTxn request should not modify the transaction record. + name: "recover transaction (not implicitly committed) after end transaction (stage) with timestamp increase", + setup: func(txn *roachpb.Transaction, now hlc.Timestamp) error { + clone := txn.Clone() + clone.RefreshedTimestamp.Forward(now) + clone.Timestamp.Forward(now) + et, etH := endTxnArgs(clone, true /* commit */) + et.InFlightWrites = inFlightWrites + return sendWrappedWithErr(etH, &et) + }, + run: func(txn *roachpb.Transaction, _ hlc.Timestamp) error { + rt := recoverTxnArgs(txn, false /* implicitlyCommitted */) + return sendWrappedWithErr(roachpb.Header{}, &rt) + }, + expTxn: func(txn *roachpb.Transaction, now hlc.Timestamp) roachpb.TransactionRecord { + // Unchanged by the RecoverTxn request. + record := txnWithStagingStatusAndInFlightWrites(txn, now) + record.Timestamp.Forward(now) + return record + }, + }, + { + // Typical case of transaction recovery from a STAGING status after + // an unsuccessful implicit commit. Transaction coordinator bumped + // epoch after a restart and is attempting implicit commit again. + // The RecoverTxn request should not modify the transaction record. + name: "recover transaction (not implicitly committed) after end transaction (stage) with epoch bump", + setup: func(txn *roachpb.Transaction, now hlc.Timestamp) error { + clone := txn.Clone() + clone.Restart(-1, 0, now) + et, etH := endTxnArgs(clone, true /* commit */) + et.InFlightWrites = inFlightWrites + return sendWrappedWithErr(etH, &et) + }, + run: func(txn *roachpb.Transaction, _ hlc.Timestamp) error { + rt := recoverTxnArgs(txn, false /* implicitlyCommitted */) + return sendWrappedWithErr(roachpb.Header{}, &rt) + }, + expTxn: func(txn *roachpb.Transaction, now hlc.Timestamp) roachpb.TransactionRecord { + record := txnWithStagingStatusAndInFlightWrites(txn, now) + record.Epoch = txn.Epoch + 1 + record.Timestamp.Forward(now) + record.OrigTimestamp.Forward(now) + return record + }, + }, + { + // A concurrent recovery process completed or the transaction + // coordinator rolled back its transaction record after an + // unsuccessful implicit commit. name: "recover transaction (not implicitly committed) after end transaction (abort)", setup: func(txn *roachpb.Transaction, _ hlc.Timestamp) error { et, etH := endTxnArgs(txn, false /* commit */) @@ -10909,6 +11468,7 @@ func TestTxnRecordLifecycleTransitions(t *testing.T) { expTxn: noTxnRecord, }, { + // Should not be possible. name: "recover transaction (not implicitly committed) after end transaction (commit)", setup: func(txn *roachpb.Transaction, _ hlc.Timestamp) error { et, etH := endTxnArgs(txn, true /* commit */) @@ -10923,6 +11483,9 @@ func TestTxnRecordLifecycleTransitions(t *testing.T) { expTxn: noTxnRecord, }, { + // A concurrent recovery process completed or the transaction + // coordinator rolled back its transaction record after an + // unsuccessful implicit commit. name: "recover transaction (not implicitly committed) after end transaction (abort) without eager gc", setup: func(txn *roachpb.Transaction, _ hlc.Timestamp) error { et, etH := endTxnArgs(txn, false /* commit */) @@ -10936,6 +11499,7 @@ func TestTxnRecordLifecycleTransitions(t *testing.T) { disableTxnAutoGC: true, }, { + // Should not be possible. name: "recover transaction (not implicitly committed) after end transaction (commit) without eager gc", setup: func(txn *roachpb.Transaction, _ hlc.Timestamp) error { et, etH := endTxnArgs(txn, true /* commit */) diff --git a/pkg/storage/replica_write.go b/pkg/storage/replica_write.go index 1b1f005adf21..d8a32285fd64 100644 --- a/pkg/storage/replica_write.go +++ b/pkg/storage/replica_write.go @@ -30,6 +30,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/util" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/timeutil" + "github.com/pkg/errors" ) // executeWriteBatch is the entry point for client requests which may mutate the @@ -71,6 +72,12 @@ func (r *Replica) executeWriteBatch( return nil, roachpb.NewError(err) } + // NB: must be performed before collecting request spans. + ba, err := maybeStripInFlightWrites(ba) + if err != nil { + return nil, roachpb.NewError(err) + } + spans, err := r.collectSpans(&ba) if err != nil { return nil, roachpb.NewError(err) @@ -465,3 +472,141 @@ func isOnePhaseCommit(ba roachpb.BatchRequest, knobs *StoreTestingKnobs) bool { } return !knobs.DisableOptional1PC || etArg.Require1PC } + +// maybeStripInFlightWrites attempts to remove all point writes and query +// intents that ended up in the same batch as an EndTransaction request from +// that EndTransaction request's "in-flight" write set. The entire batch will +// commit atomically, so there is no need to consider the writes in the same +// batch concurrent. +// +// The transformation can lead to bypassing the STAGING state for a transaction +// entirely. This is possible if the function removes all of the in-flight +// writes from an EndTransaction request that was committing in parallel with +// writes which all happened to be on the same range as the transaction record. +func maybeStripInFlightWrites(ba roachpb.BatchRequest) (roachpb.BatchRequest, error) { + args, hasET := ba.GetArg(roachpb.EndTransaction) + if !hasET { + return ba, nil + } + + et := args.(*roachpb.EndTransactionRequest) + otherReqs := ba.Requests[:len(ba.Requests)-1] + if len(et.InFlightWrites) == 0 || !et.Commit || len(otherReqs) == 0 { + return ba, nil + } + + // Clone the BatchRequest and the EndTransaction request before modifying + // it. We nil out the request's in-flight writes and make the intent spans + // immutable on append. Code below can use origET to recreate the in-flight + // write set if any elements remain in it. + origET := et + et = origET.ShallowCopy().(*roachpb.EndTransactionRequest) + et.InFlightWrites = nil + et.IntentSpans = et.IntentSpans[:len(et.IntentSpans):len(et.IntentSpans)] // immutable + ba.Requests = append([]roachpb.RequestUnion(nil), ba.Requests...) + ba.Requests[len(ba.Requests)-1].MustSetInner(et) + + // Fast-path: If we know that this batch contains all of the transaction's + // in-flight writes, then we can avoid searching in the in-flight writes set + // for each request. Instead, we can blindly merge all in-flight writes into + // the intent spans and clear out the in-flight writes set. + if len(otherReqs) >= len(origET.InFlightWrites) { + writes := 0 + for _, ru := range otherReqs { + req := ru.GetInner() + switch { + case roachpb.IsTransactionWrite(req) && !roachpb.IsRange(req): + // Concurrent point write. + writes++ + case req.Method() == roachpb.QueryIntent: + // Earlier pipelined point write that hasn't been proven yet. + writes++ + default: + // Ranged write or read. See below. + } + } + if len(origET.InFlightWrites) < writes { + return ba, errors.New("more write in batch with EndTransaction than listed in in-flight writes") + } else if len(origET.InFlightWrites) == writes { + et.IntentSpans = make([]roachpb.Span, len(origET.IntentSpans)+len(origET.InFlightWrites)) + copy(et.IntentSpans, origET.IntentSpans) + for i, w := range origET.InFlightWrites { + et.IntentSpans[len(origET.IntentSpans)+i] = roachpb.Span{Key: w.Key} + } + // See below for why we set Header.DistinctSpans here. + et.IntentSpans, ba.Header.DistinctSpans = roachpb.MergeSpans(et.IntentSpans) + return ba, nil + } + } + + // Slow-path: If not then we remove each transaction write in the batch from + // the in-flight write set and merge it into the intent spans. + copiedTo := 0 + for _, ru := range otherReqs { + req := ru.GetInner() + seq := req.Header().Sequence + switch { + case roachpb.IsTransactionWrite(req) && !roachpb.IsRange(req): + // Concurrent point write. + case req.Method() == roachpb.QueryIntent: + // Earlier pipelined point write that hasn't been proven yet. We + // could remove from the in-flight writes set when we see these, + // but doing so would prevent us from using the optimization we + // have below where we rely on increasing sequence numbers for + // each subsequent request. + // + // We already don't intend on sending QueryIntent requests in the + // same batch as EndTransaction requests because doing so causes + // a pipeline stall, so this doesn't seem worthwhile to support. + continue + default: + // Ranged write or read. These can make it into the final batch with + // a parallel committing EndTransaction request if the entire batch + // issued by DistSender lands on the same range. Skip. + continue + } + + // Remove the write from the in-flight writes set. We only need to + // search from after the previously removed sequence number forward + // because both the InFlightWrites and the Requests in the batch are + // stored in increasing sequence order. + // + // Maintaining an iterator into the in-flight writes slice and scanning + // instead of performing a binary search on each request changes the + // complexity of this loop from O(n*log(m)) to O(m) where n is the + // number of point writes in the batch and m is the number of in-flight + // writes. These complexities aren't directly comparable, but copying + // all unstripped writes back into et.InFlightWrites is already O(m), + // so the approach here was preferred over repeat binary searches. + match := -1 + for i, w := range origET.InFlightWrites[copiedTo:] { + if w.Sequence == seq { + match = i + copiedTo + break + } + } + if match == -1 { + return ba, errors.New("write in batch with EndTransaction missing from in-flight writes") + } + w := origET.InFlightWrites[match] + notInBa := origET.InFlightWrites[copiedTo:match] + et.InFlightWrites = append(et.InFlightWrites, notInBa...) + copiedTo = match + 1 + + // Move the write to the intent spans set since it's + // no longer being tracked in the in-flight write set. + et.IntentSpans = append(et.IntentSpans, roachpb.Span{Key: w.Key}) + } + if et != origET { + // Finish building up the remaining in-flight writes. + notInBa := origET.InFlightWrites[copiedTo:] + et.InFlightWrites = append(et.InFlightWrites, notInBa...) + // Re-sort and merge the intent spans. We can set the batch request's + // DistinctSpans flag based on whether any of in-flight writes in this + // batch overlap with each other. This will have (rare) false negatives + // when the in-flight writes overlap with existing intent spans, but + // never false positives. + et.IntentSpans, ba.Header.DistinctSpans = roachpb.MergeSpans(et.IntentSpans) + } + return ba, nil +} diff --git a/pkg/storage/storagebase/knobs.go b/pkg/storage/storagebase/knobs.go index 92717f9d6780..e33925dd7cf7 100644 --- a/pkg/storage/storagebase/knobs.go +++ b/pkg/storage/storagebase/knobs.go @@ -34,6 +34,12 @@ type BatchEvalTestingKnobs struct { // NumKeysEvaluatedForRangeIntentResolution is set by the stores to the // number of keys evaluated for range intent resolution. NumKeysEvaluatedForRangeIntentResolution *int64 + // RecoverIndeterminateCommitsOnFailedPushes will propagate indeterminate + // commit errors to trigger transaction recovery even if the push that + // discovered the indeterminate commit was going to fail. This increases + // the chance that conflicting transactions will prevent parallel commit + // attempts from succeeding. + RecoverIndeterminateCommitsOnFailedPushes bool } // IntentResolverTestingKnobs contains testing helpers that are used during diff --git a/pkg/storage/store_test.go b/pkg/storage/store_test.go index 5ca566d1ef90..031e3c2fca6c 100644 --- a/pkg/storage/store_test.go +++ b/pkg/storage/store_test.go @@ -1791,13 +1791,14 @@ func TestStoreResolveWriteIntentPushOnRead(t *testing.T) { // If the pushee is staging, update the transaction record. if tc.pusheeStagingRecord { - // TODO(nvanbenschoten): Avoid writing directly to the engine once - // there's a way to create a STAGING transaction record. - txnKey := keys.TransactionKey(pushee.Key, pushee.ID) - txnRecord := pushee.AsRecord() - txnRecord.Status = roachpb.STAGING - if err := engine.MVCCPutProto(ctx, store.Engine(), nil, txnKey, hlc.Timestamp{}, nil, &txnRecord); err != nil { - t.Fatal(err) + et, etH := endTxnArgs(pushee, true) + et.InFlightWrites = []roachpb.SequencedWrite{{Key: []byte("keyA"), Sequence: 1}} + etReply, pErr := client.SendWrappedWith(ctx, store.TestSender(), etH, &et) + if pErr != nil { + t.Fatal(pErr) + } + if replyTxn := etReply.Header().Txn; replyTxn.Status != roachpb.STAGING { + t.Fatalf("expected STAGING txn, found %v", replyTxn) } } @@ -1826,13 +1827,7 @@ func TestStoreResolveWriteIntentPushOnRead(t *testing.T) { etArgs, etH := endTxnArgs(pushee, true) assignSeqNumsForReqs(pushee, &etArgs) _, pErr = client.SendWrappedWith(ctx, store.TestSender(), etH, &etArgs) - if tc.pusheeStagingRecord { - // TODO(nvanbenschoten): We don't support committing STAGING - // transaction records yet. This will need to change once we do. - if !testutils.IsPError(pErr, "TransactionStatusError: bad txn status") { - t.Fatal(pErr) - } - } else if tc.expPusheeRetry { + if tc.expPusheeRetry { if _, ok := pErr.GetDetail().(*roachpb.TransactionRetryError); !ok { t.Errorf("expected transaction retry error; got %s", pErr) } diff --git a/pkg/storage/txn_recovery_integration_test.go b/pkg/storage/txn_recovery_integration_test.go index 499a38e7c69f..77891c0615f5 100644 --- a/pkg/storage/txn_recovery_integration_test.go +++ b/pkg/storage/txn_recovery_integration_test.go @@ -21,12 +21,9 @@ import ( "testing" "github.com/cockroachdb/cockroach/pkg/internal/client" - "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/roachpb" - "github.com/cockroachdb/cockroach/pkg/storage/engine" "github.com/cockroachdb/cockroach/pkg/storage/txnwait" "github.com/cockroachdb/cockroach/pkg/testutils" - "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/stop" ) @@ -77,30 +74,17 @@ func TestTxnRecoveryFromStaging(t *testing.T) { // Issue a parallel commit, which will put the transaction into a STAGING // state. Include both writes as the EndTransaction's in-flight writes. - // - // TODO(nvanbenschoten): Avoid writing directly to the engine once - // there's a way to create a STAGING transaction record. - // - // et, _ := endTxnArgs(txn, true) - // et.IntentSpans = []roachpb.Span{{Key: keyA}, {Key: keyB}} - // et.InFlightWrites = map[int32]int32{1: 0, 2: 1} - // reply, pErr := client.SendWrappedWith(ctx, store.TestSender(), h, &et) - // if pErr != nil { - // t.Fatal(pErr) - // } - // if replyTxn := reply.Header().Txn; replyTxn.Status != roachpb.STAGING { - // t.Fatalf("expected STAGING txn, found %v", replyTxn) - // } - // - txnKey := keys.TransactionKey(txn.Key, txn.ID) - txnRecord := txn.AsRecord() - txnRecord.Status = roachpb.STAGING - txnRecord.InFlightWrites = []roachpb.SequencedWrite{ + et, etH := endTxnArgs(txn, true) + et.InFlightWrites = []roachpb.SequencedWrite{ {Key: keyA, Sequence: 0}, {Key: keyB, Sequence: 1}, } - if err := engine.MVCCPutProto(ctx, store.Engine(), nil, txnKey, hlc.Timestamp{}, nil, &txnRecord); err != nil { - t.Fatal(err) + etReply, pErr := client.SendWrappedWith(ctx, store.TestSender(), etH, &et) + if pErr != nil { + t.Fatal(pErr) + } + if replyTxn := etReply.Header().Txn; replyTxn.Status != roachpb.STAGING { + t.Fatalf("expected STAGING txn, found %v", replyTxn) } // Pretend the transaction coordinator for the parallel commit died at this point.