diff --git a/docs/generated/settings/settings.html b/docs/generated/settings/settings.html index c478d897c640..bd053e5909cb 100644 --- a/docs/generated/settings/settings.html +++ b/docs/generated/settings/settings.html @@ -59,6 +59,6 @@ trace.debug.enablebooleanfalseif set, traces for recent requests can be seen in the /debug page trace.lightstep.tokenstringif set, traces go to Lightstep using this token trace.zipkin.collectorstringif set, traces go to the given Zipkin instance (example: '127.0.0.1:9411'); ignored if trace.lightstep.token is set. -versioncustom validation2.0-5set the active cluster version in the format '.'. +versioncustom validation2.0-6set the active cluster version in the format '.'. diff --git a/pkg/cmd/roachtest/tpcc.go b/pkg/cmd/roachtest/tpcc.go index 0d2d8c8175da..cf990e1d099f 100644 --- a/pkg/cmd/roachtest/tpcc.go +++ b/pkg/cmd/roachtest/tpcc.go @@ -369,7 +369,7 @@ func registerTPCCBench(r *registry) { LoadWarehouses: 1000, EstimatedMax: 325, - StoreDirVersion: "2.0-5", + StoreDirVersion: "2.0-6", }, { Nodes: 3, @@ -394,7 +394,7 @@ func registerTPCCBench(r *registry) { LoadWarehouses: 5000, EstimatedMax: 500, - StoreDirVersion: "2.0-5", + StoreDirVersion: "2.0-6", }, // objective 3, key result 2. { diff --git a/pkg/internal/client/batch.go b/pkg/internal/client/batch.go index 19c5a3a38b3e..ba714452693f 100644 --- a/pkg/internal/client/batch.go +++ b/pkg/internal/client/batch.go @@ -167,8 +167,8 @@ func (b *Batch) fillResults(ctx context.Context) { // instead; this effectively just leaks here. // TODO(tschottdorf): returning an error here seems // to get swallowed. - panic(errors.Errorf("not enough responses for calls: %+v, %+v", - b.reqs, b.response)) + panic(errors.Errorf("not enough responses for calls: (%T) %+v\nresponses: %+v", + args, args, b.response)) } } } diff --git a/pkg/internal/client/client_test.go b/pkg/internal/client/client_test.go index 6033753970b9..28e4bdc04555 100644 --- a/pkg/internal/client/client_test.go +++ b/pkg/internal/client/client_test.go @@ -798,14 +798,18 @@ func TestReadConsistencyTypes(t *testing.T) { t.Run(rc.String(), func(t *testing.T) { // Mock out DistSender's sender function to check the read consistency for // outgoing BatchRequests and return an empty reply. - factory := client.TxnSenderFactoryFunc(func(_ client.TxnType) client.TxnSender { - return client.TxnSenderFunc(func(_ context.Context, ba roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { - if ba.ReadConsistency != rc { - return nil, roachpb.NewErrorf("BatchRequest has unexpected ReadConsistency %s", ba.ReadConsistency) - } - return ba.CreateReply(), nil - }) + factory := client.TxnSenderFactoryFunc(func(client.TxnType) client.TxnSender { + return client.TxnSenderAdapter{ + StartTrackingWrapped: func(context.Context) error { panic("unimplemented") }, + Wrapped: func(_ context.Context, ba roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { + if ba.ReadConsistency != rc { + return nil, roachpb.NewErrorf("BatchRequest has unexpected ReadConsistency %s", ba.ReadConsistency) + } + return ba.CreateReply(), nil + }, + } }) + clock := hlc.NewClock(hlc.UnixNano, time.Nanosecond) db := client.NewDB(factory, clock) ctx := context.TODO() @@ -971,10 +975,13 @@ func TestNodeIDAndObservedTimestamps(t *testing.T) { // Mock out sender function to check that created transactions // have the observed timestamp set for the configured node ID. - factory := client.TxnSenderFactoryFunc(func(_ client.TxnType) client.TxnSender { - return client.TxnSenderFunc(func(_ context.Context, ba roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { - return ba.CreateReply(), nil - }) + factory := client.TxnSenderFactoryFunc(func(client.TxnType) client.TxnSender { + return client.TxnSenderAdapter{ + StartTrackingWrapped: func(context.Context) error { panic("unimplemented") }, + Wrapped: func(_ context.Context, ba roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { + return ba.CreateReply(), nil + }, + } }) clock := hlc.NewClock(hlc.UnixNano, time.Nanosecond) diff --git a/pkg/internal/client/db.go b/pkg/internal/client/db.go index 719345f9d601..df37cca60bd2 100644 --- a/pkg/internal/client/db.go +++ b/pkg/internal/client/db.go @@ -199,11 +199,22 @@ type DB struct { ctx DBContext } -// GetSender returns a transaction-capable Sender instance. The same -// sender must be used for the entirety of a transaction. Get a new -// instance to start a new transaction. +// GetSender returns a Sender that can be used to send requests through. +// Note that a new Sender created; it is not shared. +// +// The Sender returned should not be used for sending transactional requests. +// Use db.Txn() or db.NewTxn() for that. func (db *DB) GetSender() Sender { - return db.factory.New(RootTxn) + // We pass nil for the txn here because we don't have a txn on hand. + // That's why this method says to not use the Sender for transactional + // requests, plus the fact that if a Sender is used directly, the caller needs + // to be mindful of the need to start a heartbeat loop when writing. + // + // Note that even non-transactional requests need to go through a + // TxnCoordSender because batches that get split need to be wrapped in + // transactions (and the TxnCoordSender handles that). So we can't simply + // return the wrapped handler here. + return db.factory.New(RootTxn, nil /* txn */) } // GetFactory returns the DB's TxnSenderFactory. diff --git a/pkg/internal/client/sender.go b/pkg/internal/client/sender.go index 3ab13975e815..aeddc164f016 100644 --- a/pkg/internal/client/sender.go +++ b/pkg/internal/client/sender.go @@ -69,7 +69,7 @@ type TxnSender interface { GetMeta() roachpb.TxnCoordMeta // AugmentMeta combines the TxnCoordMeta from another distributed // TxnSender which is part of the same transaction. - AugmentMeta(meta roachpb.TxnCoordMeta) + AugmentMeta(ctx context.Context, meta roachpb.TxnCoordMeta) // OnFinish invokes the supplied closure when the sender has finished // with the txn (i.e. it's been abandoned, aborted, or committed). // The error passed is meant to indicate to an extant distributed @@ -78,16 +78,21 @@ type TxnSender interface { // if this method is invoked multiple times, the most recent callback // is the only one which will be invoked. OnFinish(func(error)) + + // StartTracking starts a heartbeat loop and tracking of intents. + StartTracking(ctx context.Context) error } // TxnSenderFactory is the interface used to create new instances // of TxnSender. type TxnSenderFactory interface { - // New returns a new instance of TxnSender. The typ parameter - // specifies whether the sender is the root or one of potentially - // many child "leaf" nodes in a tree of transaction objects, as is - // created during a DistSQL flow. - New(typ TxnType) TxnSender + // New returns a new instance of TxnSender. + // typ specifies whether the sender is the root or one of potentially many + // child "leaf" nodes in a tree of transaction objects, as is created during a + // DistSQL flow. + // txn is the transaction whose requests this sender will carry. It can be nil + // if the sender will not be used for transactional requests. + New(typ TxnType, txn *roachpb.Transaction) TxnSender // WrappedSender returns the TxnSenderFactory's wrapped Sender. WrappedSender() Sender } @@ -103,33 +108,44 @@ func (f SenderFunc) Send( return f(ctx, ba) } -// TxnSenderFunc is an adapter to allow the use of ordinary functions -// as TxnSenders with GetMeta or AugmentMeta panicing with unimplemented. -// This is a helper mechanism to facilitate testing. -type TxnSenderFunc func(context.Context, roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) +// TxnSenderAdapter is an adapter to allow the use of ordinary functions as +// TxnSenders with GetMeta or AugmentMeta panicing with unimplemented. This is +// a helper mechanism to facilitate testing. +type TxnSenderAdapter struct { + Wrapped func(context.Context, roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) + StartTrackingWrapped func(context.Context) error +} // Send calls f(ctx, c). -func (f TxnSenderFunc) Send( +func (f TxnSenderAdapter) Send( ctx context.Context, ba roachpb.BatchRequest, ) (*roachpb.BatchResponse, *roachpb.Error) { - return f(ctx, ba) + return f.Wrapped(ctx, ba) } // GetMeta is part of the TxnSender interface. -func (f TxnSenderFunc) GetMeta() roachpb.TxnCoordMeta { panic("unimplemented") } +func (f TxnSenderAdapter) GetMeta() roachpb.TxnCoordMeta { panic("unimplemented") } // AugmentMeta is part of the TxnSender interface. -func (f TxnSenderFunc) AugmentMeta(_ roachpb.TxnCoordMeta) { panic("unimplemented") } +func (f TxnSenderAdapter) AugmentMeta(context.Context, roachpb.TxnCoordMeta) { panic("unimplemented") } // OnFinish is part of the TxnSender interface. -func (f TxnSenderFunc) OnFinish(_ func(error)) { panic("unimplemented") } +func (f TxnSenderAdapter) OnFinish(_ func(error)) { panic("unimplemented") } + +// StartTracking is part the TxnSender interface. +func (f TxnSenderAdapter) StartTracking(ctx context.Context) error { + if f.StartTrackingWrapped != nil { + return f.StartTrackingWrapped(ctx) + } + panic("unimplemented") +} // TxnSenderFactoryFunc is an adapter to allow the use of ordinary functions // as TxnSenderFactories. This is a helper mechanism to facilitate testing. type TxnSenderFactoryFunc func(TxnType) TxnSender // New calls f(). -func (f TxnSenderFactoryFunc) New(typ TxnType) TxnSender { +func (f TxnSenderFactoryFunc) New(typ TxnType, _ *roachpb.Transaction) TxnSender { return f(typ) } diff --git a/pkg/internal/client/txn.go b/pkg/internal/client/txn.go index c098404baf24..1b7ae6765219 100644 --- a/pkg/internal/client/txn.go +++ b/pkg/internal/client/txn.go @@ -73,10 +73,7 @@ type Txn struct { // txnAnchorKey is the key at which to anchor the transaction record. If // unset, the first key written in the transaction will be used. txnAnchorKey roachpb.Key - // writingTxnRecord is set when the Txn is in the middle of writing - // its transaction record. It is used to assure that even in the presence - // of concurrent requests, only one sends a BeginTxnRequest. - writingTxnRecord bool + state txnState // see IsFinalized() finalized bool // previousIDs holds the set of all previous IDs that the Txn's Proto has @@ -99,6 +96,40 @@ type Txn struct { } } +// txnState represents states relating to whether Begin/EndTxn requests need to +// be sent. +type txnState int + +const ( + // txnReadOnly means that the transaction never sent any writes. There's no + // transaction record, so an EndTxn does not need to be sent. + txnReadOnly txnState = iota + // txnWriting means that the transaction has sent some writes (and so it also + // sent a BeginTxn). An EndTransaction must be sent to resolve intents and/or + // to cleanup the txn record. + // txnWriting does not guarantee that the transaction record has been written. + // In case the BeginTxn batch encoutered an error, it might not have been. In + // this case, a rollback will get an error (ignored by SQL). + txnWriting + // txnWriteInOldEpoch means that the txn has been writing in an old epoch, + // but then restarted with a new epoch, and there have been no writes sent + // since then. This means that an EndTransaction(commit=false) needs to be + // sent to clean up intents. It also means that a BeginTransaction needs to be + // sent on the first write: the TransactionRestartError might have been + // received by the batch with the BeginTransaction in it, in which case there + // is no transaction record (and so it needs to be created). + // We could be smarter about not transitioning to this state if there's ever + // been a successful write (in which case we know that there is a txn record + // and a BeginTransaction is not necessary) but as of May 2018 we don't do + // that. Note that the server accepts a BeginTxn with a higher epoch if a + // transaction record already exists. + txnWriteInOldEpoch + // txnError means that the txn had performed some writes and then a batch got + // a non-retriable error. Further batches except EndTransaction(commit=false) + // will be rejected. + txnError +) + // NewTxn returns a new txn. The typ parameter specifies whether this // transaction is the top level (root), or one of potentially many // distributed transactions (leaf). @@ -141,7 +172,7 @@ func NewTxnWithProto( proto.AssertInitialized(context.TODO()) txn := &Txn{db: db, typ: typ, gatewayNodeID: gatewayNodeID} txn.mu.Proto = proto - txn.mu.sender = db.factory.New(typ) + txn.mu.sender = db.factory.New(typ, &proto) return txn } @@ -298,7 +329,7 @@ func (txn *Txn) CommitTimestamp() hlc.Timestamp { func (txn *Txn) SetTxnAnchorKey(key roachpb.Key) error { txn.mu.Lock() defer txn.mu.Unlock() - if txn.mu.Proto.Writing || txn.mu.writingTxnRecord { + if txn.mu.state != txnReadOnly { return errors.Errorf("transaction anchor key already set") } txn.mu.txnAnchorKey = key @@ -596,6 +627,14 @@ func (txn *Txn) Rollback(ctx context.Context) error { } func (txn *Txn) rollback(ctx context.Context) *roachpb.Error { + // TODO(andrei): It's common for rollbacks to fail with + // TransactionStatusError: txn record not found (REASON_TXN_NOT_FOUND), + // in case the txn record was never written (e.g. if a 1PC failed). One would + // think that, depending on the error received by a 1PC batch, we'd know if a + // txn record was written and, if it wasn't, we could short-circuit the + // rollback. There's two tricky things about that, though: a) we'd need to + // know whether the DistSender has split what looked like a 1PC batch to the + // client and b) ambiguous errors. log.VEventf(ctx, 2, "rolling back transaction") return txn.sendEndTxnReq(ctx, false /* commit */, nil) } @@ -614,19 +653,16 @@ func (txn *Txn) OnFinish(onFinishFn func(error)) { txn.mu.sender.OnFinish(onFinishFn) } -// maybeFinishReadonly provides a fast-path for finishing a read-only +// finishReadonlyLocked provides a fast-path for finishing a read-only // transaction without going through the overhead of creating an // EndTransactionRequest only to not send it. // // NB: The logic here must be kept in sync with the logic in txn.Send. // // TODO(andrei): Can we share this code with txn.Send? -func (txn *Txn) maybeFinishReadonly(commit bool, deadline *hlc.Timestamp) (bool, *roachpb.Error) { - txn.mu.Lock() - defer txn.mu.Unlock() - if txn.mu.Proto.Writing || txn.mu.writingTxnRecord { - return false, nil - } +func (txn *Txn) finishReadonlyLocked( + ctx context.Context, commit bool, deadline *hlc.Timestamp, +) *roachpb.Error { txn.mu.finalized = true // Check that read only transactions do not violate their deadline. This can NOT // happen since the txn deadline is normally updated when it is about to expire @@ -644,7 +680,7 @@ func (txn *Txn) maybeFinishReadonly(commit bool, deadline *hlc.Timestamp) (bool, // 4. new timestamp violates deadline // 5. txn retries the read // 6. commit fails - only thanks to this code path? - return false, roachpb.NewErrorWithTxn(roachpb.NewTransactionStatusError( + return roachpb.NewErrorWithTxn(roachpb.NewTransactionStatusError( "deadline exceeded before transaction finalization"), &txn.mu.Proto) } if commit { @@ -652,18 +688,35 @@ func (txn *Txn) maybeFinishReadonly(commit bool, deadline *hlc.Timestamp) (bool, } else { txn.mu.Proto.Status = roachpb.ABORTED } - return true, nil + return nil } func (txn *Txn) sendEndTxnReq( ctx context.Context, commit bool, deadline *hlc.Timestamp, ) *roachpb.Error { - if ok, err := txn.maybeFinishReadonly(commit, deadline); ok || err != nil { - return err + txn.mu.Lock() + if txn.mu.state == txnReadOnly { + defer txn.mu.Unlock() + return txn.finishReadonlyLocked(ctx, commit, deadline) + } + + var swallowErr bool + if txn.mu.state == txnWriteInOldEpoch && commit { + // If there was a write in an old epoch (and no writes in the current + // epoch), we need to send a rollback. We'll ignore the error from it; the + // commit is successful at this point. + log.VEventf(ctx, 2, "old epoch write turning commit into rollback") + commit = false + swallowErr = true } + txn.mu.Unlock() + var ba roachpb.BatchRequest ba.Add(endTxnReq(commit, deadline, txn.systemConfigTrigger)) _, pErr := txn.Send(ctx, ba) + if swallowErr { + return nil + } return pErr } @@ -878,6 +931,14 @@ func (txn *Txn) Send( txn.mu.Lock() defer txn.mu.Unlock() + if txn.mu.state == txnError { + singleAbort := ba.IsSingleEndTransactionRequest() && + !ba.Requests[0].GetInner().(*roachpb.EndTransactionRequest).Commit + if !singleAbort { + return roachpb.NewError(&roachpb.TxnAlreadyEncounteredErrorError{}) + } + } + sender = txn.mu.sender if txn.mu.Proto.Status != roachpb.PENDING || txn.mu.finalized { return roachpb.NewErrorf( @@ -899,14 +960,27 @@ func (txn *Txn) Send( txn.mu.active = true } - needBeginTxn = !(txn.mu.Proto.Writing || txn.mu.writingTxnRecord) && haveTxnWrite - needEndTxn := txn.mu.Proto.Writing || txn.mu.writingTxnRecord || haveTxnWrite + needBeginTxn = haveTxnWrite && (txn.mu.state != txnWriting) + // We need the EndTxn if we've ever written before or if we're writing now. + needEndTxn := haveTxnWrite || txn.mu.state != txnReadOnly elideEndTxn = haveEndTxn && !needEndTxn // If we're not yet writing in this txn, but intend to, insert a // begin transaction request before the first write command and update // transaction state accordingly. if needBeginTxn { + // Unless it's a 1PC, ask the TxnCoordSender to track the transaction. + if txn.mu.state == txnReadOnly && !haveEndTxn { + if err := txn.mu.sender.StartTracking(ctx); err != nil { + return roachpb.NewError(err) + } + } + // We're about to send a BeginTxn, so move to the Writing state. + txn.mu.state = txnWriting + // From now on, all requests need to be checked against the AbortCache on + // the server side. + txn.mu.Proto.Writing = true + // Set txn key based on the key of the first transactional write if // not already set. If the transaction already has a key (we're in a // restart), make sure we keep the anchor key the same. @@ -930,9 +1004,6 @@ func (txn *Txn) Send( copy(ba.Requests, oldRequests[:firstWriteIdx]) ba.Requests[firstWriteIdx].MustSetInner(bt) copy(ba.Requests[firstWriteIdx+1:], oldRequests[firstWriteIdx:]) - // We're going to be writing the transaction record by sending the - // begin transaction request. - txn.mu.writingTxnRecord = true } if elideEndTxn { @@ -976,9 +1047,7 @@ func (txn *Txn) Send( txn.mu.Lock() defer txn.mu.Unlock() - // If we inserted a begin transaction request, remove it here. We also - // unset the flag writingTxnRecord flag in case another ever needs to - // be sent again (for instance, if we're aborted and need to restart). + // If we inserted a begin transaction request, remove it here. if needBeginTxn { if br != nil && br.Responses != nil { br.Responses = append(br.Responses[:firstWriteIdx], br.Responses[firstWriteIdx+1:]...) @@ -994,8 +1063,6 @@ func (txn *Txn) Send( pErr.SetErrorIndex(idx - 1) } } - - txn.mu.writingTxnRecord = false } if haveEndTxn { if pErr == nil || !endTxnRequest.Commit { @@ -1006,12 +1073,19 @@ func (txn *Txn) Send( } } + // If the request was part of a previous attempt, don't return any results. + if requestTxnID != txn.mu.Proto.ID || ba.Txn.Epoch != txn.mu.Proto.Epoch { + return nil, roachpb.NewError(&roachpb.TxnPrevAttemptError{}) + } + if pErr != nil { if log.V(1) { log.Infof(ctx, "failed batch: %s", pErr) } + var retriable bool switch t := pErr.GetDetail().(type) { case *roachpb.HandledRetryableTxnError: + retriable = true retryErr := t if requestTxnID != retryErr.TxnID { // KV should not return errors for transactions other than the one that sent @@ -1019,21 +1093,25 @@ func (txn *Txn) Send( log.Fatalf(ctx, "retryable error for the wrong txn. "+ "requestTxnID: %s, retryErr.TxnID: %s. retryErr: %s", requestTxnID, retryErr.TxnID, retryErr) - } else if requestTxnID == txn.mu.Proto.ID { - // Our requestTxnID still matches the proto, so update the state. - // If it doesn't match here, it means a concurrent request through - // this Txn object has already aborted and restarted the txn. - txn.updateStateOnRetryableErrLocked(ctx, retryErr) } + txn.updateStateOnRetryableErrLocked(ctx, retryErr) } // Note that unhandled retryable txn errors are allowed from leaf // transactions. We pass them up through distributed SQL flows to // the root transactions, at the receiver. - if txn.typ == RootTxn && pErr.TransactionRestart != roachpb.TransactionRestart_NONE { - log.Fatalf(ctx, - "unexpected retryable error at the client.Txn level: (%T) %s", - pErr.GetDetail(), pErr) + if pErr.TransactionRestart != roachpb.TransactionRestart_NONE { + retriable = true + if txn.typ == RootTxn { + log.Fatalf(ctx, + "unexpected retryable error at the client.Txn level: (%T) %s", + pErr.GetDetail(), pErr) + } + } + + if !retriable && txn.mu.state == txnWriting { + txn.mu.state = txnError } + return nil, pErr } @@ -1041,9 +1119,6 @@ func (txn *Txn) Send( if br.Error != nil { panic(roachpb.ErrorUnexpectedlySet(txn.mu.sender, br)) } - if br.Txn != nil && br.Txn.ID != txn.mu.Proto.ID { - return nil, roachpb.NewError(&roachpb.TxnPrevAttemptError{}) - } // Only successful requests can carry an updated Txn in their response // header. Some errors (e.g. a restart) have a Txn attached to them as @@ -1108,11 +1183,11 @@ func (txn *Txn) GetTxnCoordMeta() roachpb.TxnCoordMeta { // AugmentTxnCoordMeta augments this transaction's TxnCoordMeta // information with the supplied meta. For use with GetTxnCoordMeta(). -func (txn *Txn) AugmentTxnCoordMeta(meta roachpb.TxnCoordMeta) { +func (txn *Txn) AugmentTxnCoordMeta(ctx context.Context, meta roachpb.TxnCoordMeta) { txn.mu.Lock() defer txn.mu.Unlock() txn.mu.Proto.Update(&meta.Txn) - txn.mu.sender.AugmentMeta(meta) + txn.mu.sender.AugmentMeta(ctx, meta) } // UpdateStateOnRemoteRetryableErr updates the Txn, and the @@ -1167,9 +1242,11 @@ func (txn *Txn) updateStateOnRetryableErrLocked( // attempt. The txn inside pErr was correctly prepared for this by // TxnCoordSender. txn.mu.Proto = *newTxn + // We're starting a fresh transaction, with no state. + txn.mu.state = txnReadOnly // Create a new txn sender. - txn.mu.sender = txn.db.factory.New(txn.typ) + txn.mu.sender = txn.db.factory.New(txn.typ, newTxn) } else { // Update the transaction proto with the one to be used for the next // attempt. The txn inside pErr was correctly prepared for this by @@ -1178,6 +1255,12 @@ func (txn *Txn) updateStateOnRetryableErrLocked( // we rely on the associativity of Transaction.Update to sort out this // lack of ordering guarantee. txn.mu.Proto.Update(newTxn) + + // If we've been writing, we'll need to send a BeginTxn with the next + // request. + if txn.mu.state == txnWriting { + txn.mu.state = txnWriteInOldEpoch + } } } diff --git a/pkg/internal/client/txn_test.go b/pkg/internal/client/txn_test.go index 4e8d4bf8dcbc..1bc64b2e14b1 100644 --- a/pkg/internal/client/txn_test.go +++ b/pkg/internal/client/txn_test.go @@ -70,7 +70,11 @@ func TestTxnSnowballTrace(t *testing.T) { // 0.275ms 0.171ms event:client.Txn did AutoCommit. err: //txn: "internal/client/txn_test.go:67 TestTxnSnowballTrace" id= key=/Min rw=false pri=0.00000000 iso=SERIALIZABLE stat=COMMITTED epo=0 ts=0.000000000,0 orig=0.000000000,0 max=0.000000000,0 wto=false rop=false // 0.278ms 0.173ms event:txn complete - found, err := regexp.MatchString(".*event:inside txn\n.*event:client.Txn did AutoCommit. err: \n.*\n.*event:txn complete.*", dump) + found, err := regexp.MatchString( + // The (?s) makes "." match \n. This makes the test resilient to other log + // lines being interspersed. + "(?s).*event:inside txn\n.*event:client.Txn did AutoCommit. err: \n.*\n.*event:txn complete.*", + dump) if err != nil { t.Fatal(err) } @@ -82,55 +86,58 @@ func TestTxnSnowballTrace(t *testing.T) { func newTestTxnFactory( createReply func(roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error), ) TxnSenderFactoryFunc { - return TxnSenderFactoryFunc(func(_ TxnType) TxnSender { - return TxnSenderFunc(func(_ context.Context, ba roachpb.BatchRequest, - ) (*roachpb.BatchResponse, *roachpb.Error) { - if ba.UserPriority == 0 { - ba.UserPriority = 1 - } + return TxnSenderFactoryFunc(func(TxnType) TxnSender { + return TxnSenderAdapter{ + StartTrackingWrapped: func(context.Context) error { return nil }, + Wrapped: func(_ context.Context, ba roachpb.BatchRequest, + ) (*roachpb.BatchResponse, *roachpb.Error) { + if ba.UserPriority == 0 { + ba.UserPriority = 1 + } - var br *roachpb.BatchResponse - var pErr *roachpb.Error - if createReply != nil { - br, pErr = createReply(ba) - } else { - br = ba.CreateReply() - } - if pErr != nil { - return nil, pErr - } - var writing bool - status := roachpb.PENDING - for i, req := range ba.Requests { - args := req.GetInner() - if _, ok := args.(*roachpb.PutRequest); ok { - testPutRespCopy := testPutResp - union := &br.Responses[i] // avoid operating on copy - union.MustSetInner(&testPutRespCopy) + var br *roachpb.BatchResponse + var pErr *roachpb.Error + if createReply != nil { + br, pErr = createReply(ba) + } else { + br = ba.CreateReply() } - if roachpb.IsTransactionWrite(args) { - writing = true + if pErr != nil { + return nil, pErr } - } - if args, ok := ba.GetArg(roachpb.EndTransaction); ok { - et := args.(*roachpb.EndTransactionRequest) - writing = true - if et.Commit { - status = roachpb.COMMITTED - } else { - status = roachpb.ABORTED + var writing bool + status := roachpb.PENDING + for i, req := range ba.Requests { + args := req.GetInner() + if _, ok := args.(*roachpb.PutRequest); ok { + testPutRespCopy := testPutResp + union := &br.Responses[i] // avoid operating on copy + union.MustSetInner(&testPutRespCopy) + } + if roachpb.IsTransactionWrite(args) { + writing = true + } } - } - if ba.Txn != nil && br.Txn == nil { - txnClone := ba.Txn.Clone() - br.Txn = &txnClone - if pErr == nil { - br.Txn.Writing = writing - br.Txn.Status = status + if args, ok := ba.GetArg(roachpb.EndTransaction); ok { + et := args.(*roachpb.EndTransactionRequest) + writing = true + if et.Commit { + status = roachpb.COMMITTED + } else { + status = roachpb.ABORTED + } } - } - return br, pErr - }) + if ba.Txn != nil && br.Txn == nil { + txnClone := ba.Txn.Clone() + br.Txn = &txnClone + if pErr == nil { + br.Txn.Writing = writing + br.Txn.Status = status + } + } + return br, pErr + }, + } }) } diff --git a/pkg/kv/dist_sender_server_test.go b/pkg/kv/dist_sender_server_test.go index f7c2799d0191..21de65a9dd12 100644 --- a/pkg/kv/dist_sender_server_test.go +++ b/pkg/kv/dist_sender_server_test.go @@ -1774,8 +1774,7 @@ func TestTxnStarvation(t *testing.T) { // TestTxnCoordSenderHeartbeatFailurePostSplit verifies that on // heartbeat timeout, the transaction is aborted asynchronously, // leaving abort span entries which cause concurrent reads to fail -// with txn aborted errors on both the range the transaction started -// on and a separate range involved in the same transaction. +// with txn aborted errors on a range different that the txn's anchor. // // Note that this is a post-split version of TestTxnCoordSenderGCTimeout. func TestTxnCoordSenderHeartbeatFailurePostSplit(t *testing.T) { @@ -1836,7 +1835,6 @@ func TestTxnCoordSenderHeartbeatFailurePostSplit(t *testing.T) { }() return errCh } - errChA := startReader(keyA) errChB := startReader(keyB) stores := s.GetStores().(*storage.Stores) @@ -1863,9 +1861,6 @@ func TestTxnCoordSenderHeartbeatFailurePostSplit(t *testing.T) { // Now signal the inflight readers to continue; they should witness // abort span entries. close(signal) - if err := <-errChA; !testutils.IsError(err, "txn aborted") { - t.Errorf("expected transaction aborted error reading %s; got %s", keyA, err) - } if err := <-errChB; !testutils.IsError(err, "txn aborted") { t.Errorf("expected transaction aborted error reading %s; got %s", keyB, err) } @@ -1955,31 +1950,8 @@ func TestTxnCoordSenderRetries(t *testing.T) { return err }, retryable: func(ctx context.Context, txn *client.Txn) error { - return txn.InitPut(ctx, "a", "put", false) // put to advance txn ts - }, - }, - { - name: "forwarded timestamp with get and initput value exists", - beforeTxnStart: func(ctx context.Context, db *client.DB) error { - return db.Put(ctx, "a", "put") - }, - afterTxnStart: func(ctx context.Context, db *client.DB) error { - _, err := db.Get(ctx, "a") // read key to set ts cache - return err - }, - retryable: func(ctx context.Context, txn *client.Txn) error { - err := txn.InitPut(ctx, "a", "init-put", false) // init-put to advance txn ts - // Swallow expected condition failed error. - if _, ok := err.(*roachpb.ConditionFailedError); !ok { - if err != nil { - return errors.New("expected condition failed error") - } - return err - } - log.Infof(ctx, "Swallowed error") - return nil + return txn.InitPut(ctx, "a", "put", false /* failOnTombstones */) // put to advance txn ts }, - // No retries, this is a straight failure. }, { name: "forwarded timestamp with get and cput", diff --git a/pkg/kv/txn_coord_sender.go b/pkg/kv/txn_coord_sender.go index 7f64698a8016..35ce8e15224f 100644 --- a/pkg/kv/txn_coord_sender.go +++ b/pkg/kv/txn_coord_sender.go @@ -100,6 +100,14 @@ const ( type TxnCoordSender struct { mu struct { syncutil.Mutex + + // tracking is set if the TxnCoordSender has a heartbeat loop running for + // the transaction record. It also means that the TxnCoordSender is + // accumulating intents for the transaction. + // tracking is set by the client just before a BeginTransaction request is + // sent. If set, an EndTransaction will also be sent eventually to clean up. + tracking bool + // meta contains all coordinator state which may be passed between // distributed TxnCoordSenders via MetaRelease() and MetaAugment(). meta roachpb.TxnCoordMeta @@ -259,17 +267,30 @@ func NewTxnCoordSenderFactory( } } -// New is part of the TxnCoordSenderFactory interface. -func (tcf *TxnCoordSenderFactory) New(typ client.TxnType) client.TxnSender { +// New is part of the TxnSenderFactory interface. +func (tcf *TxnCoordSenderFactory) New( + typ client.TxnType, txn *roachpb.Transaction, +) client.TxnSender { tcs := &TxnCoordSender{ typ: typ, TxnCoordSenderFactory: tcf, } tcs.mu.meta.RefreshValid = true + + // If a transaction was passed in bind the TxnCoordSender to it. + // TODO(andrei): Ideally, if a transaction is not passed it, we should take + // that to mean that a TxnCoordSender is not needed and we should return the + // wrapped sender directly. However, there are tests that pass nil and still + // send transactional requests. That's why the TxnCoordSender is still + // littered with code handling the case where it is not yet bound to a + // transaction. + if txn != nil { + tcs.mu.meta.Txn = txn.Clone() + } return tcs } -// WrappedSender is part of the TxnCoordSenderFactory interface. +// WrappedSender is part of the TxnSenderFactory interface. func (tcf *TxnCoordSenderFactory) WrappedSender() client.Sender { return tcf.wrapped } @@ -295,11 +316,16 @@ func (tc *TxnCoordSender) GetMeta() roachpb.TxnCoordMeta { } // AugmentMeta is part of the client.TxnSender interface. -func (tc *TxnCoordSender) AugmentMeta(meta roachpb.TxnCoordMeta) { +func (tc *TxnCoordSender) AugmentMeta(ctx context.Context, meta roachpb.TxnCoordMeta) { tc.mu.Lock() defer tc.mu.Unlock() + + if tc.mu.meta.Txn.ID == (uuid.UUID{}) { + log.Fatalf(ctx, "cannot AugmentMeta on unbound TxnCoordSender. meta id: %s", meta.Txn.ID) + } + // Sanity check: don't combine if the meta is for a different txn ID. - if tc.mu.meta.Txn.ID != (uuid.UUID{}) && tc.mu.meta.Txn.ID != meta.Txn.ID { + if tc.mu.meta.Txn.ID != meta.Txn.ID { return } tc.mu.meta.Txn.Update(&meta.Txn) @@ -343,15 +369,12 @@ func (tc *TxnCoordSender) OnFinish(onFinishFn func(error)) { tc.mu.onFinishFn = onFinishFn } -// Send implements the batch.Sender interface. If the request is part of a -// transaction, the TxnCoordSender adds the transaction to a map of active -// transactions and begins heartbeating it. Every subsequent request for the -// same transaction updates the lastUpdate timestamp to prevent live -// transactions from being considered abandoned and garbage collected. +// Send implements the batch.Sender interface. +// // Read/write mutating requests have their key or key range added to the // transaction's interval tree of key ranges for eventual cleanup via resolved -// write intents; they're tagged to an outgoing EndTransaction request, with -// the receiving replica in charge of resolving them. +// write intents; they're tagged to an outgoing EndTransaction request, with the +// receiving replica in charge of resolving them. func (tc *TxnCoordSender) Send( ctx context.Context, ba roachpb.BatchRequest, ) (*roachpb.BatchResponse, *roachpb.Error) { @@ -369,6 +392,10 @@ func (tc *TxnCoordSender) Send( startNS := tc.clock.PhysicalNow() if ba.Txn != nil { + if tc.mu.meta.Txn.ID == (uuid.UUID{}) { + log.Fatalf(ctx, "cannot send transactional request through unbound TxnCoordSender") + } + ctx = log.WithLogTag(ctx, "txn", uuid.ShortStringer(ba.Txn.ID)) if log.V(2) { ctx = log.WithLogTag(ctx, "ts", ba.Txn.Timestamp) @@ -409,14 +436,24 @@ func (tc *TxnCoordSender) Send( tc.mu.Lock() defer tc.mu.Unlock() - if tc.mu.meta.Txn.ID == (uuid.UUID{}) { - // Ensure that the txn is bound. - tc.mu.meta.Txn = ba.Txn.Clone() - } - if ba.Txn.Writing { - if pErr := tc.maybeRejectClientLocked(ctx, ba.Txn.ID); pErr != nil { - return pErr - } + // Copy a few fields from the request's txn. This is technically only + // required during the first send, as these fields are set before + // the first send and can't change afterwards. Keeping these fields in + // sync between the TxnCoordSender and the client.Txn is needed because, + // when the TxnCoordSender synthesizes TransactionAbortedErrors, it + // creates a new proto that it passes to the client.Txn and then these + // fields are used when creating that proto that will then be used for the + // client.Txn. On subsequent retries of the transaction, it's important + // for the values of these fields to have been preserved because that + // makes future calls to txn.SetIsolation() and such no-ops. + // If this makes no sense it's because the TxnCoordSender having a copy of + // the Transaction proto generally makes no sense. + tc.mu.meta.Txn.Name = ba.Txn.Name + tc.mu.meta.Txn.Isolation = ba.Txn.Isolation + tc.mu.meta.Txn.Priority = ba.Txn.Priority + + if pErr := tc.maybeRejectClientLocked(ctx, ba.Txn.ID); pErr != nil { + return pErr } tc.mu.meta.CommandCount += int32(len(ba.Requests)) @@ -851,24 +888,19 @@ func (tc *TxnCoordSender) maybeRejectClientLocked( // validateTxn validates properties of a txn specified on a request. // The transaction is expected to be initialized by the time it reaches -// the TxnCoordSender. Furthermore, no transactional writes are allowed -// unless preceded by a begin transaction request within the same batch. -// The exception is if the transaction is already in state txn.Writing=true. +// the TxnCoordSender. func (tc *TxnCoordSender) validateTxnForBatch(ctx context.Context, ba *roachpb.BatchRequest) error { if len(ba.Requests) == 0 { return errors.Errorf("empty batch with txn") } ba.Txn.AssertInitialized(ctx) - // Check for a begin transaction to set txn key based on the key of - // the first transactional write. Also enforce that no transactional - // writes occur before a begin transaction. var haveBeginTxn bool for _, req := range ba.Requests { args := req.GetInner() if _, ok := args.(*roachpb.BeginTransactionRequest); ok { - if haveBeginTxn || ba.Txn.Writing { - return errors.Errorf("begin transaction requested twice in the same txn: %s", ba.Txn) + if haveBeginTxn { + return errors.Errorf("begin transaction requested twice in the same batch: %s", ba.Txn) } if ba.Txn.Key == nil { return errors.Errorf("transaction with BeginTxnRequest missing anchor key: %v", ba) @@ -977,8 +1009,17 @@ func (tc *TxnCoordSender) heartbeatLoop(ctx context.Context) { // then heartbeat loop ignores the timeout check and this case is // responsible for client timeouts. log.VEventf(ctx, 2, "transaction heartbeat stopped: %s", ctx.Err()) - tc.tryAsyncAbort(ctx) - return + + // Check if the closer channel had also been closed; in that case, that + // takes priority. + select { + case <-closer: + // Transaction finished normally. + return + default: + tc.tryAsyncAbort(ctx) + return + } case <-tc.stopper.ShouldQuiesce(): return } @@ -1000,10 +1041,15 @@ func (tc *TxnCoordSender) tryAsyncAbort(ctx context.Context) { // raced here. That's fine (and probably better than the alternative, which // is missing new intents sometimes). Note that the txn may be uninitialized // here if a failure occurred before the first write succeeded. - if txn.Status != roachpb.PENDING || txn.ID == (uuid.UUID{}) { + if txn.Status != roachpb.PENDING { return } + // Update out status to Aborted, since we're about to send a rollback. Besides + // being sane, this prevents the heartbeat loop from incrementing an + // "Abandons" metric. + tc.mu.meta.Txn.Status = roachpb.ABORTED + // NB: use context.Background() here because we may be called when the // caller's context has been canceled. if err := tc.stopper.RunAsyncTask( @@ -1136,6 +1182,39 @@ func (tc *TxnCoordSender) heartbeat(ctx context.Context) bool { return true } +// StartTracking is part of the client.TxnSender interface. +func (tc *TxnCoordSender) StartTracking(ctx context.Context) error { + tc.mu.Lock() + defer tc.mu.Unlock() + + tc.mu.tracking = true + tc.mu.firstUpdateNanos = tc.clock.PhysicalNow() + + // Only heartbeat the txn record if we're the root transaction. + if tc.typ != client.RootTxn { + return nil + } + + log.VEventf(ctx, 2, "coordinator spawns heartbeat loop") + // Create a channel to stop the heartbeat with the lock held + // to avoid a race between the async task and a subsequent commit. + tc.mu.txnEnd = make(chan struct{}) + if err := tc.stopper.RunAsyncTask( + ctx, "kv.TxnCoordSender: heartbeat loop", func(ctx context.Context) { + tc.heartbeatLoop(ctx) + }); err != nil { + // The system is already draining and we can't start the + // heartbeat. We refuse new transactions for now because + // they're likely not going to have all intents committed. + // In principle, we can relax this as needed though. + tc.cleanupTxnLocked(ctx, aborted) + duration, restarts, status := tc.finalTxnStatsLocked() + tc.updateStats(duration, restarts, status, false /* onePC */) + return err + } + return nil +} + // updateState updates the transaction state in both the success and // error cases, applying those updates to the corresponding txnMeta // object when adequate. It also updates retryable errors with the @@ -1284,71 +1363,20 @@ func (tc *TxnCoordSender) updateState( // a serializable retry error. We can use the set of read spans to // avoid retrying the transaction if all the spans can be updated to // the current transaction timestamp. - // - // A tricky edge case is that of a transaction which "fails" on the - // first writing request, but actually manages to write some intents - // (for example, due to being multi-range). In this case, there will - // be an error, but the transaction will be marked as Writing and the - // coordinator must track the state, for the client's retry will be - // performed with a Writing transaction which the coordinator rejects - // unless it is tracking it (on top of it making sense to track it; - // after all, it **has** laid down intents and only the coordinator - // can augment a potential EndTransaction call). See #3303. - // - // An extension of this case is that of a transaction which receives an - // ambiguous result error on its first writing request. Here, the - // transaction will not be marked as Writing, but still could have laid - // down intents (we don't know, it's ambiguous!). As with the other case, - // we still track the possible writes so they can be cleaned up cleanup - // to avoid dangling intents. However, since the Writing flag is not - // set in these cases, it may be possible that the request was read-only. - // This is ok, since the following block will be a no-op if the batch - // contained no transactional write requests. - _, ambiguousErr := pErr.GetDetail().(*roachpb.AmbiguousResultError) - if pErr == nil || ambiguousErr || newTxn.Writing { + if tc.mu.tracking { // Adding the intents even on error reduces the likelihood of dangling // intents blocking concurrent writers for extended periods of time. // See #3346. tc.appendAndCondenseIntentsLocked(ctx, ba, br) - - // Initialize the first update time and maybe start the heartbeat. - if tc.mu.firstUpdateNanos == 0 && len(tc.mu.meta.Intents) > 0 { - // If the transaction is already over, there's no point in - // launching a one-off heartbeat which will shut down right - // away. If we ended up here with an error, we'll always start - // the coordinator - the transaction has laid down intents, so - // we expect it to be committed/aborted at some point in the - // future. - if _, isEnding := ba.GetArg(roachpb.EndTransaction); pErr != nil || !isEnding { - log.Event(ctx, "coordinator spawns") - tc.mu.firstUpdateNanos = startNS - - // Only heartbeat the txn record if we're the root transaction. - if tc.typ == client.RootTxn { - // Create a channel to stop the heartbeat with the lock held - // to avoid a race between the async task and a subsequent commit. - tc.mu.txnEnd = make(chan struct{}) - if err := tc.stopper.RunAsyncTask( - ctx, "kv.TxnCoordSender: heartbeat loop", func(ctx context.Context) { - tc.heartbeatLoop(ctx) - }); err != nil { - // The system is already draining and we can't start the - // heartbeat. We refuse new transactions for now because - // they're likely not going to have all intents committed. - // In principle, we can relax this as needed though. - tc.cleanupTxnLocked(ctx, aborted) - duration, restarts, status := tc.finalTxnStatsLocked() - tc.updateStats(duration, restarts, status, false) - return roachpb.NewError(err) - } - } - } else { - // If this was a successful one phase commit, update stats - // directly as they won't otherwise be updated on heartbeat - // loop shutdown. - etArgs, ok := br.Responses[len(br.Responses)-1].GetInner().(*roachpb.EndTransactionResponse) - tc.updateStats(tc.clock.PhysicalNow()-startNS, 0, newTxn.Status, ok && etArgs.OnePhaseCommit) - } + } else { + // If this was a successful one phase commit, update stats + // directly as they won't otherwise be updated on heartbeat + // loop shutdown. + _, isBeginning := ba.GetArg(roachpb.BeginTransaction) + _, isEnding := ba.GetArg(roachpb.EndTransaction) + if pErr == nil && isBeginning && isEnding { + etArgs, ok := br.Responses[len(br.Responses)-1].GetInner().(*roachpb.EndTransactionResponse) + tc.updateStats(tc.clock.PhysicalNow()-startNS, 0, newTxn.Status, ok && etArgs.OnePhaseCommit) } } diff --git a/pkg/kv/txn_coord_sender_test.go b/pkg/kv/txn_coord_sender_test.go index 3a15de7e0d4d..c3a5d18f3a3f 100644 --- a/pkg/kv/txn_coord_sender_test.go +++ b/pkg/kv/txn_coord_sender_test.go @@ -688,9 +688,9 @@ func TestTxnCoordSenderCancel(t *testing.T) { ctx, cancel := context.WithCancel(context.Background()) - tc := s.DB.GetSender().(*TxnCoordSender) - origSender := tc.TxnCoordSenderFactory.wrapped - tc.TxnCoordSenderFactory.wrapped = client.SenderFunc( + factory := s.DB.GetFactory().(*TxnCoordSenderFactory) + origSender := factory.WrappedSender() + factory.wrapped = client.SenderFunc( func(ctx context.Context, args roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { if _, hasET := args.GetArg(roachpb.EndTransaction); hasET { // Cancel the transaction while also sending it along. This tickled a @@ -1109,7 +1109,7 @@ func TestTxnCoordIdempotentCleanup(t *testing.T) { } // TestTxnMultipleCoord checks that multiple txn coordinators can be -// used by a single transaction, and their state can be combined. +// used for reads by a single transaction, and their state can be combined. func TestTxnMultipleCoord(t *testing.T) { defer leaktest.AfterTest(t)() s := createTestDB(t) @@ -1118,66 +1118,36 @@ func TestTxnMultipleCoord(t *testing.T) { ctx := context.Background() txn := client.NewTxn(s.DB, 0 /* gatewayNodeID */, client.RootTxn) tc := txn.Sender().(*TxnCoordSender) - defer teardownHeartbeat(tc) // Start the transaction. key := roachpb.Key("a") - if err := txn.Put(ctx, key, []byte("value")); err != nil { - t.Fatalf("expected error %s", err) + if _, err := txn.Get(ctx, key); err != nil { + t.Fatal(err) } // New create a second, leaf coordinator. txn2 := client.NewTxnWithProto(s.DB, 0 /* gatewayNodeID */, client.LeafTxn, *txn.Proto()) - tc2 := txn2.Sender().(*TxnCoordSender) - defer teardownHeartbeat(tc2) // Start the second transaction. key2 := roachpb.Key("b") - if err := txn2.Put(ctx, key2, []byte("value2")); err != nil { - t.Fatalf("expected error %s", err) - } - - // Verify heartbeat started on root txn. - testutils.SucceedsSoon(t, func() error { - tc.mu.Lock() - defer tc.mu.Unlock() - if tc.mu.txnEnd == nil { - return errors.New("expected heartbeat on root coordinator") - } - return nil - }) - // Verify no heartbeat started on leaf txn. - tc2.mu.Lock() - if tc2.mu.txnEnd != nil { - t.Fatalf("unexpected heartbeat on leaf coordinator") - } - tc2.mu.Unlock() - - // Verify it's an error to commit on the leaf txn node. - ba := txn2.NewBatch() - ba.AddRawRequest(&roachpb.EndTransactionRequest{Commit: true}) - if err := txn2.Run(context.TODO(), ba); !testutils.IsError(err, "cannot commit on a leaf transaction coordinator") { - t.Fatalf("expected cannot commit on leaf coordinator error; got %v", err) + if _, err := txn2.Get(ctx, key2); err != nil { + t.Fatal(err) } // Augment txn with txn2's meta & commit. - txn.AugmentTxnCoordMeta(txn2.GetTxnCoordMeta()) + txn.AugmentTxnCoordMeta(ctx, txn2.GetTxnCoordMeta()) // Verify presence of both intents. tc.mu.Lock() - if a, e := tc.mu.meta.Intents, []roachpb.Span{{Key: key}, {Key: key2}}; !reflect.DeepEqual(a, e) { - t.Fatalf("expected intents %+v; got %+v", e, a) - } - if a, e := tc.mu.intentsSizeBytes, int64(len(key)+len(key2)); a != e { - t.Fatalf("expected intentsSizeBytes %d; got %d", e, a) + if a, e := tc.mu.meta.RefreshReads, []roachpb.Span{{Key: key}, {Key: key2}}; !reflect.DeepEqual(a, e) { + tc.mu.Unlock() + t.Fatalf("expected read spans %+v; got %+v", e, a) } tc.mu.Unlock() - ba = txn.NewBatch() + ba := txn.NewBatch() ba.AddRawRequest(&roachpb.EndTransactionRequest{Commit: true}) - if err := txn.Run(context.TODO(), ba); err != nil { + if err := txn.Run(ctx, ba); err != nil { t.Fatal(err) } - - verifyCleanup(key, s.Eng, t, tc, tc2) } // TestTxnCoordSenderSingleRoundtripTxn checks that a batch which completely @@ -1201,7 +1171,6 @@ func TestTxnCoordSenderSingleRoundtripTxn(t *testing.T) { ambient, cluster.MakeTestingClusterSettings(), senderFn, clock, false, stopper, MakeTxnMetrics(metric.TestSampleInterval), ) - tc := factory.New(client.RootTxn) // Stop the stopper manually, prior to trying the transaction. This has the // effect of returning a NodeUnavailableError for any attempts at launching @@ -1214,6 +1183,7 @@ func TestTxnCoordSenderSingleRoundtripTxn(t *testing.T) { ba.Add(&roachpb.PutRequest{RequestHeader: roachpb.RequestHeader{Key: key}}) ba.Add(&roachpb.EndTransactionRequest{}) txn := roachpb.MakeTransaction("test", key, 0, 0, clock.Now(), 0) + tc := factory.New(client.RootTxn, &txn) ba.Txn = &txn _, pErr := tc.Send(context.Background(), ba) if pErr != nil { @@ -1263,8 +1233,6 @@ func TestTxnCoordSenderErrorWithIntent(t *testing.T) { stopper, MakeTxnMetrics(metric.TestSampleInterval), ) - tc := factory.New(client.RootTxn) - defer teardownHeartbeat(tc.(*TxnCoordSender)) var ba roachpb.BatchRequest key := roachpb.Key("test") @@ -1272,6 +1240,8 @@ func TestTxnCoordSenderErrorWithIntent(t *testing.T) { ba.Add(&roachpb.PutRequest{RequestHeader: roachpb.RequestHeader{Key: key}}) ba.Add(&roachpb.EndTransactionRequest{}) txn := roachpb.MakeTransaction("test", key, 0, 0, clock.Now(), 0) + tc := factory.New(client.RootTxn, &txn) + defer teardownHeartbeat(tc.(*TxnCoordSender)) ba.Txn = &txn _, pErr := tc.Send(context.Background(), ba) if !testutils.IsPError(pErr, test.errMsg) { @@ -1418,7 +1388,7 @@ func checkTxnMetricsOnce( func setupMetricsTest(t *testing.T) (*localtestcluster.LocalTestCluster, TxnMetrics, func()) { s := createTestDB(t) metrics := MakeTxnMetrics(metric.TestSampleInterval) - s.DB.GetSender().(*TxnCoordSender).TxnCoordSenderFactory.metrics = metrics + s.DB.GetFactory().(*TxnCoordSenderFactory).metrics = metrics return s, metrics, func() { s.Stop() } @@ -1459,7 +1429,8 @@ func TestTxnOnePhaseCommit(t *testing.T) { defer cleanupFn() value := []byte("value") - if err := s.DB.Txn(context.TODO(), func(ctx context.Context, txn *client.Txn) error { + ctx := context.TODO() + if err := s.DB.Txn(ctx, func(ctx context.Context, txn *client.Txn) error { key := []byte("key-commit") b := txn.NewBatch() b.Put(key, value) @@ -1467,6 +1438,20 @@ func TestTxnOnePhaseCommit(t *testing.T) { }); err != nil { t.Fatal(err) } + kv, err := s.DB.Get(ctx, []byte("key-commit")) + if err != nil { + t.Fatal(err) + } + if kv.Value == nil { + t.Fatal("expected value not found") + } + val, err := kv.Value.GetBytes() + if err != nil { + t.Fatal(err) + } + if !bytes.Equal(val, value) { + t.Fatalf("expected: %s, got: %s", value, val) + } checkTxnMetrics(t, metrics, "commit 1PC txn", 1, 1 /* 1PC */, 0, 0, 0) } diff --git a/pkg/roachpb/api.pb.go b/pkg/roachpb/api.pb.go index 1c2107d0a7fc..40703cff4a05 100644 --- a/pkg/roachpb/api.pb.go +++ b/pkg/roachpb/api.pb.go @@ -147,6 +147,7 @@ UnhandledRetryableError HandledRetryableTxnError TxnPrevAttemptError + TxnAlreadyEncounteredErrorError IntegerOverflowError MixedSuccessError BatchTimestampBeforeGCError diff --git a/pkg/roachpb/batch.go b/pkg/roachpb/batch.go index 1473893fff7a..228422166c6a 100644 --- a/pkg/roachpb/batch.go +++ b/pkg/roachpb/batch.go @@ -404,6 +404,9 @@ func (ba BatchRequest) String() string { req := arg.GetInner() if _, ok := req.(*NoopRequest); ok { str = append(str, req.Method().String()) + } else if et, ok := req.(*EndTransactionRequest); ok { + h := req.Header() + str = append(str, fmt.Sprintf("%s(commit:%t) [%s]", req.Method(), et.Commit, h.Key)) } else { h := req.Header() str = append(str, fmt.Sprintf("%s [%s,%s)", req.Method(), h.Key, h.EndKey)) diff --git a/pkg/roachpb/data.go b/pkg/roachpb/data.go index 14fb79c1eb28..42021c304e6f 100644 --- a/pkg/roachpb/data.go +++ b/pkg/roachpb/data.go @@ -877,6 +877,9 @@ func (t *Transaction) Restart( t.WriteTooOld = false t.RetryOnPush = false t.Sequence = 0 + // Reset Writing. Since we're using a new epoch, we don't care about the abort + // cache. + t.Writing = false } // BumpEpoch increments the transaction's epoch, allowing for an in-place diff --git a/pkg/roachpb/data.pb.go b/pkg/roachpb/data.pb.go index f101f99ae1fe..ce3a4d203eb6 100644 --- a/pkg/roachpb/data.pb.go +++ b/pkg/roachpb/data.pb.go @@ -438,8 +438,12 @@ type Transaction struct { // The list of observed timestamps is kept sorted by NodeID. Use // Transaction.UpdateObservedTimestamp to maintain the sorted order. ObservedTimestamps []ObservedTimestamp `protobuf:"bytes,8,rep,name=observed_timestamps,json=observedTimestamps" json:"observed_timestamps"` - // Writing is true if the transaction has previously executed a successful - // write request, i.e. a request that may have left intents (across retries). + // Writing is true if the transaction has previously sent a Begin transaction + // (i.e. if it ever attempted to perform a write, so if it ever attempted to + // leave intents (across retries)). The flag will be set even if the BeginTxn + // batch failed. + // When set, the AbortCache must be checked by reads so that they don't miss + // to see the txn's previous writes. Writing bool `protobuf:"varint,9,opt,name=writing,proto3" json:"writing,omitempty"` // If this is true, the transaction must retry. Relevant only for // SNAPSHOT transactions: a SERIALIZABLE transaction would have to diff --git a/pkg/roachpb/data.proto b/pkg/roachpb/data.proto index e2d60c94f297..5cb31c256947 100644 --- a/pkg/roachpb/data.proto +++ b/pkg/roachpb/data.proto @@ -335,8 +335,12 @@ message Transaction { // The list of observed timestamps is kept sorted by NodeID. Use // Transaction.UpdateObservedTimestamp to maintain the sorted order. repeated ObservedTimestamp observed_timestamps = 8 [(gogoproto.nullable) = false]; - // Writing is true if the transaction has previously executed a successful - // write request, i.e. a request that may have left intents (across retries). + // Writing is true if the transaction has previously sent a Begin transaction + // (i.e. if it ever attempted to perform a write, so if it ever attempted to + // leave intents (across retries)). The flag will be set even if the BeginTxn + // batch failed. + // When set, the AbortCache must be checked by reads so that they don't miss + // to see the txn's previous writes. bool writing = 9; // If this is true, the transaction must retry. Relevant only for // SNAPSHOT transactions: a SERIALIZABLE transaction would have to diff --git a/pkg/roachpb/errors.go b/pkg/roachpb/errors.go index de86c087b96c..d435660c5270 100644 --- a/pkg/roachpb/errors.go +++ b/pkg/roachpb/errors.go @@ -628,6 +628,16 @@ func (*TxnPrevAttemptError) message(_ *Error) string { var _ ErrorDetailInterface = &TxnPrevAttemptError{} +func (e *TxnAlreadyEncounteredErrorError) Error() string { + return e.message(nil) +} + +func (*TxnAlreadyEncounteredErrorError) message(_ *Error) string { + return "txn already encountered an error; cannot be used anymore" +} + +var _ ErrorDetailInterface = &TxnAlreadyEncounteredErrorError{} + func (e *IntegerOverflowError) Error() string { return e.message(nil) } diff --git a/pkg/roachpb/errors.pb.go b/pkg/roachpb/errors.pb.go index 1d14641d70f1..0dd7e57fa1a9 100644 --- a/pkg/roachpb/errors.pb.go +++ b/pkg/roachpb/errors.pb.go @@ -526,6 +526,19 @@ func (m *TxnPrevAttemptError) String() string { return proto.CompactT func (*TxnPrevAttemptError) ProtoMessage() {} func (*TxnPrevAttemptError) Descriptor() ([]byte, []int) { return fileDescriptorErrors, []int{24} } +// TxnAlreadyEncounteredErrorError indicates that an operation tried to use a +// transaction that already received an error from a previous request. Once that +// happens, client.Txn rejects future requests. +type TxnAlreadyEncounteredErrorError struct { +} + +func (m *TxnAlreadyEncounteredErrorError) Reset() { *m = TxnAlreadyEncounteredErrorError{} } +func (m *TxnAlreadyEncounteredErrorError) String() string { return proto.CompactTextString(m) } +func (*TxnAlreadyEncounteredErrorError) ProtoMessage() {} +func (*TxnAlreadyEncounteredErrorError) Descriptor() ([]byte, []int) { + return fileDescriptorErrors, []int{25} +} + // An IntegerOverflowError indicates that an operation was aborted because // it would have caused an integeter overflow. type IntegerOverflowError struct { @@ -537,7 +550,7 @@ type IntegerOverflowError struct { func (m *IntegerOverflowError) Reset() { *m = IntegerOverflowError{} } func (m *IntegerOverflowError) String() string { return proto.CompactTextString(m) } func (*IntegerOverflowError) ProtoMessage() {} -func (*IntegerOverflowError) Descriptor() ([]byte, []int) { return fileDescriptorErrors, []int{25} } +func (*IntegerOverflowError) Descriptor() ([]byte, []int) { return fileDescriptorErrors, []int{26} } // A MixedSuccessError indicates that some portion of the batch // request may have succeeded, but the batch as a whole failed with @@ -549,7 +562,7 @@ type MixedSuccessError struct { func (m *MixedSuccessError) Reset() { *m = MixedSuccessError{} } func (m *MixedSuccessError) String() string { return proto.CompactTextString(m) } func (*MixedSuccessError) ProtoMessage() {} -func (*MixedSuccessError) Descriptor() ([]byte, []int) { return fileDescriptorErrors, []int{26} } +func (*MixedSuccessError) Descriptor() ([]byte, []int) { return fileDescriptorErrors, []int{27} } // A BatchTimestampBeforeGCError indicates that a request's timestamp was // before the GC threshold. @@ -562,7 +575,7 @@ func (m *BatchTimestampBeforeGCError) Reset() { *m = BatchTimestampBefor func (m *BatchTimestampBeforeGCError) String() string { return proto.CompactTextString(m) } func (*BatchTimestampBeforeGCError) ProtoMessage() {} func (*BatchTimestampBeforeGCError) Descriptor() ([]byte, []int) { - return fileDescriptorErrors, []int{27} + return fileDescriptorErrors, []int{28} } // ErrorDetail is a union type containing all available errors. @@ -593,18 +606,19 @@ type ErrorDetail struct { // The following three are ErrorDetails (and proto messages) because they // needs to be communicated from the TxnCoordSender and Txn to the upper // layers through the Sender interface. - HandledRetryableTxnError *HandledRetryableTxnError `protobuf:"bytes,28,opt,name=handled_retryable_txn_error,json=handledRetryableTxnError" json:"handled_retryable_txn_error,omitempty"` - TxnAbortedAsyncErr *TxnPrevAttemptError `protobuf:"bytes,30,opt,name=txn_aborted_async_err,json=txnAbortedAsyncErr" json:"txn_aborted_async_err,omitempty"` - IntegerOverflow *IntegerOverflowError `protobuf:"bytes,31,opt,name=integer_overflow,json=integerOverflow" json:"integer_overflow,omitempty"` - UnsupportedRequest *UnsupportedRequestError `protobuf:"bytes,32,opt,name=unsupported_request,json=unsupportedRequest" json:"unsupported_request,omitempty"` - MixedSuccess *MixedSuccessError `protobuf:"bytes,33,opt,name=mixed_success,json=mixedSuccess" json:"mixed_success,omitempty"` - TimestampBefore *BatchTimestampBeforeGCError `protobuf:"bytes,34,opt,name=timestamp_before,json=timestampBefore" json:"timestamp_before,omitempty"` + HandledRetryableTxnError *HandledRetryableTxnError `protobuf:"bytes,28,opt,name=handled_retryable_txn_error,json=handledRetryableTxnError" json:"handled_retryable_txn_error,omitempty"` + TxnAbortedAsyncErr *TxnPrevAttemptError `protobuf:"bytes,30,opt,name=txn_aborted_async_err,json=txnAbortedAsyncErr" json:"txn_aborted_async_err,omitempty"` + IntegerOverflow *IntegerOverflowError `protobuf:"bytes,31,opt,name=integer_overflow,json=integerOverflow" json:"integer_overflow,omitempty"` + UnsupportedRequest *UnsupportedRequestError `protobuf:"bytes,32,opt,name=unsupported_request,json=unsupportedRequest" json:"unsupported_request,omitempty"` + MixedSuccess *MixedSuccessError `protobuf:"bytes,33,opt,name=mixed_success,json=mixedSuccess" json:"mixed_success,omitempty"` + TimestampBefore *BatchTimestampBeforeGCError `protobuf:"bytes,34,opt,name=timestamp_before,json=timestampBefore" json:"timestamp_before,omitempty"` + TxnAlreadyEncounteredError *TxnAlreadyEncounteredErrorError `protobuf:"bytes,35,opt,name=txn_already_encountered_error,json=txnAlreadyEncounteredError" json:"txn_already_encountered_error,omitempty"` } func (m *ErrorDetail) Reset() { *m = ErrorDetail{} } func (m *ErrorDetail) String() string { return proto.CompactTextString(m) } func (*ErrorDetail) ProtoMessage() {} -func (*ErrorDetail) Descriptor() ([]byte, []int) { return fileDescriptorErrors, []int{28} } +func (*ErrorDetail) Descriptor() ([]byte, []int) { return fileDescriptorErrors, []int{29} } // ErrPosition describes the position of an error in a Batch. A simple nullable // primitive field would break compatibility with proto3, where primitive fields @@ -616,7 +630,7 @@ type ErrPosition struct { func (m *ErrPosition) Reset() { *m = ErrPosition{} } func (m *ErrPosition) String() string { return proto.CompactTextString(m) } func (*ErrPosition) ProtoMessage() {} -func (*ErrPosition) Descriptor() ([]byte, []int) { return fileDescriptorErrors, []int{29} } +func (*ErrPosition) Descriptor() ([]byte, []int) { return fileDescriptorErrors, []int{30} } // Error is a generic representation including a string message // and information about retryability. @@ -646,7 +660,7 @@ type Error struct { func (m *Error) Reset() { *m = Error{} } func (*Error) ProtoMessage() {} -func (*Error) Descriptor() ([]byte, []int) { return fileDescriptorErrors, []int{30} } +func (*Error) Descriptor() ([]byte, []int) { return fileDescriptorErrors, []int{31} } func init() { proto.RegisterType((*NotLeaseHolderError)(nil), "cockroach.roachpb.NotLeaseHolderError") @@ -674,6 +688,7 @@ func init() { proto.RegisterType((*UnhandledRetryableError)(nil), "cockroach.roachpb.UnhandledRetryableError") proto.RegisterType((*HandledRetryableTxnError)(nil), "cockroach.roachpb.HandledRetryableTxnError") proto.RegisterType((*TxnPrevAttemptError)(nil), "cockroach.roachpb.TxnPrevAttemptError") + proto.RegisterType((*TxnAlreadyEncounteredErrorError)(nil), "cockroach.roachpb.TxnAlreadyEncounteredErrorError") proto.RegisterType((*IntegerOverflowError)(nil), "cockroach.roachpb.IntegerOverflowError") proto.RegisterType((*MixedSuccessError)(nil), "cockroach.roachpb.MixedSuccessError") proto.RegisterType((*BatchTimestampBeforeGCError)(nil), "cockroach.roachpb.BatchTimestampBeforeGCError") @@ -1303,6 +1318,27 @@ func (this *TxnPrevAttemptError) Equal(that interface{}) bool { } return true } +func (this *TxnAlreadyEncounteredErrorError) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + that1, ok := that.(*TxnAlreadyEncounteredErrorError) + if !ok { + that2, ok := that.(TxnAlreadyEncounteredErrorError) + if ok { + that1 = &that2 + } else { + return false + } + } + if that1 == nil { + return this == nil + } else if this == nil { + return false + } + return true +} func (this *IntegerOverflowError) Equal(that interface{}) bool { if that == nil { return this == nil @@ -1484,6 +1520,9 @@ func (this *ErrorDetail) Equal(that interface{}) bool { if !this.TimestampBefore.Equal(that1.TimestampBefore) { return false } + if !this.TxnAlreadyEncounteredError.Equal(that1.TxnAlreadyEncounteredError) { + return false + } return true } func (this *ErrPosition) Equal(that interface{}) bool { @@ -2250,6 +2289,24 @@ func (m *TxnPrevAttemptError) MarshalTo(dAtA []byte) (int, error) { return i, nil } +func (m *TxnAlreadyEncounteredErrorError) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalTo(dAtA) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *TxnAlreadyEncounteredErrorError) MarshalTo(dAtA []byte) (int, error) { + var i int + _ = i + var l int + _ = l + return i, nil +} + func (m *IntegerOverflowError) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) @@ -2651,6 +2708,18 @@ func (m *ErrorDetail) MarshalTo(dAtA []byte) (int, error) { } i += n48 } + if m.TxnAlreadyEncounteredError != nil { + dAtA[i] = 0x9a + i++ + dAtA[i] = 0x2 + i++ + i = encodeVarintErrors(dAtA, i, uint64(m.TxnAlreadyEncounteredError.Size())) + n49, err := m.TxnAlreadyEncounteredError.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n49 + } return i, nil } @@ -2701,11 +2770,11 @@ func (m *Error) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x22 i++ i = encodeVarintErrors(dAtA, i, uint64(m.UnexposedTxn.Size())) - n49, err := m.UnexposedTxn.MarshalTo(dAtA[i:]) + n50, err := m.UnexposedTxn.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n49 + i += n50 } dAtA[i] = 0x28 i++ @@ -2714,30 +2783,30 @@ func (m *Error) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x32 i++ i = encodeVarintErrors(dAtA, i, uint64(m.Detail.Size())) - n50, err := m.Detail.MarshalTo(dAtA[i:]) + n51, err := m.Detail.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n50 + i += n51 } if m.Index != nil { dAtA[i] = 0x3a i++ i = encodeVarintErrors(dAtA, i, uint64(m.Index.Size())) - n51, err := m.Index.MarshalTo(dAtA[i:]) + n52, err := m.Index.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n51 + i += n52 } dAtA[i] = 0x42 i++ i = encodeVarintErrors(dAtA, i, uint64(m.Now.Size())) - n52, err := m.Now.MarshalTo(dAtA[i:]) + n53, err := m.Now.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n52 + i += n53 return i, nil } @@ -2991,6 +3060,12 @@ func (m *TxnPrevAttemptError) Size() (n int) { return n } +func (m *TxnAlreadyEncounteredErrorError) Size() (n int) { + var l int + _ = l + return n +} + func (m *IntegerOverflowError) Size() (n int) { var l int _ = l @@ -3134,6 +3209,10 @@ func (m *ErrorDetail) Size() (n int) { l = m.TimestampBefore.Size() n += 2 + l + sovErrors(uint64(l)) } + if m.TxnAlreadyEncounteredError != nil { + l = m.TxnAlreadyEncounteredError.Size() + n += 2 + l + sovErrors(uint64(l)) + } return n } @@ -3263,6 +3342,9 @@ func (this *ErrorDetail) GetValue() interface{} { if this.TimestampBefore != nil { return this.TimestampBefore } + if this.TxnAlreadyEncounteredError != nil { + return this.TxnAlreadyEncounteredError + } return nil } @@ -3322,6 +3404,8 @@ func (this *ErrorDetail) SetValue(value interface{}) bool { this.MixedSuccess = vt case *BatchTimestampBeforeGCError: this.TimestampBefore = vt + case *TxnAlreadyEncounteredErrorError: + this.TxnAlreadyEncounteredError = vt default: return false } @@ -5599,6 +5683,56 @@ func (m *TxnPrevAttemptError) Unmarshal(dAtA []byte) error { } return nil } +func (m *TxnAlreadyEncounteredErrorError) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowErrors + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: TxnAlreadyEncounteredErrorError: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: TxnAlreadyEncounteredErrorError: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + default: + iNdEx = preIndex + skippy, err := skipErrors(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthErrors + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} func (m *IntegerOverflowError) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 @@ -6831,6 +6965,39 @@ func (m *ErrorDetail) Unmarshal(dAtA []byte) error { return err } iNdEx = postIndex + case 35: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field TxnAlreadyEncounteredError", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowErrors + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthErrors + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.TxnAlreadyEncounteredError == nil { + m.TxnAlreadyEncounteredError = &TxnAlreadyEncounteredErrorError{} + } + if err := m.TxnAlreadyEncounteredError.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipErrors(dAtA[iNdEx:]) @@ -7275,150 +7442,153 @@ var ( func init() { proto.RegisterFile("roachpb/errors.proto", fileDescriptorErrors) } var fileDescriptorErrors = []byte{ - // 2315 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x9c, 0x98, 0xcf, 0x53, 0x1b, 0xc9, - 0x15, 0xc7, 0x19, 0x24, 0x10, 0x3c, 0x21, 0x24, 0xda, 0x98, 0x1d, 0xe3, 0xb5, 0x60, 0xd9, 0xfc, - 0xb0, 0x9d, 0x0a, 0x6c, 0x79, 0xe3, 0x4d, 0xad, 0x93, 0x1c, 0x04, 0x12, 0xac, 0x0c, 0x48, 0xa4, - 0x11, 0xeb, 0x65, 0x37, 0x55, 0x53, 0xc3, 0x4c, 0x23, 0x66, 0x2d, 0xcd, 0x28, 0x3d, 0x3d, 0x20, - 0x2e, 0x39, 0xe7, 0x98, 0xdc, 0x72, 0x48, 0x55, 0x5c, 0xc9, 0x29, 0x95, 0x4b, 0x0e, 0xa9, 0x1c, - 0x73, 0xf6, 0x31, 0xc7, 0x54, 0xaa, 0x42, 0x25, 0xe4, 0x92, 0xaa, 0xfc, 0x07, 0x3e, 0xa5, 0xfa, - 0xc7, 0x68, 0x46, 0x68, 0x06, 0x2b, 0xbe, 0x49, 0xaf, 0x5f, 0x7f, 0xfb, 0xf5, 0x8f, 0xf7, 0xfa, - 0xd3, 0x03, 0x8b, 0xd4, 0x33, 0xad, 0xb3, 0xde, 0xc9, 0x06, 0xa1, 0xd4, 0xa3, 0xfe, 0x7a, 0x8f, - 0x7a, 0xcc, 0x43, 0x0b, 0x96, 0x67, 0xbd, 0x14, 0x2d, 0xeb, 0xaa, 0x7d, 0x79, 0x29, 0x74, 0xec, - 0x12, 0x66, 0xda, 0x26, 0x33, 0xa5, 0xeb, 0x32, 0x0a, 0xed, 0x31, 0x9b, 0x1e, 0x30, 0xa7, 0xb3, - 0x71, 0xd6, 0xb1, 0x36, 0x98, 0xd3, 0x25, 0x3e, 0x33, 0xbb, 0x3d, 0xd5, 0xb2, 0xd8, 0xf6, 0xda, - 0x9e, 0xf8, 0xb9, 0xc1, 0x7f, 0x49, 0xeb, 0xda, 0x9f, 0x27, 0xe1, 0x4e, 0xc3, 0x63, 0x7b, 0xc4, - 0xf4, 0xc9, 0x67, 0x5e, 0xc7, 0x26, 0xb4, 0xc6, 0xa3, 0x41, 0x55, 0xc8, 0x51, 0xd2, 0xeb, 0x38, - 0x96, 0xa9, 0x6b, 0xab, 0xda, 0xc3, 0xfc, 0x93, 0x6f, 0xac, 0x8f, 0x04, 0xb6, 0x8e, 0xa5, 0x47, - 0x95, 0xf8, 0x16, 0x75, 0x7a, 0xcc, 0xa3, 0x9b, 0xd9, 0xd7, 0x57, 0x2b, 0x13, 0x38, 0xec, 0x8a, - 0x76, 0x60, 0xae, 0xc3, 0x95, 0x8d, 0x33, 0x21, 0xad, 0x4f, 0x8e, 0x2f, 0x85, 0xf3, 0x9d, 0x28, - 0x26, 0xf4, 0x14, 0x66, 0xa8, 0xe9, 0xb6, 0x89, 0xe1, 0xd8, 0x7a, 0x66, 0x55, 0x7b, 0x98, 0xd9, - 0x5c, 0xe6, 0x23, 0x5d, 0x5f, 0xad, 0xe4, 0x30, 0xb7, 0xd7, 0xab, 0x6f, 0xa2, 0x9f, 0x38, 0x27, - 0x7c, 0xeb, 0x36, 0x5a, 0x87, 0x29, 0xa1, 0xa2, 0x67, 0xc5, 0xc0, 0x7a, 0xc2, 0xc0, 0x62, 0xe6, - 0x58, 0xba, 0xa1, 0x0f, 0x01, 0xac, 0xc0, 0x67, 0x5e, 0xd7, 0xe8, 0xfa, 0x6d, 0x7d, 0x6a, 0x55, - 0x7b, 0x38, 0xab, 0xa6, 0x34, 0x2b, 0xed, 0xfb, 0x7e, 0xfb, 0x59, 0xf6, 0x3f, 0xaf, 0x56, 0xb4, - 0xb5, 0xf7, 0x61, 0xb1, 0xe1, 0xd9, 0xe4, 0xc8, 0x35, 0xcf, 0x4d, 0xa7, 0x63, 0x9e, 0x74, 0x88, - 0x58, 0x38, 0xd5, 0xba, 0x02, 0xef, 0x1d, 0xb9, 0x7e, 0xd0, 0xeb, 0x79, 0x94, 0x11, 0x1b, 0x93, - 0x9f, 0x06, 0xc4, 0x67, 0x71, 0x87, 0x1f, 0x03, 0x12, 0xd1, 0x36, 0x3c, 0xb6, 0xed, 0x05, 0xae, - 0x2d, 0x57, 0x3d, 0x3e, 0x4d, 0x6d, 0xec, 0x69, 0x2a, 0xc9, 0xdf, 0x4c, 0xc2, 0x5d, 0xd1, 0xb4, - 0x4b, 0x2e, 0xf7, 0x1d, 0xbf, 0x6b, 0x32, 0xeb, 0x4c, 0xca, 0x7e, 0x0c, 0x0b, 0x54, 0x86, 0x60, - 0xf8, 0xcc, 0xa4, 0xcc, 0x78, 0x49, 0x2e, 0x85, 0xfe, 0xdc, 0x66, 0xee, 0xcd, 0xd5, 0x4a, 0x66, - 0x97, 0x5c, 0xe2, 0xa2, 0xf2, 0x38, 0xe4, 0x0e, 0xbb, 0xe4, 0x12, 0x6d, 0x40, 0x68, 0x32, 0x88, - 0x6b, 0x8b, 0x2e, 0x93, 0xc3, 0x5d, 0x0a, 0xaa, 0xbd, 0xe6, 0xda, 0xbc, 0xc3, 0x3e, 0x94, 0xba, - 0x6a, 0x58, 0x62, 0x1b, 0x22, 0x36, 0xb1, 0x57, 0xf9, 0x27, 0x6b, 0x49, 0x1b, 0xce, 0xdb, 0x63, - 0xdb, 0x5d, 0x8c, 0xfa, 0x8a, 0x26, 0xb4, 0x0b, 0x45, 0x3f, 0x68, 0xb7, 0x89, 0xcf, 0x06, 0x6a, - 0xd9, 0xb1, 0xd5, 0xe6, 0x07, 0x5d, 0x45, 0x8b, 0x5a, 0xa1, 0xff, 0x4e, 0xc2, 0x1a, 0x26, 0xa6, - 0xfd, 0xc2, 0x61, 0x67, 0x8e, 0x7b, 0xe4, 0x5a, 0x84, 0x32, 0xd3, 0x71, 0xd9, 0x65, 0xdd, 0x65, - 0x84, 0x9e, 0x9b, 0x1d, 0xb9, 0x5c, 0xcf, 0x61, 0x9e, 0x12, 0xd3, 0x36, 0x06, 0x19, 0xa4, 0x52, - 0xe0, 0x41, 0x6c, 0x60, 0x9e, 0x66, 0xeb, 0x67, 0x1d, 0x6b, 0xbd, 0x15, 0x3a, 0xa9, 0x83, 0x52, - 0xe0, 0x5d, 0x07, 0x46, 0x84, 0x01, 0x91, 0xbe, 0xe3, 0x33, 0xc7, 0x6d, 0xc7, 0xf4, 0x26, 0xc7, - 0xd7, 0x5b, 0x08, 0xbb, 0x47, 0x9a, 0x9b, 0x50, 0xe8, 0x9a, 0xfd, 0x98, 0x5c, 0x66, 0x0c, 0x39, - 0x3c, 0xd7, 0x35, 0xfb, 0x91, 0xc6, 0x57, 0x70, 0xc7, 0x3b, 0xf1, 0x09, 0x3d, 0x27, 0xb1, 0x79, - 0xfa, 0x7a, 0x76, 0x35, 0x93, 0x92, 0xa0, 0x4d, 0xe5, 0x7d, 0x33, 0x3e, 0xe4, 0xdd, 0x6c, 0xf0, - 0xa3, 0x1c, 0x68, 0x51, 0xd3, 0xf5, 0x4d, 0x8b, 0x39, 0x9e, 0x5b, 0x39, 0x11, 0xa9, 0x10, 0xcf, - 0x01, 0x13, 0x16, 0x63, 0x0e, 0x07, 0x81, 0xaf, 0x8e, 0xeb, 0x16, 0x40, 0x2f, 0xf0, 0xcf, 0x08, - 0x31, 0x58, 0xdf, 0x55, 0x6b, 0x5f, 0x4e, 0x08, 0x29, 0xd6, 0x39, 0xcc, 0x52, 0xd9, 0xaf, 0xd5, - 0x77, 0xd5, 0x10, 0xa7, 0x70, 0x37, 0xe6, 0x85, 0x09, 0xa3, 0x97, 0x72, 0x8c, 0x1d, 0x98, 0xa6, - 0xc4, 0xf4, 0x3d, 0xa9, 0x3f, 0xff, 0xe4, 0xd1, 0xed, 0xfa, 0xa2, 0x27, 0x16, 0x1d, 0xd4, 0x50, - 0xaa, 0xbb, 0x1a, 0xa7, 0x0c, 0x4b, 0x43, 0xde, 0xbd, 0x8e, 0x79, 0x19, 0x9f, 0xea, 0x5f, 0xb4, - 0x21, 0x87, 0x43, 0x66, 0xb2, 0xc0, 0x97, 0x91, 0x2c, 0x41, 0x86, 0x17, 0x1b, 0x2d, 0x56, 0x6c, - 0xb8, 0x01, 0x35, 0x06, 0x11, 0x4e, 0x8a, 0x08, 0x3f, 0xba, 0x3d, 0xc2, 0x98, 0xe4, 0x7a, 0x52, - 0xa0, 0x6b, 0x9f, 0xc0, 0xb4, 0xb4, 0x23, 0x04, 0xf3, 0xb8, 0x56, 0x39, 0x6c, 0x36, 0x8c, 0xa3, - 0xc6, 0x6e, 0xa3, 0xf9, 0xa2, 0x51, 0x9a, 0x40, 0x3a, 0x2c, 0x2a, 0x5b, 0xeb, 0x8b, 0x86, 0xd1, - 0x68, 0xb6, 0x8c, 0xed, 0xe6, 0x51, 0xa3, 0x5a, 0xd2, 0xd4, 0x04, 0x8e, 0xa1, 0xf4, 0x82, 0x3a, - 0x8c, 0xf0, 0x4c, 0x71, 0x65, 0x25, 0x43, 0x9f, 0x42, 0xce, 0x11, 0x7f, 0x7d, 0x5d, 0x13, 0xe7, - 0xe6, 0x5e, 0x42, 0x88, 0xb2, 0x43, 0x78, 0x31, 0x28, 0x7f, 0x29, 0xfa, 0x3c, 0x3b, 0x33, 0x59, - 0xca, 0xac, 0xfd, 0x41, 0x53, 0xda, 0x2d, 0xcf, 0x6b, 0x76, 0x54, 0x25, 0xac, 0xc0, 0xec, 0x3b, - 0xa5, 0x5f, 0xd4, 0x0b, 0x35, 0xa0, 0x64, 0x5a, 0x2c, 0x30, 0x3b, 0xef, 0x96, 0x78, 0x45, 0xd9, - 0x79, 0x60, 0x56, 0x0b, 0xb1, 0x0c, 0xa8, 0xd9, 0xe3, 0x05, 0xdd, 0xa1, 0xc4, 0x6f, 0xf5, 0xdd, - 0xf8, 0x2e, 0x1f, 0xc3, 0xe2, 0x96, 0xe7, 0xda, 0x0e, 0xdf, 0x8f, 0x6d, 0xd3, 0xe9, 0xa8, 0xe3, - 0x8e, 0x7e, 0x00, 0x73, 0x2a, 0x92, 0x73, 0xb3, 0x13, 0x10, 0x35, 0x9f, 0xa4, 0xdb, 0xe8, 0x73, - 0xde, 0x8e, 0xf3, 0xd2, 0x5b, 0xfc, 0x51, 0xd2, 0x7f, 0xd2, 0x00, 0xc9, 0xab, 0x8a, 0x7c, 0x4d, - 0xac, 0x30, 0x91, 0x50, 0x19, 0x72, 0x5d, 0xe2, 0xfb, 0x66, 0x9b, 0x0c, 0x1d, 0xa0, 0xd0, 0x88, - 0x7e, 0x08, 0xb3, 0xaa, 0x48, 0x13, 0x5b, 0x4d, 0x3e, 0xf5, 0x12, 0x0c, 0x57, 0x70, 0xd0, 0x01, - 0x3d, 0x83, 0x99, 0xb0, 0xfa, 0xa8, 0x1a, 0xf3, 0xb6, 0xce, 0x03, 0x7f, 0x15, 0xf6, 0xf7, 0x61, - 0xf6, 0x90, 0xb8, 0xe3, 0x05, 0x3b, 0x74, 0x28, 0x2e, 0x60, 0xb1, 0xd2, 0x3d, 0x71, 0xda, 0x81, - 0x17, 0xf8, 0x98, 0xf8, 0x41, 0x87, 0x8d, 0x37, 0xe1, 0x4f, 0x21, 0x7f, 0x41, 0xcd, 0x5e, 0x8f, - 0xd8, 0x06, 0xa1, 0xf4, 0x96, 0x29, 0x0b, 0x39, 0x0c, 0xca, 0xb9, 0x46, 0xc3, 0x3d, 0x7c, 0xc0, - 0x2f, 0xd1, 0x53, 0xb6, 0x43, 0xbd, 0xa0, 0x57, 0x25, 0x1d, 0x72, 0xa3, 0x66, 0x11, 0x58, 0x52, - 0xa8, 0xb2, 0xe5, 0x51, 0x1a, 0xf4, 0xf8, 0x56, 0xcb, 0xc8, 0x3e, 0x80, 0x59, 0x01, 0x72, 0xc6, - 0xcd, 0x6c, 0x9e, 0x11, 0xe6, 0x7d, 0xbf, 0x8d, 0xd6, 0x60, 0xb6, 0x47, 0x3d, 0x8b, 0xf8, 0xbe, - 0xda, 0x8d, 0x99, 0x41, 0xdd, 0x0a, 0xcd, 0x83, 0x93, 0x84, 0xd4, 0x30, 0xf1, 0xa4, 0xf8, 0x11, - 0x80, 0x22, 0xab, 0x10, 0x10, 0xa6, 0x36, 0xcb, 0x0a, 0x10, 0x66, 0x95, 0xbf, 0x40, 0x84, 0xe8, - 0x0f, 0xdf, 0x4e, 0xf9, 0xd3, 0x8e, 0xc8, 0xe3, 0x90, 0x79, 0x74, 0x94, 0x3c, 0x7c, 0x6e, 0x4d, - 0x20, 0x0f, 0xe1, 0x2d, 0xc9, 0x43, 0xfd, 0xc4, 0x39, 0xe1, 0x3b, 0x90, 0xdc, 0xe7, 0xb4, 0x73, - 0x66, 0xba, 0x76, 0x87, 0xb3, 0x0e, 0xa3, 0x97, 0x03, 0x1c, 0x42, 0x4f, 0x20, 0xdb, 0xab, 0x51, - 0x7a, 0xcb, 0x91, 0x17, 0x7e, 0x6a, 0x1d, 0x84, 0xef, 0xda, 0x3f, 0x34, 0xd0, 0x3f, 0xbb, 0xa1, - 0x16, 0x66, 0x5a, 0x6a, 0xb9, 0xfc, 0x0a, 0xa6, 0x59, 0xdf, 0xe5, 0xe1, 0x4b, 0x4a, 0xa9, 0xf2, - 0xa6, 0xbf, 0x5f, 0xad, 0x7c, 0xdc, 0x76, 0xd8, 0x59, 0x70, 0xb2, 0x6e, 0x79, 0xdd, 0x8d, 0xc1, - 0xe0, 0xf6, 0x49, 0xf4, 0x7b, 0xa3, 0xf7, 0xb2, 0xbd, 0x21, 0xa8, 0x39, 0x08, 0x1c, 0x7b, 0xfd, - 0xe8, 0xa8, 0x5e, 0xbd, 0xbe, 0x5a, 0x99, 0x6a, 0xf5, 0xdd, 0x7a, 0x15, 0x4f, 0xb1, 0xbe, 0x5b, - 0xb7, 0xd1, 0x36, 0xe4, 0x59, 0x54, 0x6a, 0x55, 0x2e, 0x8c, 0x77, 0x25, 0xc5, 0x3b, 0xaa, 0xe5, - 0xba, 0x0f, 0x77, 0x5a, 0x7d, 0xf7, 0x80, 0x92, 0xf3, 0x0a, 0x63, 0xa4, 0xdb, 0x1b, 0x02, 0xc3, - 0x5f, 0x6a, 0xb0, 0xc8, 0x6b, 0x66, 0x9b, 0xd0, 0xe6, 0x39, 0xa1, 0xa7, 0x1d, 0xef, 0x42, 0x4e, - 0xfc, 0x1e, 0x64, 0x12, 0xb0, 0x8d, 0xdb, 0xd0, 0x23, 0x28, 0x58, 0x01, 0xa5, 0xc4, 0x65, 0xaa, - 0xc0, 0x4c, 0x8a, 0x1d, 0x94, 0x01, 0xcc, 0xa9, 0x26, 0x51, 0x4d, 0xd0, 0x77, 0xa1, 0xe8, 0xb8, - 0x16, 0x25, 0xdd, 0xc8, 0x39, 0x13, 0x73, 0x9e, 0x1f, 0x34, 0xc6, 0x8b, 0xcf, 0x3e, 0x2c, 0xec, - 0x3b, 0x7d, 0x62, 0x1f, 0x06, 0x16, 0x3f, 0xa5, 0xe1, 0xce, 0xe6, 0x54, 0xf2, 0xbc, 0x6d, 0x73, - 0x71, 0xe8, 0xa8, 0xe4, 0x7e, 0xaf, 0xc1, 0xfd, 0x4d, 0x8e, 0x7a, 0x51, 0xc9, 0x25, 0xa7, 0x1e, - 0x25, 0x3b, 0x5b, 0x83, 0xda, 0xdf, 0x7a, 0xa7, 0xda, 0x1f, 0xe1, 0x0d, 0x97, 0x38, 0xa3, 0xc4, - 0xe7, 0xef, 0x8e, 0xff, 0xa7, 0xe8, 0x47, 0xbd, 0x54, 0xac, 0xbf, 0x45, 0x90, 0x17, 0x51, 0x55, - 0x09, 0x33, 0x9d, 0x0e, 0x3a, 0x80, 0x92, 0xeb, 0x31, 0x63, 0xe8, 0x55, 0x23, 0x43, 0xfc, 0x56, - 0xc2, 0xf4, 0x13, 0x5e, 0x56, 0x78, 0xde, 0x1d, 0x32, 0xa2, 0x7d, 0x28, 0x4a, 0xe6, 0xe7, 0xba, - 0xa7, 0x3c, 0x23, 0x55, 0xc0, 0xdf, 0x4c, 0xe3, 0xdc, 0xa1, 0xcc, 0xc5, 0x05, 0x1a, 0xb7, 0xa1, - 0xcf, 0x01, 0x49, 0xb9, 0x97, 0xe4, 0xd2, 0x08, 0x99, 0x5a, 0x9d, 0xd8, 0x87, 0x69, 0x8a, 0x37, - 0x5f, 0x0c, 0xb8, 0x44, 0x6f, 0x98, 0xd1, 0xcf, 0x60, 0x55, 0x40, 0xf1, 0x85, 0x60, 0x67, 0x23, - 0x88, 0xe0, 0xd9, 0x70, 0x14, 0x3d, 0x2b, 0x3e, 0x7f, 0x9a, 0xf8, 0xbc, 0x7b, 0x1b, 0x75, 0xe3, - 0x07, 0xf4, 0x36, 0x1f, 0x0e, 0xac, 0xb1, 0x4c, 0x32, 0x4c, 0x89, 0x93, 0xe2, 0x8d, 0x96, 0x7f, - 0xf2, 0xf8, 0xf6, 0x54, 0x8c, 0xb3, 0x27, 0x46, 0x6c, 0xa4, 0x01, 0x61, 0x28, 0xc5, 0xc5, 0x39, - 0x45, 0xea, 0xd3, 0x42, 0xf9, 0xdb, 0xb7, 0x2b, 0x0f, 0xa0, 0x15, 0x17, 0xd9, 0xb0, 0x15, 0x1d, - 0xc1, 0x42, 0x5c, 0x93, 0xf2, 0x4a, 0xa6, 0xe7, 0x52, 0xf7, 0x21, 0x11, 0x53, 0x71, 0x3c, 0x2c, - 0x61, 0x46, 0x5f, 0x40, 0x7c, 0x02, 0xfc, 0x3d, 0xc7, 0x02, 0x5f, 0x9f, 0x11, 0xba, 0x8f, 0xc6, - 0x46, 0x44, 0x1c, 0x8f, 0x4d, 0xda, 0xd1, 0x36, 0xcc, 0x5d, 0x70, 0x0c, 0x33, 0x24, 0xa4, 0xe9, - 0xb3, 0x42, 0xf3, 0xc3, 0x04, 0xcd, 0x9b, 0x24, 0x88, 0xf3, 0x17, 0x91, 0x05, 0xed, 0x40, 0x41, - 0xea, 0x30, 0xcf, 0x33, 0x78, 0xfe, 0xc1, 0xed, 0x42, 0xb1, 0x1b, 0x4e, 0x09, 0x49, 0x0b, 0xcf, - 0x0c, 0xaf, 0x67, 0x50, 0xc5, 0x5a, 0xe2, 0x31, 0x90, 0x4f, 0xcd, 0x8c, 0x51, 0x28, 0xc3, 0x05, - 0x2f, 0x6e, 0xe3, 0x9b, 0x6c, 0x85, 0x74, 0x66, 0x9c, 0x0a, 0x3c, 0xd3, 0xe7, 0x52, 0x37, 0x39, - 0x09, 0xe4, 0x70, 0xd1, 0x1a, 0xb6, 0xa2, 0x3d, 0x98, 0x97, 0xa5, 0x80, 0x2a, 0x2c, 0xd3, 0x0b, - 0xa9, 0x11, 0x8e, 0xe2, 0x1b, 0x2e, 0x74, 0xe2, 0x36, 0x1e, 0xa1, 0xeb, 0xd9, 0xc4, 0x08, 0xa2, - 0x8f, 0x0a, 0xfa, 0x7c, 0x6a, 0x84, 0x49, 0x9f, 0x1f, 0x70, 0xd1, 0x1d, 0xb6, 0xa2, 0x8f, 0x20, - 0xeb, 0x13, 0xd7, 0xd6, 0x8b, 0x42, 0xe7, 0xfd, 0x04, 0x9d, 0x01, 0xa0, 0x61, 0xe1, 0x29, 0x2b, - 0xc8, 0x29, 0x33, 0xda, 0x1c, 0x81, 0x0c, 0x5b, 0x32, 0x90, 0x5e, 0xba, 0xa5, 0x82, 0x24, 0xe0, - 0x12, 0xaf, 0x20, 0xc3, 0x66, 0x7e, 0x72, 0x43, 0x7a, 0xb1, 0x06, 0xec, 0xa4, 0x2f, 0xa4, 0x9e, - 0xdc, 0x64, 0xce, 0xc2, 0x0b, 0xf4, 0xa6, 0x5d, 0x94, 0x50, 0xa5, 0x1c, 0x9e, 0x39, 0x94, 0x5e, - 0x42, 0x47, 0xb8, 0x8a, 0xbf, 0xd9, 0x63, 0xb6, 0x9b, 0x29, 0x46, 0xc5, 0x6b, 0x4e, 0x5f, 0x1a, - 0x27, 0xc5, 0x62, 0x2f, 0xbf, 0xa1, 0x14, 0x93, 0x76, 0xbe, 0xc1, 0x66, 0x48, 0xb5, 0x06, 0x15, - 0x58, 0xab, 0x2f, 0xa7, 0x6e, 0x70, 0x12, 0x00, 0xe3, 0xa2, 0x39, 0x6c, 0xe5, 0x93, 0x97, 0xe4, - 0x16, 0xdd, 0x1f, 0xf7, 0x53, 0x27, 0x3f, 0x4a, 0x7e, 0xb8, 0xe0, 0xc7, 0x6d, 0xe8, 0x6b, 0xb8, - 0xaf, 0x48, 0x4e, 0x96, 0x2c, 0x7e, 0x88, 0x78, 0xea, 0x19, 0x82, 0x62, 0xf5, 0xf7, 0x85, 0xf4, - 0x77, 0x12, 0xa4, 0xd3, 0x88, 0x0d, 0xeb, 0x67, 0x69, 0x2c, 0x77, 0x0c, 0x77, 0xb9, 0xb2, 0xaa, - 0xe5, 0x86, 0xe9, 0x5f, 0xba, 0x96, 0xc0, 0xf6, 0x72, 0xea, 0x8d, 0x9a, 0x00, 0x4e, 0x18, 0xb1, - 0x7e, 0x58, 0xc9, 0x2b, 0x5c, 0xa2, 0x46, 0x29, 0x5f, 0x69, 0x47, 0x52, 0x94, 0xe1, 0x29, 0x8c, - 0xd2, 0x57, 0x52, 0x57, 0x3a, 0x09, 0xb8, 0x70, 0xd1, 0x19, 0xb6, 0xf2, 0x2b, 0x28, 0x88, 0x3e, - 0xea, 0x19, 0xea, 0x9d, 0xa4, 0xaf, 0xa6, 0x5e, 0x41, 0x29, 0x9f, 0x00, 0x31, 0x0a, 0x46, 0x1a, - 0x50, 0x1d, 0x0a, 0x5d, 0xce, 0x58, 0x86, 0x2f, 0x21, 0x4b, 0xff, 0x20, 0xf5, 0x5b, 0xe9, 0x08, - 0x8b, 0xe1, 0xb9, 0x6e, 0xcc, 0x84, 0x8e, 0xa1, 0x34, 0x78, 0xf1, 0x1a, 0x27, 0x02, 0xad, 0xf4, - 0x35, 0xa1, 0xb6, 0x9e, 0xa0, 0x76, 0x0b, 0x89, 0xe1, 0x22, 0x1b, 0xb6, 0x3f, 0x9b, 0x79, 0xfd, - 0x6a, 0x45, 0x53, 0xcf, 0xb4, 0x7b, 0xa5, 0xe5, 0xe7, 0xd9, 0x99, 0x07, 0xa5, 0xf2, 0xda, 0x86, - 0x60, 0xa4, 0x03, 0xcf, 0x17, 0xa5, 0x11, 0x2d, 0xc3, 0x94, 0xe3, 0xda, 0xa4, 0xaf, 0x5e, 0x28, - 0x92, 0xac, 0xa4, 0x49, 0x51, 0xd5, 0x1f, 0x33, 0x30, 0x35, 0xde, 0x7b, 0xee, 0x27, 0xc3, 0xd7, - 0x3e, 0x25, 0xe2, 0x03, 0xa6, 0xe0, 0x99, 0xf9, 0xc4, 0x13, 0x3e, 0x94, 0x8c, 0xc2, 0x39, 0xfc, - 0x50, 0xc5, 0x46, 0x5a, 0xd0, 0x16, 0x14, 0x02, 0x97, 0xf4, 0x7b, 0x9e, 0x4f, 0x6c, 0x71, 0xbf, - 0x64, 0xc7, 0x21, 0x7b, 0x3c, 0x37, 0xe8, 0xc4, 0xef, 0x95, 0x0d, 0xc8, 0x7b, 0xd4, 0x69, 0x3b, - 0xae, 0xc1, 0x6b, 0xaf, 0x20, 0x92, 0xa9, 0xcd, 0x79, 0x3e, 0xe6, 0x9b, 0xab, 0x95, 0x69, 0x5e, - 0xa5, 0xeb, 0x55, 0x0c, 0xd2, 0x85, 0xff, 0x43, 0x9f, 0xc0, 0xb4, 0x2d, 0x68, 0x52, 0x31, 0x46, - 0x39, 0x0d, 0x9c, 0x25, 0x73, 0x62, 0xe5, 0x8d, 0xbe, 0x17, 0xae, 0x6b, 0xee, 0xb6, 0x6e, 0xe1, - 0x36, 0xa8, 0x15, 0x47, 0x4f, 0x21, 0xe3, 0x7a, 0x17, 0x8a, 0x10, 0xc6, 0x82, 0x60, 0xee, 0xff, - 0x6c, 0xe6, 0x57, 0xaf, 0x56, 0x26, 0xa2, 0x07, 0xf9, 0xe3, 0x5f, 0x6b, 0x37, 0x3e, 0x71, 0x0d, - 0x3e, 0x88, 0xc9, 0x6f, 0x47, 0x2d, 0x7c, 0x6c, 0x8c, 0x7c, 0x55, 0x7a, 0x0f, 0xee, 0xc8, 0x96, - 0x17, 0xb8, 0xde, 0xaa, 0x19, 0xad, 0x66, 0xd3, 0x68, 0xee, 0x55, 0x4b, 0x1a, 0x5a, 0x02, 0x24, - 0x1b, 0xaa, 0xb5, 0xbd, 0x5a, 0xab, 0x66, 0xe0, 0x4a, 0x63, 0xa7, 0x56, 0x9a, 0x8c, 0xec, 0x87, - 0x35, 0x5c, 0xaf, 0xec, 0xd5, 0xbf, 0xac, 0x6c, 0xee, 0xd5, 0x4a, 0x19, 0x74, 0x0f, 0xee, 0x4a, - 0xfb, 0x41, 0xf3, 0xf0, 0xb0, 0xbe, 0xb9, 0x57, 0x33, 0x70, 0xed, 0x60, 0xaf, 0x72, 0x5c, 0xca, - 0x2e, 0x67, 0x7f, 0xfe, 0xbb, 0xf2, 0xc4, 0xe3, 0x67, 0x80, 0x46, 0x77, 0x1e, 0xcd, 0x40, 0xb6, - 0xd1, 0x6c, 0xd4, 0x4a, 0x13, 0x28, 0x0f, 0xb9, 0xcd, 0xca, 0xd6, 0x6e, 0x73, 0x7b, 0xbb, 0xa4, - 0xa1, 0x02, 0xcc, 0xd6, 0xf7, 0xf7, 0x6b, 0xd5, 0x7a, 0xa5, 0x55, 0x2b, 0x4d, 0x6e, 0x7e, 0xf0, - 0xfa, 0x5f, 0xe5, 0x89, 0xd7, 0xd7, 0x65, 0xed, 0xaf, 0xd7, 0x65, 0xed, 0x6f, 0xd7, 0x65, 0xed, - 0x9f, 0xd7, 0x65, 0xed, 0x17, 0xff, 0x2e, 0x4f, 0x7c, 0x99, 0x53, 0xeb, 0xfa, 0xbf, 0x00, 0x00, - 0x00, 0xff, 0xff, 0x75, 0x40, 0xad, 0xe7, 0xac, 0x19, 0x00, 0x00, + // 2366 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x9c, 0x98, 0xcd, 0x6f, 0x23, 0x49, + 0xf9, 0xc7, 0xe3, 0xd8, 0x89, 0x93, 0xc7, 0x71, 0xec, 0xd4, 0x64, 0xb2, 0x3d, 0x99, 0x1d, 0x3b, + 0x9b, 0xfd, 0xfd, 0xd8, 0xd9, 0x45, 0x24, 0xab, 0x2c, 0xb3, 0x68, 0x07, 0x38, 0x38, 0xb1, 0x93, + 0xf5, 0x24, 0xb1, 0x43, 0xc5, 0xd9, 0xd9, 0xec, 0x22, 0xb5, 0x3a, 0xdd, 0x15, 0xa7, 0x77, 0xec, + 0x6e, 0x53, 0x5d, 0x9d, 0x38, 0x17, 0xce, 0x1c, 0xe1, 0x06, 0x12, 0x12, 0x23, 0x71, 0x42, 0x5c, + 0x38, 0x20, 0x8e, 0x9c, 0xe7, 0xc8, 0x11, 0x21, 0x11, 0x41, 0xb8, 0x20, 0xf1, 0x1f, 0xcc, 0x09, + 0xd5, 0x4b, 0xbb, 0xdb, 0x71, 0xb7, 0xc7, 0xcc, 0xcd, 0x7e, 0xea, 0xa9, 0x6f, 0x3d, 0xf5, 0xf2, + 0x3c, 0xf5, 0xa9, 0x86, 0x65, 0xea, 0x1a, 0xe6, 0x45, 0xef, 0x6c, 0x93, 0x50, 0xea, 0x52, 0x6f, + 0xa3, 0x47, 0x5d, 0xe6, 0xa2, 0x25, 0xd3, 0x35, 0x5f, 0x88, 0x96, 0x0d, 0xd5, 0xbe, 0xba, 0x12, + 0x38, 0x76, 0x09, 0x33, 0x2c, 0x83, 0x19, 0xd2, 0x75, 0x15, 0x05, 0xf6, 0x88, 0x4d, 0xf3, 0x99, + 0xdd, 0xd9, 0xbc, 0xe8, 0x98, 0x9b, 0xcc, 0xee, 0x12, 0x8f, 0x19, 0xdd, 0x9e, 0x6a, 0x59, 0x6e, + 0xbb, 0x6d, 0x57, 0xfc, 0xdc, 0xe4, 0xbf, 0xa4, 0x75, 0xfd, 0x4f, 0xd3, 0x70, 0xaf, 0xe1, 0xb2, + 0x03, 0x62, 0x78, 0xe4, 0x73, 0xb7, 0x63, 0x11, 0x5a, 0xe3, 0xd1, 0xa0, 0x2a, 0x64, 0x29, 0xe9, + 0x75, 0x6c, 0xd3, 0xd0, 0x52, 0x6b, 0xa9, 0xc7, 0xb9, 0xad, 0xff, 0xdb, 0x18, 0x09, 0x6c, 0x03, + 0x4b, 0x8f, 0x2a, 0xf1, 0x4c, 0x6a, 0xf7, 0x98, 0x4b, 0xb7, 0x33, 0xaf, 0x6e, 0xca, 0x53, 0x38, + 0xe8, 0x8a, 0xf6, 0x60, 0xa1, 0xc3, 0x95, 0xf5, 0x0b, 0x21, 0xad, 0x4d, 0x4f, 0x2e, 0x85, 0x73, + 0x9d, 0x30, 0x26, 0xf4, 0x04, 0xe6, 0xa8, 0xe1, 0xb4, 0x89, 0x6e, 0x5b, 0x5a, 0x7a, 0x2d, 0xf5, + 0x38, 0xbd, 0xbd, 0xca, 0x47, 0xba, 0xbd, 0x29, 0x67, 0x31, 0xb7, 0xd7, 0xab, 0xaf, 0xc3, 0x9f, + 0x38, 0x2b, 0x7c, 0xeb, 0x16, 0xda, 0x80, 0x19, 0xa1, 0xa2, 0x65, 0xc4, 0xc0, 0x5a, 0xcc, 0xc0, + 0x62, 0xe6, 0x58, 0xba, 0xa1, 0xf7, 0x01, 0x4c, 0xdf, 0x63, 0x6e, 0x57, 0xef, 0x7a, 0x6d, 0x6d, + 0x66, 0x2d, 0xf5, 0x78, 0x5e, 0x4d, 0x69, 0x5e, 0xda, 0x0f, 0xbd, 0xf6, 0xd3, 0xcc, 0xbf, 0x5f, + 0x96, 0x53, 0xeb, 0xef, 0xc2, 0x72, 0xc3, 0xb5, 0xc8, 0x89, 0x63, 0x5c, 0x1a, 0x76, 0xc7, 0x38, + 0xeb, 0x10, 0xb1, 0x70, 0xaa, 0xb5, 0x0c, 0xef, 0x9c, 0x38, 0x9e, 0xdf, 0xeb, 0xb9, 0x94, 0x11, + 0x0b, 0x93, 0x9f, 0xf8, 0xc4, 0x63, 0x51, 0x87, 0x1f, 0x01, 0x12, 0xd1, 0x36, 0x5c, 0xb6, 0xeb, + 0xfa, 0x8e, 0x25, 0x57, 0x3d, 0x3a, 0xcd, 0xd4, 0xc4, 0xd3, 0x54, 0x92, 0xbf, 0x99, 0x86, 0xfb, + 0xa2, 0x69, 0x9f, 0x5c, 0x1f, 0xda, 0x5e, 0xd7, 0x60, 0xe6, 0x85, 0x94, 0xfd, 0x04, 0x96, 0xa8, + 0x0c, 0x41, 0xf7, 0x98, 0x41, 0x99, 0xfe, 0x82, 0x5c, 0x0b, 0xfd, 0x85, 0xed, 0xec, 0xeb, 0x9b, + 0x72, 0x7a, 0x9f, 0x5c, 0xe3, 0x82, 0xf2, 0x38, 0xe6, 0x0e, 0xfb, 0xe4, 0x1a, 0x6d, 0x42, 0x60, + 0xd2, 0x89, 0x63, 0x89, 0x2e, 0xd3, 0xc3, 0x5d, 0xf2, 0xaa, 0xbd, 0xe6, 0x58, 0xbc, 0xc3, 0x21, + 0x14, 0xbb, 0x6a, 0x58, 0x62, 0xe9, 0x22, 0x36, 0xb1, 0x57, 0xb9, 0xad, 0xf5, 0xb8, 0x0d, 0xe7, + 0xed, 0x91, 0xed, 0x2e, 0x84, 0x7d, 0x45, 0x13, 0xda, 0x87, 0x82, 0xe7, 0xb7, 0xdb, 0xc4, 0x63, + 0x03, 0xb5, 0xcc, 0xc4, 0x6a, 0x8b, 0x83, 0xae, 0xa2, 0x45, 0xad, 0xd0, 0x7f, 0xa6, 0x61, 0x1d, + 0x13, 0xc3, 0x7a, 0x6e, 0xb3, 0x0b, 0xdb, 0x39, 0x71, 0x4c, 0x42, 0x99, 0x61, 0x3b, 0xec, 0xba, + 0xee, 0x30, 0x42, 0x2f, 0x8d, 0x8e, 0x5c, 0xae, 0x67, 0xb0, 0x48, 0x89, 0x61, 0xe9, 0x83, 0x0c, + 0x52, 0x29, 0xf0, 0x28, 0x32, 0x30, 0x4f, 0xb3, 0x8d, 0x8b, 0x8e, 0xb9, 0xd1, 0x0a, 0x9c, 0xd4, + 0x41, 0xc9, 0xf3, 0xae, 0x03, 0x23, 0xc2, 0x80, 0x48, 0xdf, 0xf6, 0x98, 0xed, 0xb4, 0x23, 0x7a, + 0xd3, 0x93, 0xeb, 0x2d, 0x05, 0xdd, 0x43, 0xcd, 0x6d, 0xc8, 0x77, 0x8d, 0x7e, 0x44, 0x2e, 0x3d, + 0x81, 0x1c, 0x5e, 0xe8, 0x1a, 0xfd, 0x50, 0xe3, 0x6b, 0xb8, 0xe7, 0x9e, 0x79, 0x84, 0x5e, 0x92, + 0xc8, 0x3c, 0x3d, 0x2d, 0xb3, 0x96, 0x4e, 0x48, 0xd0, 0xa6, 0xf2, 0xbe, 0x1b, 0x1f, 0x72, 0xef, + 0x36, 0x78, 0x61, 0x0e, 0xb4, 0xa8, 0xe1, 0x78, 0x86, 0xc9, 0x6c, 0xd7, 0xa9, 0x9c, 0x89, 0x54, + 0x88, 0xe6, 0x80, 0x01, 0xcb, 0x11, 0x87, 0x23, 0xdf, 0x53, 0xc7, 0x75, 0x07, 0xa0, 0xe7, 0x7b, + 0x17, 0x84, 0xe8, 0xac, 0xef, 0xa8, 0xb5, 0x2f, 0xc5, 0x84, 0x14, 0xe9, 0x1c, 0x64, 0xa9, 0xec, + 0xd7, 0xea, 0x3b, 0x6a, 0x88, 0x73, 0xb8, 0x1f, 0xf1, 0xc2, 0x84, 0xd1, 0x6b, 0x39, 0xc6, 0x1e, + 0xcc, 0x52, 0x62, 0x78, 0xae, 0xd4, 0x5f, 0xdc, 0xfa, 0x70, 0xbc, 0xbe, 0xe8, 0x89, 0x45, 0x07, + 0x35, 0x94, 0xea, 0xae, 0xc6, 0x29, 0xc1, 0xca, 0x90, 0x77, 0xaf, 0x63, 0x5c, 0x47, 0xa7, 0xfa, + 0xe7, 0xd4, 0x90, 0xc3, 0x31, 0x33, 0x98, 0xef, 0xc9, 0x48, 0x56, 0x20, 0xcd, 0x8b, 0x4d, 0x2a, + 0x52, 0x6c, 0xb8, 0x01, 0x35, 0x06, 0x11, 0x4e, 0x8b, 0x08, 0x3f, 0x1e, 0x1f, 0x61, 0x44, 0x72, + 0x23, 0x2e, 0xd0, 0xf5, 0x4f, 0x61, 0x56, 0xda, 0x11, 0x82, 0x45, 0x5c, 0xab, 0x1c, 0x37, 0x1b, + 0xfa, 0x49, 0x63, 0xbf, 0xd1, 0x7c, 0xde, 0x28, 0x4e, 0x21, 0x0d, 0x96, 0x95, 0xad, 0xf5, 0x65, + 0x43, 0x6f, 0x34, 0x5b, 0xfa, 0x6e, 0xf3, 0xa4, 0x51, 0x2d, 0xa6, 0xd4, 0x04, 0x4e, 0xa1, 0xf8, + 0x9c, 0xda, 0x8c, 0xf0, 0x4c, 0x71, 0x64, 0x25, 0x43, 0x9f, 0x41, 0xd6, 0x16, 0x7f, 0x3d, 0x2d, + 0x25, 0xce, 0xcd, 0x83, 0x98, 0x10, 0x65, 0x87, 0xe0, 0x62, 0x50, 0xfe, 0x52, 0xf4, 0x59, 0x66, + 0x6e, 0xba, 0x98, 0x5e, 0xff, 0x7d, 0x4a, 0x69, 0xb7, 0x5c, 0xb7, 0xd9, 0x51, 0x95, 0xb0, 0x02, + 0xf3, 0x6f, 0x95, 0x7e, 0x61, 0x2f, 0xd4, 0x80, 0xa2, 0x61, 0x32, 0xdf, 0xe8, 0xbc, 0x5d, 0xe2, + 0x15, 0x64, 0xe7, 0x81, 0x59, 0x2d, 0xc4, 0x2a, 0xa0, 0x66, 0x8f, 0x17, 0x74, 0x9b, 0x12, 0xaf, + 0xd5, 0x77, 0xa2, 0xbb, 0x7c, 0x0a, 0xcb, 0x3b, 0xae, 0x63, 0xd9, 0x7c, 0x3f, 0x76, 0x0d, 0xbb, + 0xa3, 0x8e, 0x3b, 0xfa, 0x3e, 0x2c, 0xa8, 0x48, 0x2e, 0x8d, 0x8e, 0x4f, 0xd4, 0x7c, 0xe2, 0x6e, + 0xa3, 0x2f, 0x78, 0x3b, 0xce, 0x49, 0x6f, 0xf1, 0x47, 0x49, 0xff, 0x31, 0x05, 0x48, 0x5e, 0x55, + 0xe4, 0x1b, 0x62, 0x06, 0x89, 0x84, 0x4a, 0x90, 0xed, 0x12, 0xcf, 0x33, 0xda, 0x64, 0xe8, 0x00, + 0x05, 0x46, 0xf4, 0x03, 0x98, 0x57, 0x45, 0x9a, 0x58, 0x6a, 0xf2, 0x89, 0x97, 0x60, 0xb0, 0x82, + 0x83, 0x0e, 0xe8, 0x29, 0xcc, 0x05, 0xd5, 0x47, 0xd5, 0x98, 0x37, 0x75, 0x1e, 0xf8, 0xab, 0xb0, + 0xbf, 0x07, 0xf3, 0xc7, 0xc4, 0x99, 0x2c, 0xd8, 0xa1, 0x43, 0x71, 0x05, 0xcb, 0x95, 0xee, 0x99, + 0xdd, 0xf6, 0x5d, 0xdf, 0xc3, 0xc4, 0xf3, 0x3b, 0x6c, 0xb2, 0x09, 0x7f, 0x06, 0xb9, 0x2b, 0x6a, + 0xf4, 0x7a, 0xc4, 0xd2, 0x09, 0xa5, 0x63, 0xa6, 0x2c, 0xe4, 0x30, 0x28, 0xe7, 0x1a, 0x0d, 0xf6, + 0xf0, 0x11, 0xbf, 0x44, 0xcf, 0xd9, 0x1e, 0x75, 0xfd, 0x5e, 0x95, 0x74, 0xc8, 0x9d, 0x9a, 0x45, + 0x60, 0x45, 0xa1, 0xca, 0x8e, 0x4b, 0xa9, 0xdf, 0xe3, 0x5b, 0x2d, 0x23, 0x7b, 0x0f, 0xe6, 0x05, + 0xc8, 0xe9, 0x77, 0xb3, 0x79, 0x4e, 0x98, 0x0f, 0xbd, 0x36, 0x5a, 0x87, 0xf9, 0x1e, 0x75, 0x4d, + 0xe2, 0x79, 0x6a, 0x37, 0xe6, 0x06, 0x75, 0x2b, 0x30, 0x0f, 0x4e, 0x12, 0x52, 0xc3, 0x44, 0x93, + 0xe2, 0x87, 0x00, 0x8a, 0xac, 0x02, 0x40, 0x98, 0xd9, 0x2e, 0x29, 0x40, 0x98, 0x57, 0xfe, 0x02, + 0x11, 0xc2, 0x3f, 0x7c, 0x3b, 0xe5, 0x4f, 0x2b, 0x24, 0x8f, 0x63, 0xe6, 0xd2, 0x51, 0xf2, 0xf0, + 0xb8, 0x35, 0x86, 0x3c, 0x84, 0xb7, 0x24, 0x0f, 0xf5, 0x13, 0x67, 0x85, 0xef, 0x40, 0xf2, 0x90, + 0xd3, 0xce, 0x85, 0xe1, 0x58, 0x1d, 0xce, 0x3a, 0x8c, 0x5e, 0x0f, 0x70, 0x08, 0x6d, 0x41, 0xa6, + 0x57, 0xa3, 0x74, 0xcc, 0x91, 0x17, 0x7e, 0x6a, 0x1d, 0x84, 0xef, 0xfa, 0xdf, 0x53, 0xa0, 0x7d, + 0x7e, 0x47, 0x2d, 0xc8, 0xb4, 0xc4, 0x72, 0xf9, 0x35, 0xcc, 0xb2, 0xbe, 0xc3, 0xc3, 0x97, 0x94, + 0x52, 0xe5, 0x4d, 0x7f, 0xbb, 0x29, 0x7f, 0xd2, 0xb6, 0xd9, 0x85, 0x7f, 0xb6, 0x61, 0xba, 0xdd, + 0xcd, 0xc1, 0xe0, 0xd6, 0x59, 0xf8, 0x7b, 0xb3, 0xf7, 0xa2, 0xbd, 0x29, 0xa8, 0xd9, 0xf7, 0x6d, + 0x6b, 0xe3, 0xe4, 0xa4, 0x5e, 0xbd, 0xbd, 0x29, 0xcf, 0xb4, 0xfa, 0x4e, 0xbd, 0x8a, 0x67, 0x58, + 0xdf, 0xa9, 0x5b, 0x68, 0x17, 0x72, 0x2c, 0x2c, 0xb5, 0x2a, 0x17, 0x26, 0xbb, 0x92, 0xa2, 0x1d, + 0xd5, 0x72, 0x3d, 0x84, 0x7b, 0xad, 0xbe, 0x73, 0x44, 0xc9, 0x65, 0x85, 0x31, 0xd2, 0xed, 0x0d, + 0x81, 0xe1, 0x07, 0x50, 0x6e, 0xf5, 0x9d, 0x4a, 0x87, 0x63, 0xc4, 0x75, 0xcd, 0x31, 0x5d, 0x9f, + 0xb3, 0x89, 0x3a, 0x84, 0x51, 0xc7, 0x5f, 0xa4, 0x60, 0x99, 0x17, 0xd7, 0x36, 0xa1, 0xcd, 0x4b, + 0x42, 0xcf, 0x3b, 0xee, 0x95, 0x5c, 0xa1, 0x07, 0x90, 0x8e, 0xe1, 0x3b, 0x6e, 0x43, 0x1f, 0x42, + 0xde, 0xf4, 0x29, 0x25, 0x0e, 0x53, 0x95, 0x68, 0x5a, 0x6c, 0xb5, 0x8c, 0x74, 0x41, 0x35, 0x89, + 0xb2, 0x83, 0xbe, 0x03, 0x05, 0xdb, 0x31, 0x29, 0xe9, 0x86, 0xce, 0xe9, 0x88, 0xf3, 0xe2, 0xa0, + 0x31, 0x5a, 0xa5, 0x0e, 0x61, 0xe9, 0xd0, 0xee, 0x13, 0xeb, 0xd8, 0x37, 0xf9, 0x71, 0x0e, 0x8e, + 0x40, 0x56, 0x65, 0xd9, 0x9b, 0x4e, 0x01, 0x0e, 0x1c, 0x95, 0xdc, 0xef, 0x52, 0xf0, 0x70, 0x9b, + 0x33, 0x61, 0x58, 0x9b, 0xc9, 0xb9, 0x4b, 0xc9, 0xde, 0xce, 0xe0, 0x92, 0x68, 0xbd, 0xd5, 0x25, + 0x11, 0x72, 0x10, 0x97, 0xb8, 0xa0, 0xc4, 0xe3, 0x0f, 0x94, 0xff, 0xe5, 0x76, 0x08, 0x7b, 0xa9, + 0x58, 0x7f, 0x75, 0x0f, 0x72, 0x22, 0xaa, 0x2a, 0x61, 0x86, 0xdd, 0x41, 0x47, 0x50, 0x74, 0x5c, + 0xa6, 0x0f, 0x3d, 0x7f, 0x64, 0x88, 0xdf, 0x8a, 0x99, 0x7e, 0xcc, 0x13, 0x0c, 0x2f, 0x3a, 0x43, + 0x46, 0x74, 0x08, 0x05, 0xf9, 0x38, 0xe0, 0xba, 0xe7, 0x3c, 0x75, 0x55, 0xc0, 0xff, 0x9f, 0x04, + 0xc4, 0x43, 0x29, 0x8e, 0xf3, 0x34, 0x6a, 0x43, 0x5f, 0x00, 0x92, 0x72, 0x2f, 0xc8, 0xb5, 0x1e, + 0xc0, 0xb7, 0x3a, 0xda, 0x8f, 0x93, 0x14, 0xef, 0x3e, 0x2d, 0x70, 0x91, 0xde, 0x31, 0xa3, 0x9f, + 0xc2, 0x9a, 0xa0, 0xe7, 0x2b, 0x01, 0xd9, 0xba, 0x1f, 0x52, 0xb6, 0x6e, 0x2b, 0xcc, 0x56, 0x20, + 0xff, 0x24, 0xf6, 0x1d, 0xf8, 0x26, 0x3c, 0xc7, 0x8f, 0xe8, 0x38, 0x1f, 0x4e, 0xb6, 0x91, 0x94, + 0xd3, 0x0d, 0xc9, 0x9d, 0xe2, 0x31, 0x97, 0xdb, 0xfa, 0x68, 0x7c, 0xce, 0x46, 0x21, 0x15, 0x23, + 0x36, 0xd2, 0x80, 0x30, 0x14, 0xa3, 0xe2, 0x1c, 0x37, 0xb5, 0x59, 0xa1, 0xfc, 0xc1, 0x78, 0xe5, + 0x01, 0xdd, 0xe2, 0x02, 0x1b, 0xb6, 0xa2, 0x13, 0x58, 0x8a, 0x6a, 0x52, 0x5e, 0xf2, 0xb4, 0x6c, + 0xe2, 0x3e, 0xc4, 0xf2, 0x2c, 0x8e, 0x86, 0x25, 0xcc, 0xe8, 0x4b, 0x88, 0x4e, 0x80, 0x3f, 0xfc, + 0x98, 0xef, 0x69, 0x73, 0x42, 0xf7, 0xc3, 0x89, 0x59, 0x12, 0x47, 0x63, 0x93, 0x76, 0xb4, 0x0b, + 0x0b, 0x57, 0x9c, 0xd7, 0x74, 0x49, 0x73, 0xda, 0xbc, 0xd0, 0x7c, 0x3f, 0x46, 0xf3, 0x2e, 0x32, + 0xe2, 0xdc, 0x55, 0x68, 0x41, 0x7b, 0x90, 0x97, 0x3a, 0xcc, 0x75, 0x75, 0x9e, 0x7f, 0x30, 0x5e, + 0x28, 0x72, 0x15, 0x2a, 0x21, 0x69, 0xe1, 0x99, 0xe1, 0xf6, 0x74, 0xaa, 0xa0, 0x4c, 0xbc, 0x1a, + 0x72, 0x89, 0x99, 0x31, 0x4a, 0x6f, 0x38, 0xef, 0x46, 0x6d, 0x7c, 0x93, 0xcd, 0x00, 0xe3, 0xf4, + 0x73, 0xc1, 0x71, 0xda, 0x42, 0xe2, 0x26, 0xc7, 0x11, 0x1f, 0x2e, 0x98, 0xc3, 0x56, 0x74, 0x00, + 0x8b, 0xb2, 0x14, 0x50, 0xc5, 0x6f, 0x5a, 0x3e, 0x31, 0xc2, 0x51, 0xce, 0xc3, 0xf9, 0x4e, 0xd4, + 0xc6, 0x23, 0x74, 0x5c, 0x8b, 0xe8, 0x7e, 0xf8, 0xf5, 0x41, 0x5b, 0x4c, 0x8c, 0x30, 0xee, 0x3b, + 0x05, 0x2e, 0x38, 0xc3, 0x56, 0xf4, 0x31, 0x64, 0x3c, 0xe2, 0x58, 0x5a, 0x41, 0xe8, 0xbc, 0x1b, + 0xa3, 0x33, 0x20, 0x39, 0x2c, 0x3c, 0x65, 0x05, 0x39, 0x67, 0x7a, 0x9b, 0xb3, 0x92, 0x6e, 0x49, + 0x58, 0xd2, 0x8a, 0x63, 0x2a, 0x48, 0x0c, 0x57, 0xf1, 0x0a, 0x32, 0x6c, 0xe6, 0x27, 0x37, 0xc0, + 0x1c, 0x73, 0x00, 0x59, 0xda, 0x52, 0xe2, 0xc9, 0x8d, 0x07, 0x32, 0xbc, 0x44, 0xef, 0xda, 0x45, + 0x09, 0x55, 0xca, 0xc1, 0x99, 0x43, 0xc9, 0x25, 0x74, 0x04, 0xc0, 0xf8, 0xe3, 0x3e, 0x62, 0xbb, + 0x9b, 0x62, 0x54, 0x3c, 0xfb, 0xb4, 0x95, 0x49, 0x52, 0x2c, 0xf2, 0x44, 0x1c, 0x4a, 0x31, 0x69, + 0xe7, 0x1b, 0x6c, 0x04, 0xf8, 0xab, 0x53, 0xc1, 0xbf, 0xda, 0x6a, 0xe2, 0x06, 0xc7, 0x91, 0x32, + 0x2e, 0x18, 0xc3, 0x56, 0x3e, 0x79, 0x89, 0x78, 0xe1, 0xfd, 0xf1, 0x30, 0x71, 0xf2, 0xa3, 0x88, + 0x88, 0xf3, 0x5e, 0xd4, 0x86, 0xbe, 0x81, 0x87, 0x0a, 0xf9, 0x64, 0xc9, 0xe2, 0x87, 0x88, 0xa7, + 0x9e, 0x2e, 0x70, 0x57, 0x7b, 0x57, 0x48, 0x7f, 0x3b, 0x46, 0x3a, 0x09, 0xed, 0xb0, 0x76, 0x91, + 0x04, 0x7d, 0xa7, 0x70, 0x9f, 0x2b, 0xab, 0x5a, 0xae, 0x1b, 0xde, 0xb5, 0x63, 0x0a, 0xbe, 0x2f, + 0x25, 0xde, 0xa8, 0x31, 0x84, 0x85, 0x11, 0xeb, 0x07, 0x95, 0xbc, 0xc2, 0x25, 0x6a, 0x94, 0xf2, + 0x95, 0xb6, 0x25, 0x45, 0xe9, 0xae, 0xc2, 0x28, 0xad, 0x9c, 0xb8, 0xd2, 0x71, 0xc0, 0x85, 0x0b, + 0xf6, 0xb0, 0x95, 0x5f, 0x41, 0x7e, 0xf8, 0xf5, 0x4f, 0x57, 0x0f, 0x2a, 0x6d, 0x2d, 0xf1, 0x0a, + 0x4a, 0xf8, 0x56, 0x88, 0x91, 0x3f, 0xd2, 0x80, 0xea, 0x90, 0xef, 0x72, 0xc6, 0xd2, 0x3d, 0x09, + 0x59, 0xda, 0x7b, 0x89, 0x1f, 0x55, 0x47, 0x58, 0x0c, 0x2f, 0x74, 0x23, 0x26, 0x74, 0x0a, 0xc5, + 0xc1, 0xd3, 0x58, 0x3f, 0x13, 0x68, 0xa5, 0xad, 0x0b, 0xb5, 0x8d, 0x18, 0xb5, 0x31, 0x24, 0x86, + 0x0b, 0x6c, 0xd8, 0x8e, 0x7c, 0x78, 0x24, 0x76, 0x4c, 0x72, 0xac, 0x4e, 0x42, 0x90, 0x55, 0xe7, + 0xe3, 0x7d, 0x31, 0xce, 0x56, 0xfc, 0xce, 0x8d, 0xc3, 0x5f, 0xbc, 0xca, 0x12, 0x1d, 0x9e, 0xce, + 0xbd, 0x7a, 0x59, 0x4e, 0xa9, 0x67, 0xe4, 0x83, 0xe2, 0xea, 0xb3, 0xcc, 0xdc, 0xa3, 0x62, 0x69, + 0x7d, 0x53, 0xa0, 0xd9, 0x91, 0xeb, 0x89, 0x8a, 0x8c, 0x56, 0x61, 0xc6, 0x76, 0x2c, 0xd2, 0x57, + 0x2f, 0x28, 0x09, 0x74, 0xd2, 0xa4, 0x60, 0xee, 0x0f, 0x69, 0x98, 0x99, 0xec, 0xbd, 0xf9, 0xe3, + 0x61, 0xda, 0xa0, 0x44, 0x7c, 0x60, 0x15, 0x18, 0xb5, 0x18, 0x9b, 0x58, 0x43, 0x35, 0x40, 0x38, + 0x07, 0x1f, 0xd2, 0xd8, 0x48, 0x0b, 0xda, 0x81, 0xbc, 0xef, 0x90, 0x7e, 0xcf, 0xf5, 0x88, 0x25, + 0xae, 0xb5, 0xcc, 0x24, 0x2f, 0x0f, 0xbc, 0x30, 0xe8, 0xc4, 0xaf, 0xb3, 0x4d, 0xc8, 0xb9, 0xd4, + 0x6e, 0xdb, 0x8e, 0xce, 0x4b, 0xbe, 0x00, 0xa1, 0x99, 0xed, 0x45, 0x3e, 0xe6, 0xeb, 0x9b, 0xf2, + 0x2c, 0xbf, 0x1c, 0xea, 0x55, 0x0c, 0xd2, 0x85, 0xff, 0x43, 0x9f, 0xc2, 0xac, 0x25, 0x20, 0x56, + 0xa1, 0x4d, 0x29, 0x89, 0xd7, 0x25, 0xea, 0x62, 0xe5, 0x8d, 0xbe, 0x1b, 0xac, 0x6b, 0x76, 0x5c, + 0xb7, 0x60, 0x1b, 0xd4, 0x8a, 0xa3, 0x27, 0x90, 0x76, 0xdc, 0x2b, 0x05, 0x26, 0x13, 0xb1, 0x37, + 0xf7, 0x7f, 0x3a, 0xf7, 0xcb, 0x97, 0xe5, 0xa9, 0xf0, 0x83, 0xc1, 0x47, 0xbf, 0x4e, 0xdd, 0xf9, + 0x04, 0x37, 0xf8, 0x60, 0x27, 0xbf, 0x6d, 0xb5, 0xf0, 0xa9, 0x3e, 0xf2, 0xd5, 0xeb, 0x1d, 0xb8, + 0x27, 0x5b, 0x9e, 0xe3, 0x7a, 0xab, 0xa6, 0xb7, 0x9a, 0x4d, 0xbd, 0x79, 0x50, 0x2d, 0xa6, 0xd0, + 0x0a, 0x20, 0xd9, 0x50, 0xad, 0x1d, 0xd4, 0x5a, 0x35, 0x1d, 0x57, 0x1a, 0x7b, 0xb5, 0xe2, 0x74, + 0x68, 0x3f, 0xae, 0xe1, 0x7a, 0xe5, 0xa0, 0xfe, 0x55, 0x65, 0xfb, 0xa0, 0x56, 0x4c, 0xa3, 0x07, + 0x70, 0x5f, 0xda, 0x8f, 0x9a, 0xc7, 0xc7, 0xf5, 0xed, 0x83, 0x9a, 0x8e, 0x6b, 0x47, 0x07, 0x95, + 0xd3, 0x62, 0x66, 0x35, 0xf3, 0xb3, 0xdf, 0x96, 0xa6, 0x3e, 0x7a, 0x0a, 0x68, 0x74, 0xe7, 0xd1, + 0x1c, 0x64, 0x1a, 0xcd, 0x46, 0xad, 0x38, 0x85, 0x72, 0x90, 0xdd, 0xae, 0xec, 0xec, 0x37, 0x77, + 0x77, 0x8b, 0x29, 0x94, 0x87, 0xf9, 0xfa, 0xe1, 0x61, 0xad, 0x5a, 0xaf, 0xb4, 0x6a, 0xc5, 0xe9, + 0xed, 0xf7, 0x5e, 0xfd, 0xb3, 0x34, 0xf5, 0xea, 0xb6, 0x94, 0xfa, 0xcb, 0x6d, 0x29, 0xf5, 0xd7, + 0xdb, 0x52, 0xea, 0x1f, 0xb7, 0xa5, 0xd4, 0xcf, 0xff, 0x55, 0x9a, 0xfa, 0x2a, 0xab, 0xd6, 0xf5, + 0xbf, 0x01, 0x00, 0x00, 0xff, 0xff, 0x85, 0xdd, 0x5b, 0xa8, 0x4c, 0x1a, 0x00, 0x00, } diff --git a/pkg/roachpb/errors.proto b/pkg/roachpb/errors.proto index 8dff18b9bca1..41267c98ced1 100644 --- a/pkg/roachpb/errors.proto +++ b/pkg/roachpb/errors.proto @@ -346,6 +346,13 @@ message TxnPrevAttemptError { option (gogoproto.equal) = true; } +// TxnAlreadyEncounteredErrorError indicates that an operation tried to use a +// transaction that already received an error from a previous request. Once that +// happens, client.Txn rejects future requests. +message TxnAlreadyEncounteredErrorError{ + option (gogoproto.equal) = true; +} + // An IntegerOverflowError indicates that an operation was aborted because // it would have caused an integeter overflow. message IntegerOverflowError { @@ -413,6 +420,7 @@ message ErrorDetail { optional UnsupportedRequestError unsupported_request = 32; optional MixedSuccessError mixed_success = 33; optional BatchTimestampBeforeGCError timestamp_before = 34; + optional TxnAlreadyEncounteredErrorError txn_already_encountered_error = 35; } // TransactionRestart indicates how an error should be handled in a diff --git a/pkg/roachpb/string_test.go b/pkg/roachpb/string_test.go index 58ed25b3be8a..e2d35243d8ef 100644 --- a/pkg/roachpb/string_test.go +++ b/pkg/roachpb/string_test.go @@ -72,7 +72,7 @@ func TestBatchRequestString(t *testing.T) { } br.Requests = append(br.Requests, roachpb.RequestUnion{EndTransaction: &roachpb.EndTransactionRequest{}}) - e := fmt.Sprintf(`[txn: %s], Get [/Min,/Min), Get [/Min,/Min), Get [/Min,/Min), Get [/Min,/Min), Get [/Min,/Min), Get [/Min,/Min), Get [/Min,/Min), Get [/Min,/Min), Get [/Min,/Min), Get [/Min,/Min), Get [/Min,/Min), Get [/Min,/Min), Get [/Min,/Min), Get [/Min,/Min), Get [/Min,/Min), Get [/Min,/Min), Get [/Min,/Min), Get [/Min,/Min), Get [/Min,/Min), Get [/Min,/Min), ... 76 skipped ..., Get [/Min,/Min), Get [/Min,/Min), Get [/Min,/Min), Get [/Min,/Min), EndTransaction [/Min,/Min)`, + e := fmt.Sprintf(`[txn: %s], Get [/Min,/Min), Get [/Min,/Min), Get [/Min,/Min), Get [/Min,/Min), Get [/Min,/Min), Get [/Min,/Min), Get [/Min,/Min), Get [/Min,/Min), Get [/Min,/Min), Get [/Min,/Min), Get [/Min,/Min), Get [/Min,/Min), Get [/Min,/Min), Get [/Min,/Min), Get [/Min,/Min), Get [/Min,/Min), Get [/Min,/Min), Get [/Min,/Min), Get [/Min,/Min), Get [/Min,/Min), ... 76 skipped ..., Get [/Min,/Min), Get [/Min,/Min), Get [/Min,/Min), Get [/Min,/Min), EndTransaction(commit:false) [/Min]`, br.Txn.Short()) if e != br.String() { t.Fatalf("e = %s\nv = %s", e, br.String()) diff --git a/pkg/server/updates_test.go b/pkg/server/updates_test.go index 2b85d0199fe4..a32eb13ce3fb 100644 --- a/pkg/server/updates_test.go +++ b/pkg/server/updates_test.go @@ -419,7 +419,7 @@ func TestReportUsage(t *testing.T) { "diagnostics.reporting.send_crash_reports": "false", "server.time_until_store_dead": "1m30s", "trace.debug.enable": "false", - "version": "2.0-5", + "version": "2.0-6", "cluster.secret": "", } { if got, ok := r.last.AlteredSettings[key]; !ok { diff --git a/pkg/settings/cluster/cockroach_versions.go b/pkg/settings/cluster/cockroach_versions.go index 7e393977ca72..81b7fd226e3b 100644 --- a/pkg/settings/cluster/cockroach_versions.go +++ b/pkg/settings/cluster/cockroach_versions.go @@ -53,6 +53,7 @@ const ( VersionRangeAppliedStateKey VersionImportFormats VersionSecondaryLookupJoins + VersionClientSideWritingFlag // Add new versions here (step one of two). @@ -218,6 +219,13 @@ var versionsSingleton = keyedVersions([]keyedVersion{ Key: VersionSecondaryLookupJoins, Version: roachpb.Version{Major: 2, Minor: 0, Unstable: 5}, }, + { + // VersionClientsideWritingFlag is https://github.com/cockroachdb/cockroach/pull/25541. + // Before this version, the Transaction.Writing flag used to be set by the + // server. After, it is set only by the client. + Key: VersionClientSideWritingFlag, + Version: roachpb.Version{Major: 2, Minor: 0, Unstable: 6}, + }, // Add new versions here (step two of two). diff --git a/pkg/sql/conn_executor.go b/pkg/sql/conn_executor.go index b7e17d9da8e3..4b78f86fe53f 100644 --- a/pkg/sql/conn_executor.go +++ b/pkg/sql/conn_executor.go @@ -1561,6 +1561,9 @@ func (ex *connExecutor) synchronizeParallelStmts(ctx context.Context) error { return 1 } return 2 + case *roachpb.TxnAlreadyEncounteredErrorError: + // Another parallel stmt got an error that caused this one. + return 5 case *roachpb.TxnPrevAttemptError: // Symptom of concurrent retry. return 3 diff --git a/pkg/sql/distsql_running.go b/pkg/sql/distsql_running.go index 7b4f10dbc139..340b91982ee9 100644 --- a/pkg/sql/distsql_running.go +++ b/pkg/sql/distsql_running.go @@ -376,7 +376,7 @@ func (r *distSQLReceiver) Push( if meta.TxnMeta != nil { if r.txn != nil { if r.txn.ID() == meta.TxnMeta.Txn.ID { - r.txn.AugmentTxnCoordMeta(*meta.TxnMeta) + r.txn.AugmentTxnCoordMeta(r.ctx, *meta.TxnMeta) } } else { r.resultWriter.SetError( diff --git a/pkg/sql/logictest/testdata/logic_test/crdb_internal b/pkg/sql/logictest/testdata/logic_test/crdb_internal index e1a65b8ff040..5892d284af98 100644 --- a/pkg/sql/logictest/testdata/logic_test/crdb_internal +++ b/pkg/sql/logictest/testdata/logic_test/crdb_internal @@ -244,7 +244,7 @@ select crdb_internal.set_vmodule('') query T select crdb_internal.node_executable_version() ---- -2.0-5 +2.0-6 query ITTT colnames select node_id, component, field, regexp_replace(regexp_replace(value, '^\d+$', ''), e':\\d+', ':') as value from crdb_internal.node_runtime_info @@ -332,4 +332,4 @@ select * from crdb_internal.gossip_alerts query T select crdb_internal.node_executable_version() ---- -2.0-5 +2.0-6 diff --git a/pkg/sql/logictest/testdata/logic_test/parallel_stmts b/pkg/sql/logictest/testdata/logic_test/parallel_stmts index 9c842a182967..00ae6796bd38 100644 --- a/pkg/sql/logictest/testdata/logic_test/parallel_stmts +++ b/pkg/sql/logictest/testdata/logic_test/parallel_stmts @@ -359,17 +359,6 @@ SHOW TIME ZONE ---- UTC -query TT -SHOW CREATE TABLE kv ----- -kv CREATE TABLE kv ( - k INT NOT NULL, - v INT NULL, - CONSTRAINT "primary" PRIMARY KEY (k ASC), - FAMILY "primary" (k, v), - CONSTRAINT check_v CHECK (v < 100) - ) - query T SHOW TRANSACTION STATUS ---- diff --git a/pkg/sql/logictest/testdata/logic_test/show_trace b/pkg/sql/logictest/testdata/logic_test/show_trace index b548d89e9a2d..c57f5a814572 100644 --- a/pkg/sql/logictest/testdata/logic_test/show_trace +++ b/pkg/sql/logictest/testdata/logic_test/show_trace @@ -226,6 +226,8 @@ SELECT operation, message FROM [SHOW KV TRACE FOR SESSION] sql txn CPut /Table/54/1/1/0 -> /TUPLE/2:2:Int/2 dist sender querying next range at /Table/54/1/1/0 dist sender r1: sending batch 1 CPut, 1 BeginTxn, 1 EndTxn, 1 InitPut to (n1,s1):1 +dist sender querying next range at /Table/54/1/1/0 +dist sender r1: sending batch 1 EndTxn to (n1,s1):1 statement error duplicate key value set tracing=kv; @@ -238,6 +240,8 @@ SELECT operation, message FROM [SHOW KV TRACE FOR SESSION] sql txn CPut /Table/54/1/2/0 -> /TUPLE/2:2:Int/2 dist sender querying next range at /Table/54/1/2/0 dist sender r1: sending batch 1 CPut, 1 BeginTxn, 1 EndTxn, 1 InitPut to (n1,s1):1 +dist sender querying next range at /Table/54/1/2/0 +dist sender r1: sending batch 1 EndTxn to (n1,s1):1 query TT SET tracing=kv; @@ -283,6 +287,8 @@ sql txn Del /Table/54/2/3/0 sql txn CPut /Table/54/2/2/0 -> /BYTES/Š dist sender querying next range at /Table/54/1/2/0 dist sender r1: sending batch 1 Put, 1 CPut, 1 Del, 1 BeginTxn, 1 EndTxn to (n1,s1):1 +dist sender querying next range at /Table/54/1/2/0 +dist sender r1: sending batch 1 EndTxn to (n1,s1):1 query TT SET tracing=kv; diff --git a/pkg/sql/sem/tree/stmt.go b/pkg/sql/sem/tree/stmt.go index 98a9ea598903..f991ad06952e 100644 --- a/pkg/sql/sem/tree/stmt.go +++ b/pkg/sql/sem/tree/stmt.go @@ -134,6 +134,9 @@ type HiddenFromShowQueries interface { // IndependentFromParallelizedPriors is a pseudo-interface to be implemented // by statements which do not force parallel statement execution synchronization // when they run. +// NB: Only statements that don't send any requests using the current +// transaction can implement this. Otherwise, the statement will fail if any of +// the parallel statements has encoutered a KV error (which toasts the txn). type IndependentFromParallelizedPriors interface { independentFromParallelizedPriors() } @@ -639,8 +642,7 @@ func (*ShowColumns) StatementType() StatementType { return Rows } // StatementTag returns a short string identifying the type of statement. func (*ShowColumns) StatementTag() string { return "SHOW COLUMNS" } -func (*ShowColumns) hiddenFromStats() {} -func (*ShowColumns) independentFromParallelizedPriors() {} +func (*ShowColumns) hiddenFromStats() {} // StatementType implements the Statement interface. func (*ShowCreateTable) StatementType() StatementType { return Rows } @@ -648,8 +650,7 @@ func (*ShowCreateTable) StatementType() StatementType { return Rows } // StatementTag returns a short string identifying the type of statement. func (*ShowCreateTable) StatementTag() string { return "SHOW CREATE TABLE" } -func (*ShowCreateTable) hiddenFromStats() {} -func (*ShowCreateTable) independentFromParallelizedPriors() {} +func (*ShowCreateTable) hiddenFromStats() {} // StatementType implements the Statement interface. func (*ShowCreateView) StatementType() StatementType { return Rows } @@ -657,8 +658,7 @@ func (*ShowCreateView) StatementType() StatementType { return Rows } // StatementTag returns a short string identifying the type of statement. func (*ShowCreateView) StatementTag() string { return "SHOW CREATE VIEW" } -func (*ShowCreateView) hiddenFromStats() {} -func (*ShowCreateView) independentFromParallelizedPriors() {} +func (*ShowCreateView) hiddenFromStats() {} // StatementType implements the Statement interface. func (*ShowCreateSequence) StatementType() StatementType { return Rows } @@ -666,8 +666,7 @@ func (*ShowCreateSequence) StatementType() StatementType { return Rows } // StatementTag implements the Statement interface. func (*ShowCreateSequence) StatementTag() string { return "SHOW CREATE SEQUENCE" } -func (*ShowCreateSequence) hiddenFromStats() {} -func (*ShowCreateSequence) independentFromParallelizedPriors() {} +func (*ShowCreateSequence) hiddenFromStats() {} // StatementType implements the Statement interface. func (*ShowBackup) StatementType() StatementType { return Rows } @@ -675,8 +674,7 @@ func (*ShowBackup) StatementType() StatementType { return Rows } // StatementTag returns a short string identifying the type of statement. func (*ShowBackup) StatementTag() string { return "SHOW BACKUP" } -func (*ShowBackup) hiddenFromStats() {} -func (*ShowBackup) independentFromParallelizedPriors() {} +func (*ShowBackup) hiddenFromStats() {} // StatementType implements the Statement interface. func (*ShowDatabases) StatementType() StatementType { return Rows } @@ -684,8 +682,7 @@ func (*ShowDatabases) StatementType() StatementType { return Rows } // StatementTag returns a short string identifying the type of statement. func (*ShowDatabases) StatementTag() string { return "SHOW DATABASES" } -func (*ShowDatabases) hiddenFromStats() {} -func (*ShowDatabases) independentFromParallelizedPriors() {} +func (*ShowDatabases) hiddenFromStats() {} // StatementType implements the Statement interface. func (*ShowTrace) StatementType() StatementType { return Rows } @@ -701,8 +698,7 @@ func (*ShowGrants) StatementType() StatementType { return Rows } // StatementTag returns a short string identifying the type of statement. func (*ShowGrants) StatementTag() string { return "SHOW GRANTS" } -func (*ShowGrants) hiddenFromStats() {} -func (*ShowGrants) independentFromParallelizedPriors() {} +func (*ShowGrants) hiddenFromStats() {} // StatementType implements the Statement interface. func (*ShowIndex) StatementType() StatementType { return Rows } @@ -710,8 +706,7 @@ func (*ShowIndex) StatementType() StatementType { return Rows } // StatementTag returns a short string identifying the type of statement. func (*ShowIndex) StatementTag() string { return "SHOW INDEX" } -func (*ShowIndex) hiddenFromStats() {} -func (*ShowIndex) independentFromParallelizedPriors() {} +func (*ShowIndex) hiddenFromStats() {} // StatementType implements the Statement interface. func (*ShowQueries) StatementType() StatementType { return Rows } @@ -737,8 +732,7 @@ func (*ShowRoleGrants) StatementType() StatementType { return Rows } // StatementTag returns a short string identifying the type of statement. func (*ShowRoleGrants) StatementTag() string { return "SHOW GRANTS ON ROLE" } -func (*ShowRoleGrants) hiddenFromStats() {} -func (*ShowRoleGrants) independentFromParallelizedPriors() {} +func (*ShowRoleGrants) hiddenFromStats() {} // StatementType implements the Statement interface. func (*ShowSessions) StatementType() StatementType { return Rows } @@ -755,8 +749,7 @@ func (*ShowTableStats) StatementType() StatementType { return Rows } // StatementTag returns a short string identifying the type of statement. func (*ShowTableStats) StatementTag() string { return "SHOW STATISTICS" } -func (*ShowTableStats) hiddenFromStats() {} -func (*ShowTableStats) independentFromParallelizedPriors() {} +func (*ShowTableStats) hiddenFromStats() {} // StatementType implements the Statement interface. func (*ShowHistogram) StatementType() StatementType { return Rows } @@ -764,8 +757,7 @@ func (*ShowHistogram) StatementType() StatementType { return Rows } // StatementTag returns a short string identifying the type of statement. func (*ShowHistogram) StatementTag() string { return "SHOW HISTOGRAM" } -func (*ShowHistogram) hiddenFromStats() {} -func (*ShowHistogram) independentFromParallelizedPriors() {} +func (*ShowHistogram) hiddenFromStats() {} // StatementType implements the Statement interface. func (*ShowSyntax) StatementType() StatementType { return Rows } @@ -793,8 +785,7 @@ func (*ShowUsers) StatementType() StatementType { return Rows } // StatementTag returns a short string identifying the type of statement. func (*ShowUsers) StatementTag() string { return "SHOW USERS" } -func (*ShowUsers) hiddenFromStats() {} -func (*ShowUsers) independentFromParallelizedPriors() {} +func (*ShowUsers) hiddenFromStats() {} // StatementType implements the Statement interface. func (*ShowRoles) StatementType() StatementType { return Rows } @@ -802,8 +793,7 @@ func (*ShowRoles) StatementType() StatementType { return Rows } // StatementTag returns a short string identifying the type of statement. func (*ShowRoles) StatementTag() string { return "SHOW ROLES" } -func (*ShowRoles) hiddenFromStats() {} -func (*ShowRoles) independentFromParallelizedPriors() {} +func (*ShowRoles) hiddenFromStats() {} // StatementType implements the Statement interface. func (*ShowZoneConfig) StatementType() StatementType { return Rows } @@ -825,16 +815,13 @@ func (*ShowFingerprints) StatementType() StatementType { return Rows } // StatementTag returns a short string identifying the type of statement. func (*ShowFingerprints) StatementTag() string { return "SHOW EXPERIMENTAL_FINGERPRINTS" } -func (*ShowFingerprints) independentFromParallelizedPriors() {} - // StatementType implements the Statement interface. func (*ShowConstraints) StatementType() StatementType { return Rows } // StatementTag returns a short string identifying the type of statement. func (*ShowConstraints) StatementTag() string { return "SHOW CONSTRAINTS" } -func (*ShowConstraints) hiddenFromStats() {} -func (*ShowConstraints) independentFromParallelizedPriors() {} +func (*ShowConstraints) hiddenFromStats() {} // StatementType implements the Statement interface. func (*ShowTables) StatementType() StatementType { return Rows } @@ -842,8 +829,7 @@ func (*ShowTables) StatementType() StatementType { return Rows } // StatementTag returns a short string identifying the type of statement. func (*ShowTables) StatementTag() string { return "SHOW TABLES" } -func (*ShowTables) hiddenFromStats() {} -func (*ShowTables) independentFromParallelizedPriors() {} +func (*ShowTables) hiddenFromStats() {} // StatementType implements the Statement interface. func (*ShowSchemas) StatementType() StatementType { return Rows } @@ -851,8 +837,7 @@ func (*ShowSchemas) StatementType() StatementType { return Rows } // StatementTag returns a short string identifying the type of statement. func (*ShowSchemas) StatementTag() string { return "SHOW SCHEMAS" } -func (*ShowSchemas) hiddenFromStats() {} -func (*ShowSchemas) independentFromParallelizedPriors() {} +func (*ShowSchemas) hiddenFromStats() {} // StatementType implements the Statement interface. func (*Split) StatementType() StatementType { return Rows } diff --git a/pkg/sql/txn_restart_test.go b/pkg/sql/txn_restart_test.go index 18787910417e..3eab77ed9241 100644 --- a/pkg/sql/txn_restart_test.go +++ b/pkg/sql/txn_restart_test.go @@ -738,7 +738,7 @@ END; t.Fatal(err) } if count != 1 { - t.Fatalf("Expected 1 rows, got %d", count) + t.Fatalf("Expected 1 rows, got %d from %s", count, table) } } }) diff --git a/pkg/sql/txn_state_test.go b/pkg/sql/txn_state_test.go index f38943d24d0d..c48f3fec95d6 100644 --- a/pkg/sql/txn_state_test.go +++ b/pkg/sql/txn_state_test.go @@ -57,12 +57,13 @@ func makeTestContext() testContext { manual := hlc.NewManualClock(123) clock := hlc.NewClock(manual.UnixNano, time.Nanosecond) factory := client.TxnSenderFactoryFunc(func(client.TxnType) client.TxnSender { - return client.TxnSenderFunc( - func(context.Context, roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { + return client.TxnSenderAdapter{ + Wrapped: func(context.Context, roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { return nil, nil }, - ) + } }) + settings := cluster.MakeTestingClusterSettings() return testContext{ manualClock: manual, diff --git a/pkg/storage/batcheval/cmd_begin_transaction.go b/pkg/storage/batcheval/cmd_begin_transaction.go index d43ee84d5e10..ccdf0a432509 100644 --- a/pkg/storage/batcheval/cmd_begin_transaction.go +++ b/pkg/storage/batcheval/cmd_begin_transaction.go @@ -20,6 +20,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/storage/batcheval/result" "github.com/cockroachdb/cockroach/pkg/storage/engine" "github.com/cockroachdb/cockroach/pkg/storage/spanset" @@ -48,6 +49,9 @@ func declareKeysBeginTransaction( ) { DeclareKeysWriteTransaction(desc, header, req, spans) spans.Add(spanset.SpanReadOnly, roachpb.Span{Key: keys.RangeTxnSpanGCThresholdKey(header.RangeID)}) + spans.Add(spanset.SpanReadOnly, roachpb.Span{ + Key: keys.AbortSpanKey(header.RangeID, header.Txn.ID), + }) } // BeginTransaction writes the initial transaction record. Fails in @@ -130,7 +134,10 @@ func BeginTransaction( // we bump the record's heartbeat timestamp right before laying it down. reply.Txn.LastHeartbeat.Forward(cArgs.EvalCtx.Clock().Now()) + if !cArgs.EvalCtx.ClusterSettings().Version.IsActive(cluster.VersionClientSideWritingFlag) { + reply.Txn.Writing = true + } + // Write the txn record. - reply.Txn.Writing = true return result.Result{}, engine.MVCCPutProto(ctx, batch, cArgs.Stats, key, hlc.Timestamp{}, nil, reply.Txn) } diff --git a/pkg/storage/replica.go b/pkg/storage/replica.go index c615a8a5f827..f3648e8529f5 100644 --- a/pkg/storage/replica.go +++ b/pkg/storage/replica.go @@ -2942,12 +2942,13 @@ func (r *Replica) evaluateProposal( if pErr != nil { pErr = r.maybeSetCorrupt(ctx, pErr) - // Restore the original txn's Writing bool if the error specifies - // a transaction. - if txn := pErr.GetTxn(); txn != nil { - if ba.Txn == nil { - log.Fatalf(ctx, "error had a txn but batch is non-transactional. Err txn: %s", txn) - } + txn := pErr.GetTxn() + if txn != nil && ba.Txn == nil { + log.Fatalf(ctx, "error had a txn but batch is non-transactional. Err txn: %s", txn) + } + if txn != nil && !r.ClusterSettings().Version.IsActive(cluster.VersionClientSideWritingFlag) { + // Restore the original txn's Writing bool if the error specifies a + // transaction. if txn.ID == ba.Txn.ID { txn.Writing = ba.Txn.Writing } @@ -5229,12 +5230,14 @@ func (r *Replica) evaluateWriteBatch( if pErr == nil && (ba.Timestamp == br.Timestamp || (retryLocally && !isEndTransactionExceedingDeadline(br.Timestamp, *etArg))) { clonedTxn := ba.Txn.Clone() - clonedTxn.Writing = true clonedTxn.Status = roachpb.COMMITTED // Make sure the returned txn has the actual commit // timestamp. This can be different if the stripped batch was // executed at the server's hlc now timestamp. clonedTxn.Timestamp = br.Timestamp + if !r.ClusterSettings().Version.IsActive(cluster.VersionClientSideWritingFlag) { + clonedTxn.Writing = true + } // If the end transaction is not committed, clear the batch and mark the status aborted. if !etArg.Commit { diff --git a/pkg/storage/replica_test.go b/pkg/storage/replica_test.go index f0e496842e60..02b4678fb9c0 100644 --- a/pkg/storage/replica_test.go +++ b/pkg/storage/replica_test.go @@ -5368,7 +5368,6 @@ func TestPushTxnUpgradeExistingTxn(t *testing.T) { expTxn.Timestamp = test.expTS expTxn.Status = roachpb.ABORTED expTxn.LastHeartbeat = test.startTS - expTxn.Writing = true if !reflect.DeepEqual(expTxn, reply.PusheeTxn) { t.Fatalf("unexpected push txn in trial %d: %s", i, pretty.Diff(expTxn, reply.PusheeTxn)) diff --git a/pkg/storage/store_test.go b/pkg/storage/store_test.go index be859a273504..cf5f2f039fc9 100644 --- a/pkg/storage/store_test.go +++ b/pkg/storage/store_test.go @@ -77,7 +77,7 @@ type testSenderFactory struct { store *Store } -func (f *testSenderFactory) New(typ client.TxnType) client.TxnSender { +func (f *testSenderFactory) New(typ client.TxnType, _ *roachpb.Transaction) client.TxnSender { return &testSender{store: f.store} } @@ -93,10 +93,12 @@ type testSender struct { func (db *testSender) GetMeta() roachpb.TxnCoordMeta { panic("unimplemented") } -func (db *testSender) AugmentMeta(roachpb.TxnCoordMeta) { panic("unimplemented") } +func (db *testSender) AugmentMeta(context.Context, roachpb.TxnCoordMeta) { panic("unimplemented") } func (db *testSender) OnFinish(func(error)) { panic("unimplemented") } +func (db *testSender) StartTracking(context.Context) error { panic("unimplemented") } + // Send forwards the call to the single store. This is a poor man's // version of kv.TxnCoordSender, but it serves the purposes of // supporting tests in this package. Transactions are not supported.