From 69fa9f7cff371ce00594770cebf772886ff9d445 Mon Sep 17 00:00:00 2001 From: Andrei Matei Date: Mon, 23 Jul 2018 16:11:12 -0400 Subject: [PATCH] client, kv: move logic out of Txn, rewrite some of the TxnCoordSender This patch moves most of the logic from the client.Txn into the kv.TxnCoordSender and reorganizes much of the TxnCoordSender in the process. The split between the client.Txn and the TxnCoordSender caused a lot of grief historically. The main problem is that both the Txn and the TCS each have their own copy of the roachpb.Transaction proto. They both use their copy for different things. We attempt to keep the two protos in sync, but we can't ensure that as there's no common locking between the two layers. This patch keeps the client.Txn as a mostly stateless shim, allowing one to mock everything underneath. This is nice, as previously "mocking KV" was a less clear proposition - does one mock all the logic in the Txn or just the TCS? Now the TCS has all the logic and all the locking necessary for serializing accesses to the "transaction state" - notably the proto. The Txn and TCS communicate through a (now expanded) client.TxnSender interface. Within the TCS, the biggest change is that everything that has to do with the heartbeat loop has been moved to a new interceptor. The metrics generation has also been extracted into a new interceptor. One behavior change introduced by this patch is that heartbeat loops are no longer started for (what the TCS hopes will be) 1PC txns. The motivation was concern over the price of spawning a (shortlived) heartbeat goroutine per txn in the 1PC-heavy "kv" workload. Another one is that the TxnCoordSender doesn't inherit the old Txn logic for swallowing errors on rollbacks. Instead, we're relying on a recent server change to not return errors on rollbacks when the txn record is missing - which was the reason for said swallowing. Fixes #28256 Release note: none --- pkg/internal/client/client_test.go | 50 +- pkg/internal/client/db_test.go | 26 - pkg/internal/client/sender.go | 246 ++++- pkg/internal/client/txn.go | 783 +++------------ pkg/internal/client/txn_test.go | 563 ++--------- pkg/kv/dist_sender.go | 2 +- pkg/kv/dist_sender_server_test.go | 5 +- pkg/kv/testing_knobs.go | 1 + pkg/kv/txn_coord_sender.go | 946 ++++++++---------- pkg/kv/txn_coord_sender_server_test.go | 12 +- pkg/kv/txn_coord_sender_test.go | 883 +++++++++++++--- pkg/kv/txn_interceptor_heartbeat.go | 528 ++++++++++ pkg/kv/txn_interceptor_intent_collector.go | 6 - pkg/kv/txn_interceptor_metrics.go | 101 ++ pkg/kv/txn_interceptor_sequence_nums.go | 91 ++ pkg/kv/txn_interceptor_span_refresher.go | 8 +- pkg/kv/txn_test.go | 29 +- pkg/kv/txnstate_string.go | 16 + pkg/roachpb/api.pb.go | 1 - pkg/roachpb/batch_generated.go | 4 - pkg/roachpb/errors.go | 10 - pkg/roachpb/errors.pb.go | 599 ++++------- pkg/roachpb/errors.proto | 11 +- pkg/sql/conn_executor.go | 14 +- pkg/sql/conn_executor_exec.go | 6 +- pkg/sql/conn_fsm.go | 3 +- pkg/sql/distsql_running.go | 7 - pkg/sql/distsqlplan/aggregator_funcs_test.go | 3 +- pkg/sql/distsqlrun/server_test.go | 3 +- .../logictest/testdata/logic_test/show_trace | 496 ++++----- pkg/sql/sem/builtins/builtins.go | 3 +- pkg/sql/sem/tree/eval.go | 1 - pkg/sql/sem/tree/timeconv_test.go | 12 +- pkg/sql/sqlbase/fk.go | 2 +- pkg/sql/trace_test.go | 14 +- pkg/sql/txn_state.go | 11 - pkg/sql/txn_state_test.go | 73 +- pkg/storage/client_split_test.go | 9 +- pkg/storage/log.go | 18 +- pkg/storage/log_test.go | 2 +- pkg/storage/store.go | 8 + pkg/storage/store_test.go | 18 +- 42 files changed, 2951 insertions(+), 2673 deletions(-) create mode 100644 pkg/kv/txn_interceptor_heartbeat.go create mode 100644 pkg/kv/txn_interceptor_metrics.go create mode 100644 pkg/kv/txn_interceptor_sequence_nums.go create mode 100644 pkg/kv/txnstate_string.go diff --git a/pkg/internal/client/client_test.go b/pkg/internal/client/client_test.go index 762579632e6b..9449e2d34835 100644 --- a/pkg/internal/client/client_test.go +++ b/pkg/internal/client/client_test.go @@ -850,39 +850,6 @@ func TestReadConsistencyTypes(t *testing.T) { } } -// TestReadOnlyTxnObeysDeadline tests that read-only transactions obey the -// deadline. Read-only transactions have their EndTransaction elided, so the -// enforcement of the deadline is done in the client. -func TestReadOnlyTxnObeysDeadline(t *testing.T) { - defer leaktest.AfterTest(t)() - s, _, _ := serverutils.StartServer(t, base.TestServerArgs{}) - defer s.Stopper().Stop(context.TODO()) - db := createTestClient(t, s) - ctx := context.TODO() - - if err := db.Put(ctx, "k", "v"); err != nil { - t.Fatal(err) - } - - txn := client.NewTxn(db, 0 /* gatewayNodeID */, client.RootTxn) - // Only snapshot transactions can observe deadline errors; serializable ones - // get a restart error before the deadline check. - if err := txn.SetIsolation(enginepb.SNAPSHOT); err != nil { - t.Fatal(err) - } - - // Set a deadline, then set a higher commit timestamp for the txn. - txn.UpdateDeadlineMaybe(ctx, s.Clock().Now()) - txn.Proto().Timestamp.Forward(s.Clock().Now()) - if _, err := txn.Get(ctx, "k"); err != nil { - t.Fatal(err) - } - if err := txn.Commit(ctx); !testutils.IsError( - err, "deadline exceeded before transaction finalization") { - t.Fatal(err) - } -} - // TestTxn_ReverseScan a simple test for Txn.ReverseScan func TestTxn_ReverseScan(t *testing.T) { defer leaktest.AfterTest(t)() @@ -972,13 +939,10 @@ 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.MakeMockTxnSenderFactory( + func(_ context.Context, _ *roachpb.Transaction, ba roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { + return ba.CreateReply(), nil + }) clock := hlc.NewClock(hlc.UnixNano, time.Nanosecond) dbCtx := client.DefaultDBContext() @@ -1000,7 +964,8 @@ func TestNodeIDAndObservedTimestamps(t *testing.T) { for i, test := range directCases { t.Run(fmt.Sprintf("direct-txn-%d", i), func(t *testing.T) { txn := client.NewTxn(db, test.nodeID, test.typ) - if ots := txn.Proto().ObservedTimestamps; (len(ots) == 1 && ots[0].NodeID == test.nodeID) != test.expObserved { + ots := txn.Serialize().ObservedTimestamps + if (len(ots) == 1 && ots[0].NodeID == test.nodeID) != test.expObserved { t.Errorf("expected observed ts %t; got %+v", test.expObserved, ots) } }) @@ -1021,7 +986,8 @@ func TestNodeIDAndObservedTimestamps(t *testing.T) { } if err := db.Txn( ctx, func(_ context.Context, txn *client.Txn) error { - if ots := txn.Proto().ObservedTimestamps; (len(ots) == 1 && ots[0].NodeID == test.nodeID) != test.expObserved { + ots := txn.Serialize().ObservedTimestamps + if (len(ots) == 1 && ots[0].NodeID == test.nodeID) != test.expObserved { t.Errorf("expected observed ts %t; got %+v", test.expObserved, ots) } return nil diff --git a/pkg/internal/client/db_test.go b/pkg/internal/client/db_test.go index 5d6a6d848ab9..123a35632f2e 100644 --- a/pkg/internal/client/db_test.go +++ b/pkg/internal/client/db_test.go @@ -17,14 +17,12 @@ package client_test import ( "bytes" "context" - "fmt" "testing" "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/internal/client" "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" "github.com/cockroachdb/cockroach/pkg/util/leaktest" - "github.com/cockroachdb/cockroach/pkg/util/uuid" ) func setup(t *testing.T) (serverutils.TestServerInterface, *client.DB) { @@ -377,27 +375,3 @@ func TestDB_Put_insecure(t *testing.T) { } checkResult(t, []byte("1"), result.ValueBytes()) } - -func TestDebugName(t *testing.T) { - defer leaktest.AfterTest(t)() - s, db := setup(t) - defer s.Stopper().Stop(context.TODO()) - - if err := db.Txn(context.TODO(), func(ctx context.Context, txn *client.Txn) error { - // Manually override the txn ID, to make the DebugName below deterministic. - id := "00000000-b33f-b33f-b33f-000000000000" - uuid, err := uuid.FromString(id) - if err != nil { - t.Fatal(err) - } - txn.Proto().ID = uuid - - expected := fmt.Sprintf("unnamed (id: %s)", id) - if txn.DebugName() != expected { - t.Fatalf("expected \"%s\", but found \"%s\"", expected, txn.DebugName()) - } - return nil - }); err != nil { - t.Errorf("txn failed: %s", err) - } -} diff --git a/pkg/internal/client/sender.go b/pkg/internal/client/sender.go index a0429264818b..baec42fe7c23 100644 --- a/pkg/internal/client/sender.go +++ b/pkg/internal/client/sender.go @@ -18,6 +18,8 @@ import ( "context" "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/storage/engine/enginepb" + "github.com/cockroachdb/cockroach/pkg/util/hlc" ) // TxnType specifies whether a transaction is the root (parent) @@ -63,13 +65,6 @@ type Sender interface { type TxnSender interface { Sender - // GetMeta retrieves a copy of the TxnCoordMeta, which can be sent - // upstream in situations where there are multiple, leaf TxnSenders, - // to be combined via AugmentMeta(). - GetMeta() roachpb.TxnCoordMeta - // AugmentMeta combines the TxnCoordMeta from another distributed - // TxnSender which is part of the same transaction. - 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,11 +73,105 @@ 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)) + + // SetSystemConfigTrigger sets the system db trigger to true on this transaction. + // This will impact the EndTransactionRequest. + // + // NOTE: The system db trigger will only execute correctly if the transaction + // record is located on the range that contains the system span. If a + // transaction is created which modifies both system *and* non-system data, it + // should be ensured that the transaction record itself is on the system span. + // This can be done by making sure a system key is the first key touched in the + // transaction. + SetSystemConfigTrigger() error + + // GetMeta retrieves a copy of the TxnCoordMeta, which can be sent + // upstream in situations where there are multiple, leaf TxnSenders, + // to be combined via AugmentMeta(). + GetMeta() roachpb.TxnCoordMeta + + // AugmentMeta combines the TxnCoordMeta from another distributed + // TxnSender which is part of the same transaction. + AugmentMeta(ctx context.Context, meta roachpb.TxnCoordMeta) + + // SetUserPriority sets the txn's priority. + SetUserPriority(roachpb.UserPriority) error + + // SetDebugName sets the txn's debug name. + SetDebugName(name string) + + // SetIsolation sets the transaction's isolation level. + SetIsolation(isolation enginepb.IsolationType) error + + // TxnStatus exports the txn's status. + TxnStatus() roachpb.TransactionStatus + + // SetFixedTimestamp makes the transaction run in an unusual way, at a "fixed + // timestamp": Timestamp and OrigTimestamp are set to ts, there's no clock + // uncertainty, and the txn's deadline is set to ts such that the transaction + // can't be pushed to a different timestamp. + // + // This is used to support historical queries (AS OF SYSTEM TIME queries and + // backups). This method must be called on every transaction retry (but note + // that retries should be rare for read-only queries with no clock uncertainty). + SetFixedTimestamp(ctx context.Context, ts hlc.Timestamp) + + // ManualRestart bumps the transactions epoch, and can upgrade the timestamp + // and priority. + // An uninitialized timestamp can be passed to leave the timestamp alone. + // + // Used by the SQL layer which sometimes knows that a transaction will not be + // able to commit and prefers to restart early. + // It is also used after synchronizing concurrent actors using a txn when a + // retryable error is seen. + // TODO(andrei): this second use should go away once we move to a TxnAttempt + // model. + ManualRestart(context.Context, roachpb.UserPriority, hlc.Timestamp) + + // UpdateStateOnRemoteRetryableErr updates the txn in response to an error + // encountered when running a request through the txn. + UpdateStateOnRemoteRetryableErr(context.Context, *roachpb.Error) *roachpb.Error + // DisablePipelining instructs the TxnSender not to pipeline requests. It // should rarely be necessary to call this method. It is only recommended for // transactions that need extremely precise control over the request ordering, // like the transaction that merges ranges together. - DisablePipelining() + DisablePipelining() error + + // OrigTimestamp returns the transaction's starting timestamp. + // Note a transaction can be internally pushed forward in time before + // committing so this is not guaranteed to be the commit timestamp. + // Use CommitTimestamp() when needed. + OrigTimestamp() hlc.Timestamp + + // CommitTimestamp returns the transaction's start timestamp. + // The start timestamp can get pushed but the use of this + // method will guarantee that the caller of this method sees + // the push and thus calls this method again to receive the new + // timestamp. + CommitTimestamp() hlc.Timestamp + + // IsSerializablePushAndRefreshNotPossible returns true if the transaction is + // serializable, its timestamp has been pushed and there's no chance that + // refreshing the read spans will succeed later (thus allowing the transaction + // to commit and not be restarted). Used to detect whether the txn is + // guaranteed to get a retriable error later. + // + // Note that this method allows for false negatives: sometimes the client only + // figures out that it's been pushed when it sends an EndTransaction - i.e. + // it's possible for the txn to have been pushed asynchoronously by some other + // operation (usually, but not exclusively, by a high-priority txn with + // conflicting writes). + IsSerializablePushAndRefreshNotPossible() bool + + // Epoch returns the txn's epoch. + Epoch() uint32 + + // SerializeTxn returns a clone of the transaction's current proto. + // This is a nuclear option; generally client code shouldn't deal with protos. + // However, this is used by DistSQL for sending the transaction over the wire + // when it creates flows. + SerializeTxn() *roachpb.Transaction } // TxnSenderFactory is the interface used to create new instances @@ -111,45 +200,144 @@ 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) +// MockTransactionalSender allows a function to be used as a TxnSender. +type MockTransactionalSender struct { + senderFunc func( + context.Context, *roachpb.Transaction, roachpb.BatchRequest, + ) (*roachpb.BatchResponse, *roachpb.Error) + txn roachpb.Transaction +} -// Send calls f(ctx, c). -func (f TxnSenderFunc) Send( +// NewMockTransactionalSender creates a MockTransactionalSender. +// The passed in txn is cloned. +func NewMockTransactionalSender( + f func( + context.Context, *roachpb.Transaction, roachpb.BatchRequest, + ) (*roachpb.BatchResponse, *roachpb.Error), + txn *roachpb.Transaction, +) *MockTransactionalSender { + return &MockTransactionalSender{senderFunc: f, txn: txn.Clone()} +} + +// Send is part of the TxnSender interface. +func (m *MockTransactionalSender) Send( ctx context.Context, ba roachpb.BatchRequest, ) (*roachpb.BatchResponse, *roachpb.Error) { - return f(ctx, ba) + return m.senderFunc(ctx, &m.txn, ba) } // GetMeta is part of the TxnSender interface. -func (f TxnSenderFunc) GetMeta() roachpb.TxnCoordMeta { panic("unimplemented") } +func (m *MockTransactionalSender) GetMeta() roachpb.TxnCoordMeta { panic("unimplemented") } // AugmentMeta is part of the TxnSender interface. -func (f TxnSenderFunc) AugmentMeta(context.Context, roachpb.TxnCoordMeta) { panic("unimplemented") } +func (m *MockTransactionalSender) AugmentMeta(context.Context, roachpb.TxnCoordMeta) { + panic("unimplemented") +} // OnFinish is part of the TxnSender interface. -func (f TxnSenderFunc) OnFinish(_ func(error)) { panic("unimplemented") } +func (m *MockTransactionalSender) OnFinish(_ func(error)) { panic("unimplemented") } + +// SetSystemConfigTrigger is part of the TxnSender interface. +func (m *MockTransactionalSender) SetSystemConfigTrigger() error { panic("unimplemented") } + +// TxnStatus is part of the TxnSender interface. +func (m *MockTransactionalSender) TxnStatus() roachpb.TransactionStatus { + return m.txn.Status +} + +// SetUserPriority is part of the TxnSender interface. +func (m *MockTransactionalSender) SetUserPriority(pri roachpb.UserPriority) error { + m.txn.Priority = roachpb.MakePriority(pri) + return nil +} -// DisablePipelining is part of the TxnSender interface. -func (f TxnSenderFunc) DisablePipelining() { panic("unimplemented") } +// SetDebugName is part of the TxnSender interface. +func (m *MockTransactionalSender) SetDebugName(name string) { + m.txn.Name = name +} -// 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 +// SetIsolation is part of the TxnSender interface. +func (m *MockTransactionalSender) SetIsolation(isolation enginepb.IsolationType) error { + m.txn.Isolation = isolation + return nil +} -var _ TxnSenderFactory = TxnSenderFactoryFunc(nil) +// OrigTimestamp is part of the TxnSender interface. +func (m *MockTransactionalSender) OrigTimestamp() hlc.Timestamp { + return m.txn.OrigTimestamp +} + +// CommitTimestamp is part of the TxnSender interface. +func (m *MockTransactionalSender) CommitTimestamp() hlc.Timestamp { + return m.txn.OrigTimestamp +} + +// SetFixedTimestamp is part of the TxnSender interface. +func (m *MockTransactionalSender) SetFixedTimestamp(context.Context, hlc.Timestamp) { + panic("unimplemented") +} + +// ManualRestart is part of the TxnSender interface. +func (m *MockTransactionalSender) ManualRestart( + ctx context.Context, pri roachpb.UserPriority, ts hlc.Timestamp, +) { + m.txn.Restart(pri, 0 /* upgradePriority */, ts) +} + +// IsSerializablePushAndRefreshNotPossible is part of the TxnSender interface. +func (m *MockTransactionalSender) IsSerializablePushAndRefreshNotPossible() bool { + panic("unimplemented") +} + +// Epoch is part of the TxnSender interface. +func (m *MockTransactionalSender) Epoch() uint32 { panic("unimplemented") } + +// SerializeTxn is part of the TxnSender interface. +func (m *MockTransactionalSender) SerializeTxn() *roachpb.Transaction { + cp := m.txn.Clone() + return &cp +} + +// UpdateStateOnRemoteRetryableErr is part of the TxnSender interface. +func (m *MockTransactionalSender) UpdateStateOnRemoteRetryableErr( + ctx context.Context, pErr *roachpb.Error, +) *roachpb.Error { + panic("unimplemented") +} + +// DisablePipelining is part of the client.TxnSender interface. +func (m *MockTransactionalSender) DisablePipelining() error { return nil } + +// MockTxnSenderFactory is a TxnSenderFactory producing MockTxnSenders. +type MockTxnSenderFactory struct { + senderFunc func(context.Context, *roachpb.Transaction, roachpb.BatchRequest) ( + *roachpb.BatchResponse, *roachpb.Error) +} + +var _ TxnSenderFactory = MockTxnSenderFactory{} + +// MakeMockTxnSenderFactory creates a MockTxnSenderFactory from a sender +// function that receives the transaction in addition to the request. The +// function is responsible for putting the txn inside the batch, if needed. +func MakeMockTxnSenderFactory( + senderFunc func( + context.Context, *roachpb.Transaction, roachpb.BatchRequest, + ) (*roachpb.BatchResponse, *roachpb.Error), +) MockTxnSenderFactory { + return MockTxnSenderFactory{ + senderFunc: senderFunc, + } +} // TransactionalSender is part of TxnSenderFactory. -func (f TxnSenderFactoryFunc) TransactionalSender(typ TxnType, _ roachpb.TxnCoordMeta) TxnSender { - return f(typ) +func (f MockTxnSenderFactory) TransactionalSender( + _ TxnType, coordMeta roachpb.TxnCoordMeta, +) TxnSender { + return NewMockTransactionalSender(f.senderFunc, &coordMeta.Txn) } // NonTransactionalSender is part of TxnSenderFactory. -func (f TxnSenderFactoryFunc) NonTransactionalSender() Sender { +func (f MockTxnSenderFactory) NonTransactionalSender() Sender { return nil } diff --git a/pkg/internal/client/txn.go b/pkg/internal/client/txn.go index 3f40907c2e38..72700a8996a8 100644 --- a/pkg/internal/client/txn.go +++ b/pkg/internal/client/txn.go @@ -20,7 +20,6 @@ import ( "github.com/pkg/errors" - "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/storage/engine/enginepb" "github.com/cockroachdb/cockroach/pkg/util/hlc" @@ -57,78 +56,32 @@ type Txn struct { // mu holds fields that need to be synchronized for concurrent request execution. mu struct { syncutil.Mutex - Proto roachpb.Transaction - // UserPriority is the transaction's priority. If not set, + ID uuid.UUID + debugName string + isolation enginepb.IsolationType + + // userPriority is the transaction's priority. If not set, // NormalUserPriority will be used. - // TODO(andrei): Can this go away now that UserPriority and Proto.Priority - // are initialized at the same time? - UserPriority roachpb.UserPriority - // active is set whenever the transaction is actively running. It will - // be initially set when the transaction sends its first batch, but is - // reset if the transaction is aborted. - active bool - // 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 - state txnState - // see IsFinalized() - finalized bool + userPriority roachpb.UserPriority + // previousIDs holds the set of all previous IDs that the Txn's Proto has // had across transaction aborts. This allows us to determine if a given - // response was meant for any incarnation of this transaction. - // - // TODO(andrei): This concept of the history of IDs seems problematic to me: - // if we get a retryable error for a really old incarnation of a - // transaction, a client (or Txn.Exec) probably doesn't want to retry. - // If Txn stays multi-threaded, I think restarting a transaction should - // become an explicit client operation that doesn't allow more requests to - // be sent for the old incarnation, and also doesn't let results for - // requests sent through old incarnations to be processed. - // If Txn becomes single-threaded, then the point is moot and this can again - // go away. + // response was meant for any incarnation of this transaction. This is + // useful for catching retriable errors that have escaped inner + // transactions, so that they don't cause a retry of an outer transaction. previousIDs map[uuid.UUID]struct{} - // sender is a stateful sender for use with transactions. A new sender is - // created on transaction restarts (not retries). + + // sender is a stateful sender for use with transactions (usually a + // TxnCoordSender). A new sender is created on transaction restarts (not + // retries). sender TxnSender + // The txn has to be committed by this deadline. A nil value indicates no // deadline. deadline *hlc.Timestamp } } -// 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 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). @@ -184,7 +137,7 @@ func NewTxnWithCoordMeta( } meta.Txn.AssertInitialized(context.TODO()) txn := &Txn{db: db, typ: typ, gatewayNodeID: gatewayNodeID} - txn.mu.Proto = meta.Txn + txn.mu.ID = meta.Txn.ID txn.mu.sender = db.factory.TransactionalSender(typ, meta) return txn } @@ -205,26 +158,19 @@ func (txn *Txn) Sender() TxnSender { func (txn *Txn) ID() uuid.UUID { txn.mu.Lock() defer txn.mu.Unlock() - return txn.mu.Proto.ID + return txn.mu.ID } -// IsFinalized returns true if this Txn has been finalized and should therefore -// not be used for any more KV operations. -// A Txn is considered finalized if it successfully committed or if a rollback -// was attempted (successful or not). -// Note that Commit() always leaves the transaction finalized, since it attempts -// to rollback on error. -func (txn *Txn) IsFinalized() bool { +// Epoch exports the txn's epoch. +func (txn *Txn) Epoch() uint32 { txn.mu.Lock() defer txn.mu.Unlock() - return txn.mu.finalized + return txn.mu.sender.Epoch() } // status returns the txn proto status field. func (txn *Txn) status() roachpb.TransactionStatus { - txn.mu.Lock() - defer txn.mu.Unlock() - return txn.mu.Proto.Status + return txn.mu.sender.TxnStatus() } // IsCommitted returns true if the transaction has the committed status. @@ -238,19 +184,17 @@ func (txn *Txn) IsCommitted() bool { func (txn *Txn) SetUserPriority(userPriority roachpb.UserPriority) error { txn.mu.Lock() defer txn.mu.Unlock() - if txn.mu.UserPriority == userPriority { + if txn.mu.userPriority == userPriority { return nil } - if txn.mu.active { - return errors.Errorf("cannot change the user priority of a running transaction") - } + if userPriority < roachpb.MinUserPriority || userPriority > roachpb.MaxUserPriority { return errors.Errorf("the given user priority %f is out of the allowed range [%f, %f]", userPriority, roachpb.MinUserPriority, roachpb.MaxUserPriority) } - txn.mu.UserPriority = userPriority - txn.mu.Proto.Priority = roachpb.MakePriority(userPriority) - return nil + + txn.mu.userPriority = userPriority + return txn.mu.sender.SetUserPriority(userPriority) } // InternalSetPriority sets the transaction priority. It is intended for @@ -259,8 +203,10 @@ func (txn *Txn) InternalSetPriority(priority int32) { txn.mu.Lock() // The negative user priority is translated on the server into a positive, // non-randomized, priority for the transaction. - txn.mu.UserPriority = roachpb.UserPriority(-priority) - txn.mu.Proto.Priority = roachpb.MakePriority(txn.mu.UserPriority) + txn.mu.userPriority = roachpb.UserPriority(-priority) + if err := txn.mu.sender.SetUserPriority(txn.mu.userPriority); err != nil { + log.Fatal(context.TODO(), err) + } txn.mu.Unlock() } @@ -268,7 +214,7 @@ func (txn *Txn) InternalSetPriority(priority int32) { func (txn *Txn) UserPriority() roachpb.UserPriority { txn.mu.Lock() defer txn.mu.Unlock() - return txn.mu.UserPriority + return txn.mu.userPriority } // SetDebugName sets the debug name associated with the transaction which will @@ -276,20 +222,20 @@ func (txn *Txn) UserPriority() roachpb.UserPriority { func (txn *Txn) SetDebugName(name string) { txn.mu.Lock() defer txn.mu.Unlock() - if txn.mu.Proto.Name == name { - return - } - if txn.mu.active { - panic("cannot change the debug name of a running transaction") - } - txn.mu.Proto.Name = name + + txn.mu.sender.SetDebugName(name) + txn.mu.debugName = name } // DebugName returns the debug name associated with the transaction. func (txn *Txn) DebugName() string { txn.mu.Lock() defer txn.mu.Unlock() - return fmt.Sprintf("%s (id: %s)", txn.mu.Proto.Name, txn.mu.Proto.ID) + return txn.debugNameLocked() +} + +func (txn *Txn) debugNameLocked() string { + return fmt.Sprintf("%s (id: %s)", txn.mu.debugName, txn.mu.ID) } // SetIsolation sets the transaction's isolation type. Transactions default to @@ -298,13 +244,11 @@ func (txn *Txn) DebugName() string { func (txn *Txn) SetIsolation(isolation enginepb.IsolationType) error { txn.mu.Lock() defer txn.mu.Unlock() - if txn.mu.Proto.Isolation == isolation { - return nil - } - if txn.mu.active { - return errors.Errorf("cannot change the isolation level of a running transaction") + + if err := txn.mu.sender.SetIsolation(isolation); err != nil { + return err } - txn.mu.Proto.Isolation = isolation + txn.mu.isolation = isolation return nil } @@ -312,7 +256,7 @@ func (txn *Txn) SetIsolation(isolation enginepb.IsolationType) error { func (txn *Txn) Isolation() enginepb.IsolationType { txn.mu.Lock() defer txn.mu.Unlock() - return txn.mu.Proto.Isolation + return txn.mu.isolation } // OrigTimestamp returns the transaction's starting timestamp. @@ -320,9 +264,7 @@ func (txn *Txn) Isolation() enginepb.IsolationType { // committing so this is not guaranteed to be the commit timestamp. // Use CommitTimestamp() when needed. func (txn *Txn) OrigTimestamp() hlc.Timestamp { - txn.mu.Lock() - defer txn.mu.Unlock() - return txn.mu.Proto.OrigTimestamp + return txn.mu.sender.OrigTimestamp() } // CommitTimestamp returns the transaction's start timestamp. @@ -330,42 +272,14 @@ func (txn *Txn) OrigTimestamp() hlc.Timestamp { // method will guarantee that if a timestamp push is needed // the commit will fail with a retryable error. func (txn *Txn) CommitTimestamp() hlc.Timestamp { - txn.mu.Lock() - defer txn.mu.Unlock() - txn.mu.Proto.OrigTimestampWasObserved = true - return txn.mu.Proto.OrigTimestamp -} - -// SetTxnAnchorKey sets the key at which to anchor the transaction record. The -// transaction anchor key defaults to the first key written in a transaction. -func (txn *Txn) SetTxnAnchorKey(key roachpb.Key) error { - txn.mu.Lock() - defer txn.mu.Unlock() - if txn.mu.state != txnReadOnly { - return errors.Errorf("transaction anchor key already set") - } - txn.mu.txnAnchorKey = key - return nil + return txn.mu.sender.CommitTimestamp() } // SetSystemConfigTrigger sets the system db trigger to true on this transaction. // This will impact the EndTransactionRequest. -// -// NOTE: The system db trigger will only execute correctly if the transaction -// record is located on the range that contains the system span. If a -// transaction is created which modifies both system *and* non-system data, it -// should be ensured that the transaction record itself is on the system span. -// This can be done by making sure a system key is the first key touched in the -// transaction. func (txn *Txn) SetSystemConfigTrigger() error { - if !txn.systemConfigTrigger { - txn.systemConfigTrigger = true - // The system-config trigger must be run on the system-config range which - // means any transaction with the trigger set needs to be anchored to the - // system-config range. - return txn.SetTxnAnchorKey(keys.SystemConfigSpan.Key) - } - return nil + txn.systemConfigTrigger = true + return txn.mu.sender.SetSystemConfigTrigger() } // DisablePipelining instructs the transaction not to pipeline requests. It @@ -376,22 +290,7 @@ func (txn *Txn) SetSystemConfigTrigger() error { // DisablePipelining must be called before any operations are performed on the // transaction. func (txn *Txn) DisablePipelining() error { - txn.mu.Lock() - defer txn.mu.Unlock() - if txn.mu.active { - return errors.Errorf("cannot disable pipelining on a running transaction") - } - txn.mu.sender.DisablePipelining() - return nil -} - -// Proto returns the transactions underlying protocol buffer. It is not thread-safe, -// only use if you know that no requests are executing concurrently. -// -// A thread-safe alternative would be to clone the Proto under lock and return -// this clone, but we currently have no situations where this is needed. -func (txn *Txn) Proto() *roachpb.Transaction { - return &txn.mu.Proto + return txn.mu.sender.DisablePipelining() } // NewBatch creates and returns a new empty batch object for use with the Txn. @@ -586,7 +485,9 @@ func (txn *Txn) Run(ctx context.Context, b *Batch) error { } func (txn *Txn) commit(ctx context.Context) error { - pErr := txn.sendEndTxnReq(ctx, true /* commit */, txn.deadline()) + var ba roachpb.BatchRequest + ba.Add(endTxnReq(true /* commit */, txn.deadline(), txn.systemConfigTrigger)) + _, pErr := txn.Send(ctx, ba) if pErr == nil { for _, t := range txn.commitTriggers { t(ctx) @@ -600,16 +501,11 @@ func (txn *Txn) CleanupOnError(ctx context.Context, err error) { if err == nil { log.Fatal(ctx, "CleanupOnError called with nil error") } - // This may race with a concurrent EndTxnRequests. That's fine though because - // we're just trying to clean up and will happily log the failed Rollback error - // if someone beat us. - if txn.status() == roachpb.PENDING { - if replyErr := txn.rollback(ctx); replyErr != nil { - if _, ok := replyErr.GetDetail().(*roachpb.TransactionStatusError); ok || txn.status() == roachpb.ABORTED { - log.Eventf(ctx, "failure aborting transaction: %s; abort caused by: %s", replyErr, err) - } else { - log.Warningf(ctx, "failure aborting transaction: %s; abort caused by: %s", replyErr, err) - } + if replyErr := txn.rollback(ctx); replyErr != nil { + if _, ok := replyErr.GetDetail().(*roachpb.TransactionStatusError); ok || txn.status() == roachpb.ABORTED { + log.Eventf(ctx, "failure aborting transaction: %s; abort caused by: %s", replyErr, err) + } else { + log.Warningf(ctx, "failure aborting transaction: %s; abort caused by: %s", replyErr, err) } } } @@ -657,10 +553,9 @@ func (txn *Txn) UpdateDeadlineMaybe(ctx context.Context, deadline hlc.Timestamp) txn.mu.Lock() defer txn.mu.Unlock() if txn.mu.deadline == nil || deadline.Less(*txn.mu.deadline) { - if deadline.Less(txn.mu.Proto.OrigTimestamp) { - log.Fatalf(ctx, "deadline below txn.OrigTimestamp is nonsensical; "+ - "txn has would have no change to commit. Deadline: %s, txn: %s", - deadline, txn.Proto()) + if deadline.Less(txn.OrigTimestamp()) { + log.Fatalf(ctx, "deadline below txn.OrigTimestamp() is nonsensical; "+ + "txn has would have no change to commit. Deadline: %s", deadline) } txn.mu.deadline = &deadline return true @@ -682,19 +577,14 @@ func (txn *Txn) Rollback(ctx context.Context) error { func (txn *Txn) rollback(ctx context.Context) *roachpb.Error { log.VEventf(ctx, 2, "rolling back transaction") - // Mark the txn as finalized. We don't allow any more requests to be sent once - // a rollback is attempted. Also, the SQL layer likes to assert that the - // transaction has been finalized after attempting cleanup. - txn.mu.Lock() - txn.mu.finalized = true - txn.mu.Unlock() - sync := true if ctx.Err() != nil { sync = false } if sync { - pErr := txn.sendEndTxnReq(ctx, false /* commit */, nil) + var ba roachpb.BatchRequest + ba.Add(endTxnReq(false /* commit */, nil /* deadline */, false /* systemConfigTrigger */)) + _, pErr := txn.Send(ctx, ba) if pErr == nil { return nil } @@ -709,8 +599,10 @@ func (txn *Txn) rollback(ctx context.Context) *roachpb.Error { ctx, cancel := stopper.WithCancelOnQuiesce(txn.db.AnnotateCtx(context.Background())) if err := stopper.RunAsyncTask(ctx, "async-rollback", func(ctx context.Context) { defer cancel() - if err := txn.sendEndTxnReq(ctx, false /* commit */, nil); err != nil { - log.Infof(ctx, "async rollback failed: %s", err) + var ba roachpb.BatchRequest + ba.Add(endTxnReq(false /* commit */, nil /* deadline */, false /* systemConfigTrigger */)) + if _, pErr := txn.Send(ctx, ba); pErr != nil { + log.Infof(ctx, "async rollback failed: %s", pErr) } }); err != nil { cancel() @@ -738,85 +630,6 @@ func (txn *Txn) OnCurrentIncarnationFinish(onFinishFn func(error)) { txn.mu.sender.OnFinish(onFinishFn) } -// 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) 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 - // or expired. We will just keep the code for safety (see TestReacquireLeaseOnRestart). - if deadline != nil && deadline.Less(txn.mu.Proto.Timestamp) { - // NB: The returned error contains a pointer to txn.mu.Proto, but that's ok - // because we can't have concurrent operations going on while - // committing/aborting. - // - // TODO(andrei): What is happening in this case? - // - // 1. our txn starts at ts1 - // 2. catches uncertainty read error - // 3. updates its timestamp - // 4. new timestamp violates deadline - // 5. txn retries the read - // 6. commit fails - only thanks to this code path? - return roachpb.NewErrorWithTxn(roachpb.NewTransactionStatusError( - "deadline exceeded before transaction finalization"), &txn.mu.Proto) - } - if commit { - txn.mu.Proto.Status = roachpb.COMMITTED - } else { - txn.mu.Proto.Status = roachpb.ABORTED - } - return nil -} - -func (txn *Txn) sendEndTxnReq( - ctx context.Context, commit bool, deadline *hlc.Timestamp, -) *roachpb.Error { - txn.mu.Lock() - if txn.mu.state == txnReadOnly { - defer txn.mu.Unlock() - return txn.finishReadonlyLocked(ctx, commit, deadline) - } - - 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 - } - txn.mu.Unlock() - - var ba roachpb.BatchRequest - ba.Add(endTxnReq(commit, deadline, txn.systemConfigTrigger)) - _, pErr := txn.Send(ctx, ba) - if pErr == nil { - return nil - } - if !commit { - // Swallow some status errors. 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 (i.e. if the BeginTransaction batch - // failed). One would think that, depending on the error received by that - // 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. - if tse, ok := pErr.GetDetail().(*roachpb.TransactionStatusError); ok && - tse.Reason == roachpb.TransactionStatusError_REASON_TXN_NOT_FOUND { - return nil - } - } - return pErr -} - func endTxnReq(commit bool, deadline *hlc.Timestamp, hasTrigger bool) roachpb.Request { req := &roachpb.EndTransactionRequest{ Commit: commit, @@ -866,7 +679,7 @@ func (txn *Txn) exec(ctx context.Context, fn func(context.Context, *Txn) error) } if txn.status() == roachpb.PENDING { err = txn.Commit(ctx) - log.Eventf(ctx, "client.Txn did AutoCommit. err: %v\ntxn: %+v", err, txn.Proto()) + log.Eventf(ctx, "client.Txn did AutoCommit. err: %v\n", err) if err != nil { if _, retryable := err.(*roachpb.HandledRetryableTxnError); !retryable { // We can't retry, so let the caller know we tried to @@ -895,9 +708,7 @@ func (txn *Txn) exec(ctx context.Context, fn func(context.Context, *Txn) error) // If it's not, we terminate the "retryable" character of the error. We // might get a HandledRetryableTxnError if the closure ran another // transaction internally and let the error propagate upwards. - return errors.Wrapf( - err, - "retryable error from another txn. Current txn ID: %v", txn.Proto().ID) + return errors.Wrapf(err, "retryable error from another txn") } retryable = true } @@ -927,10 +738,7 @@ func (txn *Txn) PrepareForRetry(ctx context.Context, err error) { func (txn *Txn) IsRetryableErrMeantForTxn(retryErr roachpb.HandledRetryableTxnError) bool { txn.mu.Lock() defer txn.mu.Unlock() - return txn.isRetryableErrMeantForTxnLocked(retryErr) -} -func (txn *Txn) isRetryableErrMeantForTxnLocked(retryErr roachpb.HandledRetryableTxnError) bool { errTxnID := retryErr.TxnID // Make sure the txn record that err carries is for this txn. @@ -940,7 +748,7 @@ func (txn *Txn) isRetryableErrMeantForTxnLocked(retryErr roachpb.HandledRetryabl return true } // If not, make sure it was meant for this transaction. - return errTxnID == txn.mu.Proto.ID + return errTxnID == txn.mu.ID } // Send runs the specified calls synchronously in a single batch and @@ -952,13 +760,6 @@ func (txn *Txn) isRetryableErrMeantForTxnLocked(retryErr roachpb.HandledRetryabl func (txn *Txn) Send( ctx context.Context, ba roachpb.BatchRequest, ) (*roachpb.BatchResponse, *roachpb.Error) { - // It doesn't make sense to use inconsistent reads in a transaction. However, - // we still need to accept it as a parameter for this to compile. - if ba.ReadConsistency != roachpb.CONSISTENT { - return nil, roachpb.NewErrorf("cannot use %s ReadConsistency in txn", - ba.ReadConsistency) - } - // Fill in the GatewayNodeID on the batch if the txn knows it. // NOTE(andrei): It seems a bit ugly that we're filling in the batches here as // opposed to the point where the requests are being created, but @@ -968,264 +769,30 @@ func (txn *Txn) Send( ba.Header.GatewayNodeID = txn.gatewayNodeID } - lastIndex := len(ba.Requests) - 1 - if lastIndex < 0 { - return nil, nil - } - - firstWriteIdx, pErr := firstWriteIndex(ba) - if pErr != nil { - return nil, pErr - } - - haveTxnWrite := firstWriteIdx != -1 - endTxnRequest, haveEndTxn := ba.Requests[lastIndex].GetInner().(*roachpb.EndTransactionRequest) + requestTxnID := txn.ID() - var needBeginTxn, elideEndTxn bool - var sender TxnSender - lockedPrelude := func() *roachpb.Error { - 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 { - onlyRollback := lastIndex == 0 && haveEndTxn && !endTxnRequest.Commit - if !onlyRollback { - return roachpb.NewErrorf( - "attempting to use transaction with wrong status or finalized: %s %v", - txn.mu.Proto.Status, txn.mu.finalized) - } - } - - // For testing purposes, txn.UserPriority can be a negative value (see - // roachpb.MakePriority). - if txn.mu.UserPriority != 0 { - ba.UserPriority = txn.mu.UserPriority - } - - if !txn.mu.active { - user := roachpb.MakePriority(ba.UserPriority) - if txn.mu.Proto.Priority < user { - txn.mu.Proto.Priority = user - } - txn.mu.active = true - } - - 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 { - // 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. - if len(txn.mu.Proto.Key) == 0 { - txnAnchorKey := txn.mu.txnAnchorKey - if len(txnAnchorKey) == 0 { - txnAnchorKey = ba.Requests[firstWriteIdx].GetInner().Header().Key - } - txn.mu.Proto.Key = txnAnchorKey - } - // Set the key in the begin transaction request to the txn's anchor key. - bt := &roachpb.BeginTransactionRequest{ - RequestHeader: roachpb.RequestHeader{ - Key: txn.mu.Proto.Key, - }, - } - // Inject the new request before position firstWriteIdx, taking - // care to avoid unnecessary allocations. - oldRequests := ba.Requests - ba.Requests = make([]roachpb.RequestUnion, len(ba.Requests)+1) - copy(ba.Requests, oldRequests[:firstWriteIdx]) - ba.Requests[firstWriteIdx].MustSetInner(bt) - copy(ba.Requests[firstWriteIdx+1:], oldRequests[firstWriteIdx:]) - } - - if elideEndTxn { - ba.Requests = ba.Requests[:lastIndex] - } - - // Set the sequence number of each individual Request. The sequence - // number is used for replay and reordering protection. At the Store, a - // sequence number less than or equal to the last observed one (on a - // given key) incurs a transaction restart (if the request is - // transactional). - // - // This semantic could be adjusted in the future to provide idempotency - // for replays and re-issues. However, a side effect of providing this - // property is that reorder protection would no longer be provided by - // the counter, so ordering guarantees between requests within the same - // transaction would need to be strengthened elsewhere (e.g. by the - // transport layer). - for _, ru := range ba.Requests { - txn.mu.Proto.Sequence++ - oldHeader := ru.GetInner().Header() - oldHeader.Sequence = txn.mu.Proto.Sequence - ru.GetInner().SetHeader(oldHeader) - } - - // Clone the Txn's Proto so that future modifications can be made without - // worrying about synchronization. - newTxn := txn.mu.Proto.Clone() - ba.Txn = &newTxn - return nil - } - if pErr := lockedPrelude(); pErr != nil { - return nil, pErr - } - - // Send call through the DB. - requestTxnID := ba.Txn.ID - br, pErr := txn.db.sendUsingSender(ctx, ba, sender) - - // Lock for the entire response postlude. txn.mu.Lock() - defer txn.mu.Unlock() - - // 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:]...) - } - // Handle case where inserted begin txn confused an indexed error. - if pErr != nil && pErr.Index != nil { - idx := pErr.Index.Index - if idx == int32(firstWriteIdx) { - // An error was encountered on begin txn; disallow the indexing. - pErr.Index = nil - } else if idx > int32(firstWriteIdx) { - // An error was encountered after begin txn; decrement index. - pErr.SetErrorIndex(idx - 1) - } - } - } - if haveEndTxn { - if pErr == nil || !endTxnRequest.Commit { - // Finalize the transaction if either we sent a successful commit - // EndTxnRequest, or sent a rollback EndTxnRequest (regardless of - // if it succeeded). - txn.mu.finalized = true - } - } - - // 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 - // the request. - log.Fatalf(ctx, "retryable error for the wrong txn. "+ - "requestTxnID: %s, retryErr.TxnID: %s. retryErr: %s", - requestTxnID, retryErr.TxnID, retryErr) - } - txn.updateStateOnRetryableErrLocked(ctx, retryErr) - default: - if errTxn := pErr.GetTxn(); txn != nil { - txn.mu.Proto.Update(errTxn) - } - } - // 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 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 = txnError - } - - return nil, pErr - } - - if br != nil { - if br.Error != nil { - panic(roachpb.ErrorUnexpectedlySet(txn.mu.sender, br)) - } - - // 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 - // well; these errors have been handled above. - txn.mu.Proto.Update(br.Txn) + sender := txn.mu.sender + txn.mu.Unlock() + br, pErr := txn.db.sendUsingSender(ctx, ba, sender) + if pErr == nil { + return br, nil } - if elideEndTxn { - // 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 - // or expired. We will just keep the code for safety (see TestReacquireLeaseOnRestart). - if endTxnRequest.Deadline != nil { - if endTxnRequest.Deadline.Less(txn.mu.Proto.Timestamp) { - // NB: The returned error contains a pointer to txn.mu.Proto, but - // that's ok because we can't have concurrent operations going on while - // committing/aborting. - return nil, roachpb.NewErrorWithTxn(roachpb.NewTransactionStatusError( - "deadline exceeded before transaction finalization"), &txn.mu.Proto) - } - } - // This normally happens on the server and sent back in response - // headers, but this transaction was optimized away. The caller may - // still inspect the transaction struct, so we manually update it - // here to emulate a true transaction. - if endTxnRequest.Commit { - txn.mu.Proto.Status = roachpb.COMMITTED - } else { - txn.mu.Proto.Status = roachpb.ABORTED - } - } - return br, nil -} - -// firstWriteIndex returns the index of the first transactional write in the -// BatchRequest. Returns -1 if the batch has not intention to write. It also -// verifies that if an EndTransactionRequest is included, then it is the last -// request in the batch. -func firstWriteIndex(ba roachpb.BatchRequest) (int, *roachpb.Error) { - for i, ru := range ba.Requests { - args := ru.GetInner() - if i < len(ba.Requests)-1 /* if not last*/ { - if _, ok := args.(*roachpb.EndTransactionRequest); ok { - return -1, roachpb.NewErrorf("%s sent as non-terminal call", args.Method()) - } - } - if roachpb.IsTransactionWrite(args) { - return i, nil + if retryErr, ok := pErr.GetDetail().(*roachpb.HandledRetryableTxnError); ok { + if requestTxnID != retryErr.TxnID { + // KV should not return errors for transactions other than the one that sent + // the request. + log.Fatalf(ctx, "retryable error for the wrong txn. "+ + "requestTxnID: %s, retryErr.TxnID: %s. retryErr: %s", + requestTxnID, retryErr.TxnID, retryErr) } + txn.mu.Lock() + txn.resetDeadlineLocked() + txn.replaceSenderIfTxnAbortedLocked(ctx, retryErr, requestTxnID) + txn.mu.Unlock() } - return -1, nil + return br, pErr } // GetTxnCoordMeta returns the TxnCoordMeta information for this @@ -1260,14 +827,12 @@ func (txn *Txn) GetStrippedTxnCoordMeta() 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(ctx, meta) } -// UpdateStateOnRemoteRetryableErr updates the Txn, and the -// Transaction proto inside it, in response to an error encountered -// when running a request through the txn. Returns a handled retryable -// error on success or an error on failure. +// UpdateStateOnRemoteRetryableErr updates the txn in response to an error +// encountered when running a request through the txn. Returns a +// HandledRetryableError on success or another error on failure. func (txn *Txn) UpdateStateOnRemoteRetryableErr(ctx context.Context, pErr *roachpb.Error) error { txn.mu.Lock() defer txn.mu.Unlock() @@ -1276,93 +841,63 @@ func (txn *Txn) UpdateStateOnRemoteRetryableErr(ctx context.Context, pErr *roach log.Fatalf(ctx, "unexpected non-retryable error: %s", pErr) } - // Emulate the processing that the TxnCoordSender would have done on - // this error if it was experienced by the root transaction. - newTxn := roachpb.PrepareTransactionForRetry(ctx, pErr, txn.mu.UserPriority, txn.db.clock) - newErr := roachpb.NewHandledRetryableTxnError(pErr.Message, pErr.GetTxn().ID, newTxn) - // If the transaction has been reset since this request was sent, // ignore the error. - if newErr.TxnID != txn.mu.Proto.ID { + // Note that in case of TransactionAbortedError, pErr.GetTxn() returns the + // original transaction; a new transaction has not been created yet. + origTxnID := pErr.GetTxn().ID + if origTxnID != txn.mu.ID { return errors.Errorf("retryable error for an older version of txn (current: %s), err: %s", - txn.mu.Proto, pErr) + txn.mu.ID, pErr) } - txn.updateStateOnRetryableErrLocked(ctx, newErr) - return newErr -} + pErr = txn.mu.sender.UpdateStateOnRemoteRetryableErr(ctx, pErr) + txn.replaceSenderIfTxnAbortedLocked(ctx, pErr.GetDetail().(*roachpb.HandledRetryableTxnError), origTxnID) -// BumpEpochAfterConcurrentRetryErrorLocked bumps the transaction epoch manually -// and resets the transaction state away from txnError. This is meant to be used -// after synchronizing concurrent actors using a txn when a retryable error is -// seen. -// -// TODO(andrei): this should go away once we move to a TxnAttempt model. -func (txn *Txn) BumpEpochAfterConcurrentRetryErrorLocked() { - // Invalidate any writes performed by any workers after the retry updated - // the txn's proto but before we synchronized (some of these writes might - // have been performed at the wrong epoch). - txn.Proto().BumpEpoch() - - // The txn might have entered the txnError state after the epoch was bumped. - // Reset the state for the retry. - if txn.mu.state == txnError { - txn.mu.state = txnWriteInOldEpoch - } + return pErr.GoError() } -// updateStateOnRetryableErrLocked updates the Transaction proto inside txn. +// replaceSenderIfTxnAbortedLocked handles TransactionAbortedErrors, on which a new +// sender is created to replace the current one. // -// This method is safe to call repeatedly for requests from the same txn epoch. -// The first such update will move the Transaction forward (either create a new -// one or increment the epoch), and subsequent calls will be no-ops. -func (txn *Txn) updateStateOnRetryableErrLocked( - ctx context.Context, retryErr *roachpb.HandledRetryableTxnError, +// origTxnID is the id of the txn that generated retryErr. Note that this can be +// different from retryErr.Transaction - the latter might be a new transaction. +func (txn *Txn) replaceSenderIfTxnAbortedLocked( + ctx context.Context, retryErr *roachpb.HandledRetryableTxnError, origTxnID uuid.UUID, ) { - txn.resetDeadlineLocked() + // The proto inside the error has been prepared for use by the next + // transaction attempt. newTxn := &retryErr.Transaction - abortErr := txn.mu.Proto.ID != newTxn.ID - if abortErr { - // This means that the cause was a TransactionAbortedError; - // we've created a new Transaction that we're about to start using, so we - // save the old transaction ID so that concurrent requests or delayed - // responses that that throw errors know that these errors were sent to - // the correct transaction, even once the proto is reset. - txn.recordPreviousTxnIDLocked(txn.mu.Proto.ID) - - // Overwrite the transaction proto with the one to be used for the next - // 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. - meta := roachpb.MakeTxnCoordMeta(*newTxn) - txn.mu.sender = txn.db.factory.TransactionalSender(txn.typ, meta) - } 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 - // TxnCoordSender. However, we can't blindly replace the proto, because - // multiple retryable errors might come back here out-of-order. Instead, - // 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 - } + if txn.mu.ID == newTxn.ID { + // We don't need a new transaction as a result of this error. Nothing more + // to do. + return + } + if txn.mu.ID != origTxnID { + // The transaction has changed since the request that generated the error + // was sent. Nothing more to do. + log.VEventf(ctx, 2, "retriable error for old incarnation of the transaction") + return } + + // The ID changed, which means that the cause was a TransactionAbortedError; + // we've created a new Transaction that we're about to start using, so we save + // the old transaction ID so that concurrent requests or delayed responses + // that that throw errors know that these errors were sent to the correct + // transaction, even once the proto is reset. + txn.recordPreviousTxnIDLocked(txn.mu.ID) + txn.mu.ID = newTxn.ID + // Create a new txn sender. + meta := roachpb.MakeTxnCoordMeta(*newTxn) + txn.mu.sender = txn.db.factory.TransactionalSender(txn.typ, meta) } func (txn *Txn) recordPreviousTxnIDLocked(prevTxnID uuid.UUID) { if txn.mu.previousIDs == nil { txn.mu.previousIDs = make(map[uuid.UUID]struct{}) } - txn.mu.previousIDs[txn.mu.Proto.ID] = struct{}{} + txn.mu.previousIDs[txn.mu.ID] = struct{}{} } // SetFixedTimestamp makes the transaction run in an unusual way, at a "fixed @@ -1374,12 +909,7 @@ func (txn *Txn) recordPreviousTxnIDLocked(prevTxnID uuid.UUID) { // backups). This method must be called on every transaction retry (but note // that retries should be rare for read-only queries with no clock uncertainty). func (txn *Txn) SetFixedTimestamp(ctx context.Context, ts hlc.Timestamp) { - txn.mu.Lock() - txn.mu.Proto.Timestamp = ts - txn.mu.Proto.OrigTimestamp = ts - txn.mu.Proto.MaxTimestamp = ts - txn.mu.Proto.OrigTimestampWasObserved = true - txn.mu.Unlock() + txn.mu.sender.SetFixedTimestamp(ctx, ts) } // GenerateForcedRetryableError returns a HandledRetryableTxnError that will @@ -1390,22 +920,40 @@ func (txn *Txn) SetFixedTimestamp(ctx context.Context, ts hlc.Timestamp) { // bumped to the extent that txn.OrigTimestamp is racheted up to txn.Timestamp. // TODO(andrei): This method should take in an up-to-date timestamp, but // unfortunately its callers don't currently have that handy. -func (txn *Txn) GenerateForcedRetryableError(msg string) error { - txn.Proto().Restart(txn.UserPriority(), 0 /* upgradePriority */, txn.Proto().Timestamp) +func (txn *Txn) GenerateForcedRetryableError(ctx context.Context, msg string) error { + txn.mu.Lock() + defer txn.mu.Unlock() + now := txn.db.clock.Now() + txn.mu.sender.ManualRestart(ctx, txn.mu.userPriority, now) txn.resetDeadlineLocked() return roachpb.NewHandledRetryableTxnError( msg, - txn.ID(), + txn.mu.ID, roachpb.MakeTransaction( - txn.DebugName(), + txn.debugNameLocked(), nil, // baseKey - txn.UserPriority(), - txn.Isolation(), - txn.db.clock.Now(), + txn.mu.userPriority, + txn.mu.isolation, + now, txn.db.clock.MaxOffset().Nanoseconds(), )) } +// ManualRestart bumps the transactions epoch, and can upgrade the timestamp. +// An uninitialized timestamp can be passed to leave the timestamp alone. +// +// Used by the SQL layer which sometimes knows that a transaction will not be +// able to commit and prefers to restart early. +// It is also used after synchronizing concurrent actors using a txn when a +// retryable error is seen. +// TODO(andrei): this second use should go away once we move to a TxnAttempt +// model. +func (txn *Txn) ManualRestart(ctx context.Context, ts hlc.Timestamp) { + txn.mu.Lock() + defer txn.mu.Unlock() + txn.mu.sender.ManualRestart(ctx, txn.mu.userPriority, ts) +} + // IsSerializablePushAndRefreshNotPossible returns true if the transaction is // serializable, its timestamp has been pushed and there's no chance that // refreshing the read spans will succeed later (thus allowing the transaction @@ -1418,16 +966,7 @@ func (txn *Txn) GenerateForcedRetryableError(msg string) error { // operation (usually, but not exclusively, by a high-priority txn with // conflicting writes). func (txn *Txn) IsSerializablePushAndRefreshNotPossible() bool { - txn.mu.Lock() - defer txn.mu.Unlock() - - origTimestamp := txn.Proto().OrigTimestamp - origTimestamp.Forward(txn.Proto().RefreshedTimestamp) - isTxnPushed := txn.Proto().Timestamp != origTimestamp - // We check OrigTimestampWasObserved here because, if that's set, refreshing - // of reads is not performed. - return txn.Proto().Isolation == enginepb.SERIALIZABLE && - isTxnPushed && txn.mu.Proto.OrigTimestampWasObserved + return txn.mu.sender.IsSerializablePushAndRefreshNotPossible() } // Type returns the transaction's type. @@ -1435,6 +974,16 @@ func (txn *Txn) Type() TxnType { return txn.typ } +// Serialize returns a clone of the transaction's current proto. +// This is a nuclear option; generally client code shouldn't deal with protos. +// However, this is used by DistSQL for sending the transaction over the wire +// when it creates flows. +func (txn *Txn) Serialize() *roachpb.Transaction { + txn.mu.Lock() + defer txn.mu.Unlock() + return txn.mu.sender.SerializeTxn() +} + func (txn *Txn) deadline() *hlc.Timestamp { txn.mu.Lock() defer txn.mu.Unlock() diff --git a/pkg/internal/client/txn_test.go b/pkg/internal/client/txn_test.go index 9212fe1127d9..aac15e194930 100644 --- a/pkg/internal/client/txn_test.go +++ b/pkg/internal/client/txn_test.go @@ -15,7 +15,6 @@ package client import ( - "bytes" "context" "fmt" "reflect" @@ -24,21 +23,17 @@ import ( "time" "github.com/pkg/errors" - "golang.org/x/sync/errgroup" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/log" - "github.com/cockroachdb/cockroach/pkg/util/syncutil" "github.com/cockroachdb/cockroach/pkg/util/tracing" "github.com/cockroachdb/cockroach/pkg/util/uuid" ) var ( - testKey = roachpb.Key("a") - testTS = hlc.Timestamp{WallTime: 1, Logical: 1} testPutResp = roachpb.PutResponse{} ) @@ -86,68 +81,62 @@ 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 - } +) TxnSenderFactory { + return MakeMockTxnSenderFactory( + func( + ctx context.Context, txn *roachpb.Transaction, 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) - } - if roachpb.IsTransactionWrite(args) { - writing = true - } + var br *roachpb.BatchResponse + var pErr *roachpb.Error + + ba.Txn = txn + + 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) } - if args, ok := ba.GetArg(roachpb.EndTransaction); ok { - et := args.(*roachpb.EndTransactionRequest) + if roachpb.IsTransactionWrite(args) { writing = true - if et.Commit { - status = roachpb.COMMITTED - } else { - status = roachpb.ABORTED - } } - 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 - }, - ) - }) -} - -func testPut() roachpb.BatchRequest { - var ba roachpb.BatchRequest - ba.Timestamp = testTS - put := &roachpb.PutRequest{} - put.Key = testKey - ba.Add(put) - return ba + } + 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 + } + // Update the MockTxnSender's proto. + *txn = br.Txn.Clone() + } + return br, pErr + }) } func TestInitPut(t *testing.T) { @@ -169,50 +158,6 @@ func TestInitPut(t *testing.T) { } } -// TestTxnRequestTxnTimestamp verifies response txn timestamp is -// always upgraded on successive requests. -func TestTxnRequestTxnTimestamp(t *testing.T) { - defer leaktest.AfterTest(t)() - ba := testPut() - - testCases := []struct { - expRequestTS, responseTS hlc.Timestamp - }{ - {hlc.Timestamp{WallTime: 5, Logical: 0}, hlc.Timestamp{WallTime: 10, Logical: 0}}, - {hlc.Timestamp{WallTime: 10, Logical: 0}, hlc.Timestamp{WallTime: 10, Logical: 1}}, - {hlc.Timestamp{WallTime: 10, Logical: 1}, hlc.Timestamp{WallTime: 10, Logical: 0}}, - {hlc.Timestamp{WallTime: 10, Logical: 1}, hlc.Timestamp{WallTime: 20, Logical: 1}}, - {hlc.Timestamp{WallTime: 20, Logical: 1}, hlc.Timestamp{WallTime: 20, Logical: 1}}, - {hlc.Timestamp{WallTime: 20, Logical: 1}, hlc.Timestamp{WallTime: 19, Logical: 0}}, - {hlc.Timestamp{WallTime: 20, Logical: 1}, hlc.Timestamp{WallTime: 20, Logical: 1}}, - } - - manual := hlc.NewManualClock(testCases[0].expRequestTS.WallTime) - clock := hlc.NewClock(manual.UnixNano, time.Nanosecond) - var testIdx int - db := NewDB( - testutils.MakeAmbientCtx(), - newTestTxnFactory(func(ba roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { - test := testCases[testIdx] - if test.expRequestTS != ba.Txn.Timestamp { - return nil, roachpb.NewErrorf("%d: expected ts %s got %s", testIdx, test.expRequestTS, ba.Txn.Timestamp) - } - br := ba.CreateReply() - txnClone := ba.Txn.Clone() - br.Txn = &txnClone - br.Txn.Timestamp = test.responseTS - return br, nil - }), clock) - - txn := NewTxn(db, 0 /* gatewayNodeID */, RootTxn) - - for testIdx = range testCases { - if _, pErr := txn.Send(context.Background(), ba); pErr != nil { - t.Fatal(pErr) - } - } -} - // TestTransactionConfig verifies the proper unwrapping and // re-wrapping of the client's sender when starting a transaction. // Also verifies that the UserPriority is propagated to the @@ -236,165 +181,6 @@ func TestTransactionConfig(t *testing.T) { } } -// TestCommitReadOnlyTransaction verifies that transaction is -// committed but EndTransaction is not sent if only read-only -// operations were performed. -func TestCommitReadOnlyTransaction(t *testing.T) { - defer leaktest.AfterTest(t)() - clock := hlc.NewClock(hlc.UnixNano, time.Nanosecond) - var calls []roachpb.Method - db := NewDB( - testutils.MakeAmbientCtx(), - newTestTxnFactory(func(ba roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { - calls = append(calls, ba.Methods()...) - return ba.CreateReply(), nil - }), clock) - if err := db.Txn(context.TODO(), func(ctx context.Context, txn *Txn) error { - _, err := txn.Get(ctx, "a") - return err - }); err != nil { - t.Errorf("unexpected error on commit: %s", err) - } - expectedCalls := []roachpb.Method{roachpb.Get} - if !reflect.DeepEqual(expectedCalls, calls) { - t.Errorf("expected %s, got %s", expectedCalls, calls) - } -} - -// TestCommitReadOnlyTransactionExplicit verifies that a read-only -// transaction with an explicit EndTransaction call does not send -// that call. -func TestCommitReadOnlyTransactionExplicit(t *testing.T) { - defer leaktest.AfterTest(t)() - clock := hlc.NewClock(hlc.UnixNano, time.Nanosecond) - for _, withGet := range []bool{true, false} { - var calls []roachpb.Method - db := NewDB( - testutils.MakeAmbientCtx(), - newTestTxnFactory(func(ba roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { - calls = append(calls, ba.Methods()...) - return ba.CreateReply(), nil - }), clock) - if err := db.Txn(context.TODO(), func(ctx context.Context, txn *Txn) error { - b := txn.NewBatch() - if withGet { - b.Get("foo") - } - return txn.CommitInBatch(ctx, b) - }); err != nil { - t.Errorf("unexpected error on commit: %s", err) - } - expectedCalls := []roachpb.Method(nil) - if withGet { - expectedCalls = append(expectedCalls, roachpb.Get) - } - if !reflect.DeepEqual(expectedCalls, calls) { - t.Errorf("expected %s, got %s", expectedCalls, calls) - } - } -} - -// TestCommitMutatingTransaction verifies that transaction is committed -// upon successful invocation of the retryable func. -func TestCommitMutatingTransaction(t *testing.T) { - defer leaktest.AfterTest(t)() - clock := hlc.NewClock(hlc.UnixNano, time.Nanosecond) - var calls []roachpb.Method - db := NewDB( - testutils.MakeAmbientCtx(), - newTestTxnFactory(func(ba roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { - calls = append(calls, ba.Methods()...) - if bt, ok := ba.GetArg(roachpb.BeginTransaction); ok && !bt.Header().Key.Equal(roachpb.Key("a")) { - t.Errorf("expected begin transaction key to be \"a\"; got %s", bt.Header().Key) - } - if et, ok := ba.GetArg(roachpb.EndTransaction); ok && !et.(*roachpb.EndTransactionRequest).Commit { - t.Errorf("expected commit to be true") - } - return ba.CreateReply(), nil - }), clock) - - // Test all transactional write methods. - testArgs := []struct { - f func(ctx context.Context, txn *Txn) error - expMethod roachpb.Method - }{ - {func(ctx context.Context, txn *Txn) error { return txn.Put(ctx, "a", "b") }, roachpb.Put}, - {func(ctx context.Context, txn *Txn) error { return txn.CPut(ctx, "a", "b", nil) }, roachpb.ConditionalPut}, - {func(ctx context.Context, txn *Txn) error { - _, err := txn.Inc(ctx, "a", 1) - return err - }, roachpb.Increment}, - {func(ctx context.Context, txn *Txn) error { return txn.Del(ctx, "a") }, roachpb.Delete}, - {func(ctx context.Context, txn *Txn) error { return txn.DelRange(ctx, "a", "b") }, roachpb.DeleteRange}, - } - for i, test := range testArgs { - calls = []roachpb.Method{} - if err := db.Txn(context.TODO(), func(ctx context.Context, txn *Txn) error { - return test.f(ctx, txn) - }); err != nil { - t.Errorf("%d: unexpected error on commit: %s", i, err) - } - expectedCalls := []roachpb.Method{roachpb.BeginTransaction, test.expMethod, roachpb.EndTransaction} - if !reflect.DeepEqual(expectedCalls, calls) { - t.Errorf("%d: expected %s, got %s", i, expectedCalls, calls) - } - } -} - -// TestTxnInsertBeginTransaction verifies that a begin transaction -// request is inserted just before the first mutating command. -func TestTxnInsertBeginTransaction(t *testing.T) { - defer leaktest.AfterTest(t)() - clock := hlc.NewClock(hlc.UnixNano, time.Nanosecond) - var calls []roachpb.Method - db := NewDB( - testutils.MakeAmbientCtx(), - newTestTxnFactory(func(ba roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { - calls = append(calls, ba.Methods()...) - return ba.CreateReply(), nil - }), clock) - if err := db.Txn(context.TODO(), func(ctx context.Context, txn *Txn) error { - if _, err := txn.Get(ctx, "foo"); err != nil { - return err - } - return txn.Put(ctx, "a", "b") - }); err != nil { - t.Errorf("unexpected error on commit: %s", err) - } - expectedCalls := []roachpb.Method{roachpb.Get, roachpb.BeginTransaction, roachpb.Put, roachpb.EndTransaction} - if !reflect.DeepEqual(expectedCalls, calls) { - t.Errorf("expected %s, got %s", expectedCalls, calls) - } -} - -// TestBeginTransactionErrorIndex verifies that the error index is cleared -// when a BeginTransaction command causes an error. -func TestBeginTransactionErrorIndex(t *testing.T) { - defer leaktest.AfterTest(t)() - clock := hlc.NewClock(hlc.UnixNano, time.Nanosecond) - db := NewDB( - testutils.MakeAmbientCtx(), - newTestTxnFactory(func(ba roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { - pErr := roachpb.NewError(&roachpb.WriteIntentError{}) - pErr.SetErrorIndex(0) - return nil, pErr - }), clock) - _ = db.Txn(context.TODO(), func(ctx context.Context, txn *Txn) error { - b := txn.NewBatch() - b.Put("a", "b") - err := getOneErr(txn.Run(ctx, b), b) - pErr := b.MustPErr() - // Verify that the original error type is preserved, but the error index is unset. - if _, ok := pErr.GetDetail().(*roachpb.WriteIntentError); !ok { - t.Fatalf("unexpected error %s", pErr) - } - if pErr.Index != nil { - t.Errorf("error index must not be set, but got %d", pErr.Index) - } - return err - }) -} - // TestCommitTransactionOnce verifies that if the transaction is // ended explicitly in the retryable func, it is not automatically // ended a second time at completion of retryable func. @@ -420,130 +206,6 @@ func TestCommitTransactionOnce(t *testing.T) { } } -// TestAbortReadOnlyTransaction verifies that aborting a read-only -// transaction does not prompt an EndTransaction call. -func TestAbortReadOnlyTransaction(t *testing.T) { - defer leaktest.AfterTest(t)() - clock := hlc.NewClock(hlc.UnixNano, time.Nanosecond) - db := NewDB( - testutils.MakeAmbientCtx(), - newTestTxnFactory(func(ba roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { - if _, ok := ba.GetArg(roachpb.EndTransaction); ok { - t.Errorf("did not expect EndTransaction") - } - return ba.CreateReply(), nil - }), clock) - if err := db.Txn(context.TODO(), func(ctx context.Context, txn *Txn) error { - return errors.New("foo") - }); err == nil { - t.Error("expected error on abort") - } -} - -// TestEndWriteRestartReadOnlyTransaction verifies that if -// a transaction writes, then restarts and turns read-only, -// an explicit EndTransaction call is still sent if retry- -// able didn't, regardless of whether there is an error -// or not. -func TestEndWriteRestartReadOnlyTransaction(t *testing.T) { - defer leaktest.AfterTest(t)() - clock := hlc.NewClock(hlc.UnixNano, time.Nanosecond) - for _, success := range []bool{true, false} { - expCalls := []roachpb.Method{roachpb.BeginTransaction, roachpb.Put, roachpb.EndTransaction} - var calls []roachpb.Method - db := NewDB( - testutils.MakeAmbientCtx(), - newTestTxnFactory(func(ba roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { - calls = append(calls, ba.Methods()...) - return ba.CreateReply(), nil - }), clock) - ok := false - if err := db.Txn(context.TODO(), func(ctx context.Context, txn *Txn) error { - if !ok { - if err := txn.Put(ctx, "consider", "phlebas"); err != nil { - t.Fatal(err) - } - ok = true - // Return an immediate txn retry error. - // HACK ALERT: to do without a TxnCoordSender, we jump through hoops to - // get the retryable error expected by db.Txn(). - return roachpb.NewHandledRetryableTxnError( - "bogus retryable error", txn.Proto().ID, *txn.Proto()) - } - if !success { - return errors.New("aborting on purpose") - } - return nil - }); err == nil != success { - t.Errorf("expected error: %t, got error: %v", !success, err) - } - if !reflect.DeepEqual(expCalls, calls) { - t.Fatalf("expected %v, got %v", expCalls, calls) - } - } -} - -// TestTransactionKeyNotChangedInRestart verifies that if the transaction -// already has a key (we're in a restart), the key in the begin transaction -// request is not changed. -func TestTransactionKeyNotChangedInRestart(t *testing.T) { - defer leaktest.AfterTest(t)() - - clock := hlc.NewClock(hlc.UnixNano, time.Nanosecond) - attempt := 0 - keys := []string{"first", "second"} - db := NewDB( - testutils.MakeAmbientCtx(), - newTestTxnFactory(func(ba roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { - // Ignore the final EndTxnRequest. - if _, ok := ba.GetArg(roachpb.EndTransaction); ok { - return ba.CreateReply(), nil - } - - // Attempt 0 should have a BeginTxnRequest, and a PutRequest. - // Attempt 1 should have a PutRequest. - if attempt == 0 { - if _, ok := ba.GetArg(roachpb.BeginTransaction); !ok { - t.Fatalf("failed to find a begin transaction request: %v", ba) - } - } - if _, ok := ba.GetArg(roachpb.Put); !ok { - t.Fatalf("failed to find a put request: %v", ba) - } - - // In the first attempt, the transaction key is the key of the first write command. - // This key is retained between restarts, so we see the same key in the second attempt. - if expectedKey := []byte(keys[0]); !bytes.Equal(expectedKey, ba.Txn.Key) { - t.Fatalf("expected transaction key %v, got %v", expectedKey, ba.Txn.Key) - } - - if attempt == 0 { - // Abort the first attempt so that we need to retry with - // a new transaction proto. - // - // HACK ALERT: to do without a TxnCoordSender, we jump through hoops to - // get the retryable error expected by db.Txn(). - return nil, roachpb.NewError( - roachpb.NewHandledRetryableTxnError( - "bogus retryable error", ba.Txn.ID, *ba.Txn)) - } - return ba.CreateReply(), nil - }), clock) - - if err := db.Txn(context.TODO(), func(ctx context.Context, txn *Txn) error { - defer func() { attempt++ }() - b := txn.NewBatch() - b.Put(keys[attempt], "b") - return txn.Run(ctx, b) - }); err != nil { - t.Errorf("unexpected error on commit: %s", err) - } - minimumAttempts := 2 - if attempt < minimumAttempts { - t.Errorf("expected attempt count >= %d, got %d", minimumAttempts, attempt) - } -} - // TestAbortMutatingTransaction verifies that transaction is aborted // upon failed invocation of the retryable func. func TestAbortMutatingTransaction(t *testing.T) { @@ -568,7 +230,7 @@ func TestAbortMutatingTransaction(t *testing.T) { }); err == nil { t.Error("expected error on abort") } - expectedCalls := []roachpb.Method{roachpb.BeginTransaction, roachpb.Put, roachpb.EndTransaction} + expectedCalls := []roachpb.Method{roachpb.Put, roachpb.EndTransaction} if !reflect.DeepEqual(expectedCalls, calls) { t.Errorf("expected %s, got %s", expectedCalls, calls) } @@ -675,14 +337,14 @@ func TestTransactionStatus(t *testing.T) { if pErr := txn.CommitOrCleanup(context.Background()); pErr != nil { t.Fatal(pErr) } - if a, e := txn.Proto().Status, roachpb.COMMITTED; a != e { + if a, e := txn.Serialize().Status, roachpb.COMMITTED; a != e { t.Errorf("write: %t, commit: %t transaction expected to have status %q but had %q", write, commit, e, a) } } else { if pErr := txn.Rollback(context.Background()); pErr != nil { t.Fatal(pErr) } - if a, e := txn.Proto().Status, roachpb.ABORTED; a != e { + if a, e := txn.Serialize().Status, roachpb.ABORTED; a != e { t.Errorf("write: %t, commit: %t transaction expected to have status %q but had %q", write, commit, e, a) } } @@ -795,7 +457,12 @@ func TestUpdateDeadlineMaybe(t *testing.T) { clock := hlc.NewClock(mc.UnixNano, time.Nanosecond) db := NewDB( testutils.MakeAmbientCtx(), - TxnSenderFactoryFunc(func(_ TxnType) TxnSender { return nil }), clock) + MakeMockTxnSenderFactory( + func(context.Context, *roachpb.Transaction, roachpb.BatchRequest, + ) (*roachpb.BatchResponse, *roachpb.Error) { + return nil, nil + }), + clock) txn := NewTxn(db, 0 /* gatewayNodeID */, RootTxn) if txn.deadline() != nil { @@ -826,113 +493,3 @@ func TestUpdateDeadlineMaybe(t *testing.T) { t.Errorf("unexpected deadline: %s", d) } } - -// TestSequenceNumbers verifies Requests are given sequence numbers and that -// they are incremented on successive commands. -func TestSequenceNumbers(t *testing.T) { - defer leaktest.AfterTest(t)() - clock := hlc.NewClock(hlc.UnixNano, time.Nanosecond) - - expSequence := int32(0) - db := NewDB( - testutils.MakeAmbientCtx(), - newTestTxnFactory(func(ba roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { - for _, ru := range ba.Requests { - args := ru.GetInner() - expSequence++ - if seq := args.Header().Sequence; expSequence != seq { - t.Errorf("expected Request sequence %d; got %d", expSequence, seq) - } - } - if expSequence != ba.Txn.Sequence { - t.Errorf("expected header sequence %d; got %d", expSequence, ba.Txn.Sequence) - } - br := ba.CreateReply() - br.Txn = ba.Txn - return br, nil - }), clock) - - txn := NewTxn(db, 0 /* gatewayNodeID */, RootTxn) - for i := 0; i < 5; i++ { - var ba roachpb.BatchRequest - for j := 0; j < i; j++ { - ba.Add(roachpb.NewPut(roachpb.Key("a"), roachpb.MakeValueFromString("foo")).(*roachpb.PutRequest)) - } - if _, pErr := txn.Send(context.Background(), ba); pErr != nil { - t.Fatal(pErr) - } - } -} - -// TestConcurrentTxnRequests verifies that multiple requests can be executed on -// a transaction at the same time from multiple goroutines. It makes sure that -// exactly one BeginTxnRequest and one EndTxnRequest are sent. -func TestConcurrentTxnRequests(t *testing.T) { - defer leaktest.AfterTest(t)() - - clock := hlc.NewClock(hlc.UnixNano, time.Nanosecond) - var callCountsMu syncutil.Mutex - callCounts := make(map[roachpb.Method]int) - db := NewDB( - testutils.MakeAmbientCtx(), - newTestTxnFactory(func(ba roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { - callCountsMu.Lock() - for _, m := range ba.Methods() { - callCounts[m]++ - } - callCountsMu.Unlock() - return ba.CreateReply(), nil - }), clock) - - const keys = "abcdefghijklmnopqrstuvwxyz" - const value = "value" - if err := db.Txn(context.TODO(), func(ctx context.Context, txn *Txn) error { - g, gCtx := errgroup.WithContext(ctx) - for _, keyChar := range keys { - key := string(keyChar) - g.Go(func() error { - return txn.Put(gCtx, key, value) - }) - } - return g.Wait() - }); err != nil { - t.Fatal(err) - } - - expectedCallCounts := map[roachpb.Method]int{ - roachpb.BeginTransaction: 1, - roachpb.Put: 26, - roachpb.EndTransaction: 1, - } - if !reflect.DeepEqual(expectedCallCounts, callCounts) { - t.Errorf("expected %v, got %v", expectedCallCounts, callCounts) - } -} - -// Test that a txn's anchor is set to the first write key in batches mixing -// reads with writes. -func TestAnchorKey(t *testing.T) { - defer leaktest.AfterTest(t)() - - ctx := context.Background() - clock := hlc.NewClock(hlc.UnixNano, time.Nanosecond) - key1 := roachpb.Key("a") - key2 := roachpb.Key("b") - db := NewDB( - testutils.MakeAmbientCtx(), - newTestTxnFactory(func(ba roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { - if !roachpb.Key(ba.Txn.Key).Equal(key2) { - t.Fatalf("expected anchor %q, got %q", key2, ba.Txn.Key) - } - return ba.CreateReply(), nil - }), clock) - - if err := db.Txn(ctx, func(ctx context.Context, txn *Txn) error { - ba := txn.NewBatch() - ba.Get(key1) - ba.Put(key2, "val") - return txn.Run(ctx, ba) - }); err != nil { - t.Fatal(err) - } -} diff --git a/pkg/kv/dist_sender.go b/pkg/kv/dist_sender.go index 544dfed7eb8a..ea88c310d152 100644 --- a/pkg/kv/dist_sender.go +++ b/pkg/kv/dist_sender.go @@ -607,7 +607,7 @@ func (ds *DistSender) Send( } ctx = ds.AnnotateCtx(ctx) - ctx, cleanup := tracing.EnsureChildSpan(ctx, ds.AmbientContext.Tracer, "dist sender") + ctx, cleanup := tracing.EnsureChildSpan(ctx, ds.AmbientContext.Tracer, "dist sender send") defer cleanup() var rplChunks []*roachpb.BatchResponse diff --git a/pkg/kv/dist_sender_server_test.go b/pkg/kv/dist_sender_server_test.go index e7181560dcf8..24d367db9e1e 100644 --- a/pkg/kv/dist_sender_server_test.go +++ b/pkg/kv/dist_sender_server_test.go @@ -1675,14 +1675,15 @@ func TestPropagateTxnOnError(t *testing.T) { epoch := 0 if err := db.Txn(ctx, func(ctx context.Context, txn *client.Txn) error { epoch++ + proto := txn.Serialize() if epoch >= 2 { // Writing must be true since we ran the BeginTransaction command. - if !txn.Proto().Writing { + if !proto.Writing { t.Errorf("unexpected non-writing txn") } } else { // Writing must be false since we haven't run any write command. - if txn.Proto().Writing { + if proto.Writing { t.Errorf("unexpected writing txn") } } diff --git a/pkg/kv/testing_knobs.go b/pkg/kv/testing_knobs.go index 5c01c54e96a2..58254f1537ad 100644 --- a/pkg/kv/testing_knobs.go +++ b/pkg/kv/testing_knobs.go @@ -25,6 +25,7 @@ type ClientTestingKnobs struct { // The maximum number of times a txn will attempt to refresh its // spans for a single transactional batch. + // 0 means use a default. -1 means disable refresh. MaxTxnRefreshAttempts int } diff --git a/pkg/kv/txn_coord_sender.go b/pkg/kv/txn_coord_sender.go index d3ee4e2d496b..c014d7d94bcb 100644 --- a/pkg/kv/txn_coord_sender.go +++ b/pkg/kv/txn_coord_sender.go @@ -19,13 +19,14 @@ import ( "sync" "time" - opentracing "github.com/opentracing/opentracing-go" "github.com/pkg/errors" "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/internal/client" + "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/engine/enginepb" "github.com/cockroachdb/cockroach/pkg/util/duration" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/log" @@ -37,24 +38,31 @@ import ( ) const ( - opTxnCoordSender = "txn coordinator" + opTxnCoordSender = "txn coordinator send" ) -// txnCoordState represents the state of the transaction coordinator. -// It is an intermediate state which indicates we've finished the -// transaction at the coordinator level and it's no longer legitimate -// for sending requests, even though we don't yet know for sure that -// the transaction record has been aborted / committed. -type txnCoordState int +// txnState represents states relating to whether Begin/EndTxn requests need to +// be sent. +//go:generate stringer -type=txnState +type txnState int const ( - _ txnCoordState = iota - // done indicates the transaction has been completed via end - // transaction and can no longer be used. - done - // aborted indicates the transaction was aborted. All requests other than - // rollbacks are rejected. - aborted + // txnPending is the normal state for ongoing transactions. + txnPending txnState = iota + + // txnError means that a batch encountered a non-retriable error. Further + // batches except EndTransaction(commit=false) will be rejected. + txnError + + // txnFinalized means that an EndTransaction(commit=true) has been executed + // successfully, or an EndTransaction(commit=false) was sent - regardless of + // whether it executed successfully or not. Further batches except + // EndTransaction(commit=false) will be rejected; a second rollback is allowed + // in case the first one fails. + // TODO(andrei): we'd probably benefit from splitting this state into at least + // two - transaction definitely cleaned up, and transaction potentially + // cleaned up. + txnFinalized ) // A TxnCoordSender is the production implementation of client.TxnSender. It is @@ -75,10 +83,6 @@ const ( // case of TransactionAbortedErrors, cleaning up the transaction (in this case, // the client.Txn is expected to create a new TxnCoordSender instance // transparently for the higher-level client). -// - Ensures atomic execution for non-transactional (write) batches by transparently -// wrapping them in transactions when the DistSender is forced to split them for -// multiple ranges. For this reason, generally even non-transactional batches -// need to be sent through a TxnCoordSender. // // Since it is stateful, the TxnCoordSender needs to understand when a // transaction is "finished" and the state can be destroyed. As such there's a @@ -87,8 +91,8 @@ const ( // client, the TxnCoordSender considers the transactions completed in the // following situations: // - A batch containing an EndTransactions (commit or rollback) succeeds. -// - A batch containing an EndTransaction(commit=false) succeeds or fails. I.e. -// nothing is expected to follow a rollback attempt. +// - A batch containing an EndTransaction(commit=false) succeeds or fails. Only +// more rollback attempts can follow a rollback attempt. // - A batch returns a TransactionAbortedError. As mentioned above, the client // is expected to create a new TxnCoordSender for the next transaction attempt. // @@ -100,29 +104,29 @@ type TxnCoordSender struct { mu struct { syncutil.Mutex - // txn is a copy of the transaction record, updated with each request. + txnState txnState + + // active is set whenever the transaction has sent any requests. + active bool + + // closed is set once this transaction has either committed or rolled back + // (including when the heartbeat loop cleans it up asynchronously). If the + // client sends anything other than a rollback, it will get an error + // (a retryable TransactionAbortedError in case of the async abort). + closed bool + + // systemConfigTrigger is set to true when modifying keys from the + // SystemConfig span. This sets the SystemConfigTrigger on + // EndTransactionRequest. + systemConfigTrigger bool + + // txn is the Transaction proto attached to all the requests and updated on + // all the responses. txn roachpb.Transaction - // hbRunning is set if the TxnCoordSender has a heartbeat loop running for - // the transaction record. - hbRunning bool - - // commandCount indicates how many requests have been sent through - // this transaction. Reset on retryable txn errors. - // TODO(andrei): let's get rid of this. It should be maintained - // in the SQL level. - commandCount int32 - // firstUpdateNanos is the wall time at which the transaction was - // instantiated. - firstUpdateNanos int64 - // txnEnd is closed when the transaction is aborted or committed, - // terminating the heartbeat loop. - txnEnd chan struct{} - // state indicates the state of the transaction coordinator, which - // may briefly diverge from the state of the transaction record if - // the coordinator is aborted after a failed heartbeat, but before - // we've gotten a response with the updated transaction state. - state txnCoordState + // userPriority is the txn's priority. Used when restarting the transaction. + userPriority roachpb.UserPriority + // onFinishFn is a closure invoked when state changes to done or aborted. onFinishFn func(error) } @@ -137,11 +141,14 @@ type TxnCoordSender struct { // is embedded in the interceptorAlloc struct, so the entire stack is // allocated together with TxnCoordSender without any additional heap // allocations necessary. - interceptorStack [3]txnInterceptor + interceptorStack [6]txnInterceptor interceptorAlloc struct { + txnHeartbeat txnIntentCollector txnPipeliner txnSpanRefresher + txnSeqNumAllocator + txnMetrics txnLockGatekeeper // not in interceptorStack array. } @@ -194,8 +201,13 @@ type txnInterceptor interface { // closeLocked closes the interceptor. It is called when the TxnCoordSender // shuts down due to either a txn commit or a txn abort. - // TODO(nvanbenschoten): this won't be used until txnPipeliner. Remove - // if that never goes in. + // + // This method can be called multiple times (e.g. if the txn is aborted by the + // heartbeat loop and then upon a client rollback); implementers beware. + // + // Note that EndTransaction(commit=false) requests can still be sent (via the + // lockedSender interface) after this is called, and they're expected to be + // forwarded along. The idea for this method is to stop background tasks. closeLocked() } @@ -213,6 +225,8 @@ type txnLockGatekeeper struct { func (gs *txnLockGatekeeper) SendLocked( ctx context.Context, ba roachpb.BatchRequest, ) (*roachpb.BatchResponse, *roachpb.Error) { + // Note the funky locking here: we unlock for the duration of the call and the + // lock again. gs.mu.Unlock() defer gs.mu.Lock() return gs.wrapped.Send(ctx, ba) @@ -384,10 +398,12 @@ func NewTxnCoordSenderFactory( func (tcf *TxnCoordSenderFactory) TransactionalSender( typ client.TxnType, meta roachpb.TxnCoordMeta, ) client.TxnSender { + meta.Txn.AssertInitialized(context.TODO()) tcs := &TxnCoordSender{ typ: typ, TxnCoordSenderFactory: tcf, } + tcs.mu.txnState = txnPending // Create a stack of request/response interceptors. All of the objects in // this stack are pre-allocated on the TxnCoordSender struct, so this just @@ -398,6 +414,16 @@ func (tcf *TxnCoordSenderFactory) TransactionalSender( if ds, ok := tcf.wrapped.(*DistSender); ok { ri = NewRangeIterator(ds) } + tcs.interceptorAlloc.txnHeartbeat.init( + &tcs.mu.Mutex, + &tcs.mu.txn, + tcs.clock, + tcs.heartbeatInterval, + &tcs.metrics, + tcs.stopper, + tcs.cleanupTxnLocked, + ) + tcs.interceptorAlloc.txnMetrics.init(&tcs.mu.txn, tcs.clock, &tcs.metrics) tcs.interceptorAlloc.txnIntentCollector = txnIntentCollector{ st: tcf.st, ri: ri, @@ -416,13 +442,20 @@ func (tcf *TxnCoordSenderFactory) TransactionalSender( autoRetryCounter: tcs.metrics.AutoRetries, } tcs.interceptorAlloc.txnLockGatekeeper = txnLockGatekeeper{ - mu: &tcs.mu, wrapped: tcs.wrapped, + mu: &tcs.mu, } tcs.interceptorStack = [...]txnInterceptor{ + &tcs.interceptorAlloc.txnHeartbeat, + // The seq num allocator is the below the txnHeartbeat so that it sees the + // BeginTransaction prepended by that interceptor. (An alternative would be + // to not assign seq nums to BeginTransaction; it doesn't need it.) + // Note though that it skips assigning seq nums to heartbeats. + &tcs.interceptorAlloc.txnSeqNumAllocator, &tcs.interceptorAlloc.txnIntentCollector, &tcs.interceptorAlloc.txnPipeliner, &tcs.interceptorAlloc.txnSpanRefresher, + &tcs.interceptorAlloc.txnMetrics, } for i, reqInt := range tcs.interceptorStack { if i < len(tcs.interceptorStack)-1 { @@ -453,7 +486,6 @@ func (tc *TxnCoordSender) GetMeta() roachpb.TxnCoordMeta { // Copy mutable state so access is safe for the caller. var meta roachpb.TxnCoordMeta meta.Txn = tc.mu.txn.Clone() - meta.CommandCount = tc.mu.commandCount for _, reqInt := range tc.interceptorStack { reqInt.populateMetaLocked(&meta) } @@ -478,7 +510,6 @@ func (tc *TxnCoordSender) AugmentMeta(ctx context.Context, meta roachpb.TxnCoord func (tc *TxnCoordSender) augmentMetaLocked(meta roachpb.TxnCoordMeta) { tc.mu.txn.Update(&meta.Txn) - tc.mu.commandCount += meta.CommandCount for _, reqInt := range tc.interceptorStack { reqInt.augmentMetaLocked(meta) } @@ -492,475 +523,288 @@ func (tc *TxnCoordSender) OnFinish(onFinishFn func(error)) { } // DisablePipelining is part of the client.TxnSender interface. -func (tc *TxnCoordSender) DisablePipelining() { +func (tc *TxnCoordSender) DisablePipelining() error { + tc.mu.Lock() + defer tc.mu.Unlock() + if tc.mu.active { + return errors.Errorf("cannot disable pipelining on a running transaction") + } tc.interceptorAlloc.txnPipeliner.disabled = true + return nil } -// 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. +// Send is part of the client.TxnSender interface. func (tc *TxnCoordSender) Send( ctx context.Context, ba roachpb.BatchRequest, ) (*roachpb.BatchResponse, *roachpb.Error) { + // NOTE: The locking here is unusual. Although it might look like it, we are + // NOT holding the lock continuously for the duration of the Send. We lock + // here, and unlock at the botton of the interceptor stack, in the + // txnLockGatekeeper. The we lock again in that interceptor when the response + // comes, and unlock again in the defer below. + tc.mu.Lock() + defer tc.mu.Unlock() - if ba.Txn == nil { - log.Fatalf(ctx, "can't send non-transactional request through a TxnCoordSender: %s", ba) + if pErr := tc.maybeRejectClientLocked(ctx, &ba); pErr != nil { + return nil, pErr } - ctx = tc.AnnotateCtx(ctx) + startNs := tc.clock.PhysicalNow() - // Start new or pick up active trace. From here on, there's always an active - // Trace, though its overhead is small unless it's sampled. - sp := opentracing.SpanFromContext(ctx) - if sp == nil { - sp = tc.AmbientContext.Tracer.StartSpan(opTxnCoordSender) - defer sp.Finish() - ctx = opentracing.ContextWithSpan(ctx, sp) + if _, ok := ba.GetArg(roachpb.BeginTransaction); ok { + return nil, roachpb.NewErrorf("BeginTransaction added before the TxnCoordSender") } - startNS := tc.clock.PhysicalNow() - - tc.mu.Lock() - defer tc.mu.Unlock() + ctx, sp := tc.AnnotateCtxWithSpan(ctx, opTxnCoordSender) + defer sp.Finish() + // Associate the txnID with the trace. if tc.mu.txn.ID == (uuid.UUID{}) { log.Fatalf(ctx, "cannot send transactional request through unbound TxnCoordSender") } - - // Update our copy of the transaction. It will be further updated with the - // result in updateStateLocked(). - // Besides seeming like the sane thing to do, updating the txn here assures - // that, if the heartbeat loop is started, it's assured to have a transaction - // anchor key to operate on. - tc.mu.txn.Update(ba.Txn) - - ctx = log.WithLogTag(ctx, "txn", uuid.ShortStringer(ba.Txn.ID)) + sp.SetBaggageItem("txnID", tc.mu.txn.ID.String()) + ctx = log.WithLogTag(ctx, "txn", uuid.ShortStringer(tc.mu.txn.ID)) if log.V(2) { - ctx = log.WithLogTag(ctx, "ts", ba.Txn.Timestamp) + ctx = log.WithLogTag(ctx, "ts", tc.mu.txn.Timestamp) } - // If this request is part of a transaction... - if err := tc.validateTxnForBatchLocked(ctx, &ba); err != nil { - return nil, roachpb.NewError(err) + // It doesn't make sense to use inconsistent reads in a transaction. However, + // we still need to accept it as a parameter for this to compile. + if ba.ReadConsistency != roachpb.CONSISTENT { + return nil, roachpb.NewErrorf("cannot use %s ReadConsistency in txn", + ba.ReadConsistency) } - // Associate the txnID with the trace. - txnIDStr := ba.Txn.ID.String() - sp.SetBaggageItem("txnID", txnIDStr) - - // If there's a BeginTransaction, we need to start the heartbeat loop. - // Perhaps surprisingly, this needs to be done even if the batch has both - // a BeginTransaction and an EndTransaction. Although on batch success the - // heartbeat loop will be stopped right away, on retriable errors we need the - // heartbeat loop: the intents and txn record should be kept in place just - // like for non-1PC txns. - // - // Note that we don't start the heartbeat loop if the loop is already running. - // That can happen because of send BeginTransaction again after retriable - // errors. - _, hasBegin := ba.GetArg(roachpb.BeginTransaction) - if hasBegin && (tc.mu.txnEnd == nil) { - if err := tc.startHeartbeatLoopLocked(ctx); err != nil { - return nil, roachpb.NewError(err) - } + lastIndex := len(ba.Requests) - 1 + if lastIndex < 0 { + return nil, nil } - // 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.txn.Name = ba.Txn.Name - tc.mu.txn.Isolation = ba.Txn.Isolation - tc.mu.txn.Priority = ba.Txn.Priority - - if pErr := tc.maybeRejectClientLocked(ctx, ba.Txn.ID, &ba); pErr != nil { - return nil, pErr + if !tc.mu.active { + tc.mu.active = true + // If we haven't generate a transaction priority before, do it now. + // + // NOTE(andrei): Unfortunately, as of August 2018, txn.Priority == 0 is also + // true when the priority has been generated from MinUserPriority. In that + // case, we'll generate it again. + if tc.mu.txn.Priority == 0 { + tc.mu.txn.Priority = roachpb.MakePriority(tc.mu.userPriority) + } } - - // Update the command count. - tc.mu.commandCount += int32(len(ba.Requests)) + // Clone the Txn's Proto so that future modifications can be made without + // worrying about synchronization. + newTxn := tc.mu.txn.Clone() + ba.Txn = &newTxn // Send the command through the txnInterceptor stack. br, pErr := tc.interceptorStack[0].SendLocked(ctx, ba) - pErr = tc.updateStateLocked(ctx, startNS, ba, br, pErr) + + pErr = tc.updateStateLocked(ctx, startNs, ba, br, pErr) + + // If we succeeded to commit, or we attempted to rollback, we move to + // txnFinalized. + if req, ok := ba.GetArg(roachpb.EndTransaction); ok { + etReq := req.(*roachpb.EndTransactionRequest) + if etReq.Commit { + if pErr == nil { + tc.mu.txnState = txnFinalized + tc.cleanupTxnLocked(ctx) + tc.maybeSleepForLinearizable(ctx, br, startNs) + } + } else { + // Rollbacks always move us to txnFinalized. + tc.mu.txnState = txnFinalized + tc.cleanupTxnLocked(ctx) + } + } + + // Move to the error state on non-retriable errors. if pErr != nil { - log.VEventf(ctx, 2, "error: %s", pErr) + log.VEventf(ctx, 2, "failed batch: %s", pErr) + var retriable bool + // 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 pErr.TransactionRestart != roachpb.TransactionRestart_NONE { + retriable = true + if tc.typ == client.RootTxn { + log.Fatalf(ctx, + "unexpected retryable error at the client.Txn level: (%T) %s", + pErr.GetDetail(), pErr) + } + } else if _, ok := pErr.GetDetail().(*roachpb.HandledRetryableTxnError); ok { + retriable = true + } + + if !retriable { + tc.mu.txnState = txnError + } + return nil, pErr } - if br.Txn == nil { - return br, nil + if br != nil && br.Error != nil { + panic(roachpb.ErrorUnexpectedlySet(nil /* culprit */, br)) } - if _, ok := ba.GetArg(roachpb.EndTransaction); !ok { - return br, nil - } - // If the linearizable flag is set, we want to make sure that all the - // clocks in the system are past the commit timestamp of the transaction. - // This is guaranteed if either - the commit timestamp is MaxOffset behind - // startNS - MaxOffset ns were spent in this function when returning to the - // client. Below we choose the option that involves less waiting, which is - // likely the first one unless a transaction commits with an odd timestamp. - // - // Can't use linearizable mode with clockless reads since in that case we - // don't know how long to sleep - could be forever! - if tsNS := br.Txn.Timestamp.WallTime; startNS > tsNS { - startNS = tsNS + + return br, nil +} + +// maybeSleepForLinearizable sleeps if the linearizable flag is set. We want to +// make sure that all the clocks in the system are past the commit timestamp of +// the transaction. This is guaranteed if either: +// - the commit timestamp is MaxOffset behind startNs +// - MaxOffset ns were spent in this function when returning to the +// client. +// Below we choose the option that involves less waiting, which is likely the +// first one unless a transaction commits with an odd timestamp. +// +// Can't use linearizable mode with clockless reads since in that case we don't +// know how long to sleep - could be forever! +func (tc *TxnCoordSender) maybeSleepForLinearizable( + ctx context.Context, br *roachpb.BatchResponse, startNs int64, +) { + if tsNS := br.Txn.Timestamp.WallTime; startNs > tsNS { + startNs = tsNS } maxOffset := tc.clock.MaxOffset() sleepNS := maxOffset - - time.Duration(tc.clock.PhysicalNow()-startNS) + time.Duration(tc.clock.PhysicalNow()-startNs) if maxOffset != timeutil.ClocklessMaxOffset && tc.linearizable && sleepNS > 0 { - defer func() { - if log.V(1) { - log.Infof(ctx, "%v: waiting %s on EndTransaction for linearizability", br.Txn.Short(), duration.Truncate(sleepNS, time.Millisecond)) - } - time.Sleep(sleepNS) - }() - } - if br.Txn.Status != roachpb.PENDING { - tc.mu.txn = br.Txn.Clone() - _, hasBT := ba.GetArg(roachpb.BeginTransaction) - onePC := br.Txn.Status == roachpb.COMMITTED && hasBT - if onePC { - tc.metrics.Commits1PC.Inc(1) - } - tc.cleanupTxnLocked(ctx, done) + // TODO(andrei): perhaps we shouldn't sleep with the lock held. + log.VEventf(ctx, 2, "%v: waiting %s on EndTransaction for linearizability", + br.Txn.Short(), duration.Truncate(sleepNS, time.Millisecond)) + time.Sleep(sleepNS) } - return br, nil } -// maybeRejectClientLocked checks whether the (transactional) request is in a -// state that prevents it from continuing, such as the coordinator having -// considered the client abandoned, or a heartbeat having reported an error. +// maybeRejectClientLocked checks whether the transaction is in a state that +// prevents it from continuing, such as the heartbeat having detected the +// transaction to have been aborted. func (tc *TxnCoordSender) maybeRejectClientLocked( - ctx context.Context, txnID uuid.UUID, ba *roachpb.BatchRequest, + ctx context.Context, ba *roachpb.BatchRequest, ) *roachpb.Error { - // Check whether the transaction is still tracked and has a chance of - // completing. It's possible that the coordinator learns about the - // transaction having terminated from a heartbeat, and GC queue correctness - // (along with common sense) mandates that we don't let the client - // continue. - switch { - case tc.mu.state == aborted: - fallthrough - case tc.mu.txn.Status == roachpb.ABORTED: + if singleRollback := ba != nil && + ba.IsSingleEndTransactionRequest() && + !ba.Requests[0].GetInner().(*roachpb.EndTransactionRequest).Commit; singleRollback { // As a special case, we allow rollbacks to be sent at any time. Any - // rollback attempt moves the TxnCoordSender state to aborted, but higher + // rollback attempt moves the TxnCoordSender state to txnFinalized, but higher // layers are free to retry rollbacks if they want (and they do, for // example, when the context was canceled while txn.Rollback() was running). - singleAbort := ba != nil && - ba.IsSingleEndTransactionRequest() && - !ba.Requests[0].GetInner().(*roachpb.EndTransactionRequest).Commit - if singleAbort { - return nil - } + return nil + } + if tc.mu.txnState == txnFinalized { + return roachpb.NewErrorWithTxn( + roachpb.NewTransactionStatusError( + "client already committed or rolled back the transaction"), + &tc.mu.txn) + } + if tc.mu.txnState == txnError { + return roachpb.NewError(&roachpb.TxnAlreadyEncounteredErrorError{}) + } + if tc.mu.txn.Status == roachpb.ABORTED { abortedErr := roachpb.NewErrorWithTxn(roachpb.NewTransactionAbortedError(), &tc.mu.txn) - // TODO(andrei): figure out a UserPriority to use here. newTxn := roachpb.PrepareTransactionForRetry( ctx, abortedErr, // priority is not used for aborted errors roachpb.NormalUserPriority, tc.clock) return roachpb.NewError(roachpb.NewHandledRetryableTxnError( - abortedErr.Message, txnID, newTxn)) - - case tc.mu.txn.Status == roachpb.COMMITTED: - return roachpb.NewErrorWithTxn(roachpb.NewTransactionStatusError( - "transaction is already committed"), &tc.mu.txn) - - default: - return nil + abortedErr.Message, tc.mu.txn.ID, newTxn)) } -} -// validateTxnForBatchLocked validates properties of a txn specified on -// a request. The transaction is expected to be initialized by the time -// it reaches the TxnCoordSender. -func (tc *TxnCoordSender) validateTxnForBatchLocked( - ctx context.Context, ba *roachpb.BatchRequest, -) error { - if len(ba.Requests) == 0 { - return errors.Errorf("empty batch with txn") - } - ba.Txn.AssertInitialized(ctx) - - var haveBeginTxn bool - for _, req := range ba.Requests { - args := req.GetInner() - if _, ok := args.(*roachpb.BeginTransactionRequest); ok { - 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) - } - haveBeginTxn = true - } + if tc.mu.txn.Status != roachpb.PENDING { + log.Fatalf(ctx, "unexpected txn state: %s", tc.mu.txn) } return nil } -// cleanupTxnLocked is called when a transaction ends. The heartbeat -// goroutine is signaled to stop. The TxnCoordSender's state is set to `state`. -// Future Send() calls are rejected. -func (tc *TxnCoordSender) cleanupTxnLocked(ctx context.Context, state txnCoordState) { - tc.mu.state = state +// cleanupTxnLocked calls onFinishFn and closes all the interceptors. +func (tc *TxnCoordSender) cleanupTxnLocked(ctx context.Context) { + if tc.mu.closed { + return + } + tc.mu.closed = true if tc.mu.onFinishFn != nil { - // rejectErr is guaranteed to be non-nil because state is done or - // aborted on cleanup. - rejectErr := tc.maybeRejectClientLocked(ctx, tc.mu.txn.ID, nil /* ba */).GetDetail() + rejectErr := tc.maybeRejectClientLocked(ctx, nil /* ba */).GetDetail() if rejectErr == nil { - log.Fatalf(ctx, "expected non-nil rejectErr on txn coord state %v", state) + log.Fatal(ctx, "expected non-nil rejectErr") } tc.mu.onFinishFn(rejectErr) + tc.mu.onFinishFn = nil } - - // The heartbeat might've already removed the record. Or we may have already - // closed txnEnd but we are racing with the heartbeat cleanup. - if tc.mu.txnEnd == nil { - return - } - // Trigger heartbeat shutdown. - log.VEvent(ctx, 2, "coordinator stops") - close(tc.mu.txnEnd) - tc.mu.txnEnd = nil // Close each interceptor. for _, reqInt := range tc.interceptorStack { reqInt.closeLocked() } } -// finalTxnStatsLocked collects a transaction's final statistics. Returns -// the duration, restarts, and finalized txn status. -func (tc *TxnCoordSender) finalTxnStatsLocked() (duration, restarts int64, status roachpb.TransactionStatus) { - duration = tc.clock.PhysicalNow() - tc.mu.firstUpdateNanos - restarts = int64(tc.mu.txn.Epoch) - status = tc.mu.txn.Status - return duration, restarts, status -} - -// heartbeatLoop periodically sends a HeartbeatTxn RPC to an extant transaction, -// stopping in the event the transaction is aborted or committed after -// attempting to resolve the intents. When the heartbeat stops, the transaction -// stats are updated based on its final disposition. -func (tc *TxnCoordSender) heartbeatLoop(ctx context.Context) { - var tickChan <-chan time.Time - { - ticker := time.NewTicker(tc.heartbeatInterval) - tickChan = ticker.C - defer ticker.Stop() - } - - defer func() { - tc.mu.Lock() - if tc.mu.txnEnd != nil { - tc.mu.txnEnd = nil - } - duration, restarts, status := tc.finalTxnStatsLocked() - tc.mu.hbRunning = false - tc.mu.Unlock() - tc.updateStats(duration, restarts, status) - }() - - var closer <-chan struct{} - { - tc.mu.Lock() - closer = tc.mu.txnEnd - tc.mu.Unlock() - if closer == nil { - return - } - } - // Loop with ticker for periodic heartbeats. - for { - select { - case <-tickChan: - if !tc.heartbeat(ctx) { - return - } - case <-closer: - // Transaction finished normally. - return - case <-tc.stopper.ShouldQuiesce(): - return - } - } -} - -// abortTxnAsyncLocked sends an EndTransaction asynchronously to the wrapped -// Sender. -func (tc *TxnCoordSender) abortTxnAsyncLocked() { - // NB: We use context.Background() here because we don't want a canceled - // context to interrupt the aborting. - ctx := tc.AnnotateCtx(context.Background()) - - tc.cleanupTxnLocked(ctx, aborted) - txn := tc.mu.txn.Clone() - - // Construct a batch with an EndTransaction request. - ba := roachpb.BatchRequest{} - ba.Header = roachpb.Header{Txn: &txn} - ba.Add(&roachpb.EndTransactionRequest{ - Commit: false, - // Resolved intents should maintain an abort span entry to - // prevent concurrent requests from failing to notice the - // transaction was aborted. - Poison: true, - }) - - log.VEventf(ctx, 2, "async abort for txn: %s", txn) - if err := tc.stopper.RunAsyncTask( - ctx, "kv.TxnCoordSender: aborting txn", func(ctx context.Context) { - // Send the abort request through the interceptor stack. This is - // important because we need the txnIntentCollector to append - // intents to the EndTransaction request. - tc.mu.Lock() - defer tc.mu.Unlock() - _, pErr := tc.interceptorStack[0].SendLocked(ctx, ba) - if pErr != nil { - log.VErrEventf(ctx, 1, - "async abort failed for %s: %s ", txn, pErr) - } - }, - ); err != nil { - log.Warning(ctx, err) - } -} - -// heartbeat sends a HeartbeatTxnRequest to the txn record. -// Errors that carry update txn information (e.g. TransactionAbortedError) will -// update the txn. Other errors are swallowed. -// Returns true if heartbeating should continue, false if the transaction is no -// longer Pending and so there's no point in heartbeating further. -func (tc *TxnCoordSender) heartbeat(ctx context.Context) bool { - tc.mu.Lock() - txn := tc.mu.txn.Clone() - tc.mu.Unlock() - - if txn.Key == nil { - log.Fatalf(ctx, "attempting to heartbeat txn without anchor key: %v", txn) - } - - if txn.Status != roachpb.PENDING { - // A previous iteration has already determined that the transaction is - // already finalized. - return false - } - - ba := roachpb.BatchRequest{} - ba.Txn = &txn - - hb := &roachpb.HeartbeatTxnRequest{ - RequestHeader: roachpb.RequestHeader{ - Key: txn.Key, - }, - Now: tc.clock.Now(), - } - ba.Add(hb) - - log.VEvent(ctx, 2, "heartbeat") - br, pErr := tc.wrapped.Send(ctx, ba) - - var respTxn *roachpb.Transaction - if pErr != nil { - log.VEventf(ctx, 2, "heartbeat failed: %s", pErr) - - // If the heartbeat request arrived to find a missing transaction record - // then we ignore the error. This is possible if the heartbeat loop was - // started before a BeginTxn request succeeds because of ambiguity in the - // first write request's response. - if tse, ok := pErr.GetDetail().(*roachpb.TransactionStatusError); ok && - tse.Reason == roachpb.TransactionStatusError_REASON_TXN_NOT_FOUND { - return true - } - - if pErr.GetTxn() != nil { - // It is not expected for a 2.1 node to return an error with a transaction - // in it. For one, heartbeats are not supposed to return - // TransactionAbortedErrors. - // TODO(andrei): Remove this in 2.2. - respTxn = pErr.GetTxn() - } else { - return true - } - } else { - respTxn = br.Responses[0].GetInner().(*roachpb.HeartbeatTxnResponse).Txn - } - - // Update our txn. In particular, we need to make sure that the client will - // notice when the txn has been aborted (in which case we'll give them an - // error on their next request). +// UpdateStateOnRemoteRetryableErr is part of the TxnSender interface. +func (tc *TxnCoordSender) UpdateStateOnRemoteRetryableErr( + ctx context.Context, pErr *roachpb.Error, +) *roachpb.Error { tc.mu.Lock() defer tc.mu.Unlock() - tc.mu.txn.Update(respTxn) - if tc.mu.txn.Status != roachpb.PENDING { - log.VEventf(ctx, 1, "Heartbeat detected aborted txn. Cleaning up.") - tc.abortTxnAsyncLocked() - return false - } - return true + err := tc.handleRetryableErrLocked(ctx, pErr) + tc.mu.txn.Update(&err.Transaction) + return roachpb.NewError(err) } -// startHeartbeatLoopLocked starts a heartbeat loop in a different goroutine. -func (tc *TxnCoordSender) startHeartbeatLoopLocked(ctx context.Context) error { - if tc.mu.txnEnd != nil { - log.Fatal(ctx, "attempting to start a second heartbeat loop ") - } - - tc.mu.hbRunning = 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 +// handleRetryableErrLocked takes a retriable error and creates a +// HandledRetryableError containing the transaction that needs to be used by the +// next attempt. It also handles various aspects of updating the +// TxnCoordSender's state, but notably it does not update its proto: the caller +// needs to call tc.mu.txn.Update(pErr.GetTxn()). +func (tc *TxnCoordSender) handleRetryableErrLocked( + ctx context.Context, pErr *roachpb.Error, +) *roachpb.HandledRetryableTxnError { + // If the error is a transaction retry error, update metrics to + // reflect the reason for the restart. + // TODO(spencer): this code path does not account for retry errors + // experienced by dist sql (see internal/client/txn.go). + if tErr, ok := pErr.GetDetail().(*roachpb.TransactionRetryError); ok { + switch tErr.Reason { + case roachpb.RETRY_WRITE_TOO_OLD: + tc.metrics.RestartsWriteTooOld.Inc(1) + case roachpb.RETRY_DELETE_RANGE: + tc.metrics.RestartsDeleteRange.Inc(1) + case roachpb.RETRY_SERIALIZABLE: + tc.metrics.RestartsSerializable.Inc(1) + case roachpb.RETRY_POSSIBLE_REPLAY: + tc.metrics.RestartsPossibleReplay.Inc(1) + } } - - 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{}) - // Create a new context so that the heartbeat loop doesn't inherit the - // caller's cancelation. - // We want the loop to run in a span linked to the current one, though, so we - // put our span in the new context and expect RunAsyncTask to fork it - // immediately. - hbCtx := tc.AnnotateCtx(context.Background()) - hbCtx = opentracing.ContextWithSpan(hbCtx, opentracing.SpanFromContext(ctx)) - - if err := tc.stopper.RunAsyncTask( - hbCtx, "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) - return err + errTxnID := pErr.GetTxn().ID + newTxn := roachpb.PrepareTransactionForRetry(ctx, pErr, tc.mu.userPriority, tc.clock) + + // We'll pass a HandledRetryableTxnError up to the next layer. + retErr := roachpb.NewHandledRetryableTxnError( + pErr.Message, + errTxnID, // the id of the transaction that encountered the error + newTxn) + + // If the ID changed, it means we had to start a new transaction and the + // old one is toast. This TxnCoordSender cannot be used any more - future + // Send() calls will be rejected; the client is supposed to create a new + // one. + if errTxnID != newTxn.ID { + // Remember that this txn is aborted to reject future requests. + tc.mu.txn.Status = roachpb.ABORTED + // Abort the old txn. The client is not supposed to use use this + // TxnCoordSender any more. + tc.interceptorAlloc.txnHeartbeat.abortTxnAsyncLocked(ctx) + return retErr + } + + // Reset state as this is a retryable txn error that is incrementing + // the transaction's epoch. + log.VEventf(ctx, 2, "resetting epoch-based coordinator state on retry") + for _, reqInt := range tc.interceptorStack { + reqInt.epochBumpedLocked() } - return nil -} - -// IsTracking returns true if the heartbeat loop is running. -func (tc *TxnCoordSender) IsTracking() bool { - tc.mu.Lock() - defer tc.mu.Unlock() - return tc.mu.hbRunning + return retErr } // updateStateLocked updates the transaction state in both the success and error @@ -978,7 +822,6 @@ func (tc *TxnCoordSender) updateStateLocked( br *roachpb.BatchResponse, pErr *roachpb.Error, ) *roachpb.Error { - txnID := ba.Txn.ID var responseTxn *roachpb.Transaction if pErr == nil { @@ -987,6 +830,7 @@ func (tc *TxnCoordSender) updateStateLocked( // Only handle transaction retry errors if this is a root transaction. if pErr.TransactionRestart != roachpb.TransactionRestart_NONE && tc.typ == client.RootTxn { + errTxnID := pErr.GetTxn().ID // The ID of the txn that needs to be restarted. if errTxnID != txnID { // KV should not return errors for transactions other than the one in @@ -994,48 +838,14 @@ func (tc *TxnCoordSender) updateStateLocked( log.Fatalf(ctx, "retryable error for the wrong txn. ba.Txn: %s. pErr: %s", ba.Txn, pErr) } - // If the error is a transaction retry error, update metrics to - // reflect the reason for the restart. - // TODO(spencer): this code path does not account for retry errors - // experienced by dist sql (see internal/client/txn.go). - if tErr, ok := pErr.GetDetail().(*roachpb.TransactionRetryError); ok { - switch tErr.Reason { - case roachpb.RETRY_WRITE_TOO_OLD: - tc.metrics.RestartsWriteTooOld.Inc(1) - case roachpb.RETRY_DELETE_RANGE: - tc.metrics.RestartsDeleteRange.Inc(1) - case roachpb.RETRY_SERIALIZABLE: - tc.metrics.RestartsSerializable.Inc(1) - case roachpb.RETRY_POSSIBLE_REPLAY: - tc.metrics.RestartsPossibleReplay.Inc(1) - } - } - nextTxn := roachpb.PrepareTransactionForRetry(ctx, pErr, ba.UserPriority, tc.clock) - responseTxn = &nextTxn - - // We'll pass a HandledRetryableTxnError up to the next layer. - pErr = roachpb.NewError( - roachpb.NewHandledRetryableTxnError( - pErr.Message, - errTxnID, // the id of the transaction that encountered the error - nextTxn)) - - // If the ID changed, it means we had to start a new transaction and the - // old one is toast. This TxnCoordSender cannot be used any more - future - // Send() calls will be rejected; the client is supposed to create a new - // one. - if errTxnID != nextTxn.ID { - tc.abortTxnAsyncLocked() - return pErr - } - // Reset state as this is a retryable txn error that is incrementing - // the transaction's epoch. - log.VEventf(ctx, 2, "resetting epoch-based coordinator state on retry") - tc.mu.commandCount = 0 - for _, reqInt := range tc.interceptorStack { - reqInt.epochBumpedLocked() + err := tc.handleRetryableErrLocked(ctx, pErr) + if err.Transaction.ID == ba.Txn.ID { + // We'll update our txn, unless this was an abort error. + cp := err.Transaction.Clone() + responseTxn = &cp } + pErr = roachpb.NewError(err) } else { // We got a non-retryable error, or a retryable error at a leaf // transaction, and need to pass responsibility for handling it @@ -1048,35 +858,163 @@ func (tc *TxnCoordSender) updateStateLocked( } // Update our record of this transaction, even on error. + // Note that multiple retriable errors for the same epoch might arrive; also + // we might get retriable errors for old epochs. We rely on the associativity + // of Transaction.Update to sort out this lack of ordering guarantee. if responseTxn != nil { tc.mu.txn.Update(responseTxn) } + return pErr +} - if pErr != nil { - // On rollback error, stop the heartbeat loop. No more requests can come - // after a rollback, and there's nobody else to stop the heartbeat loop. - // The rollback success, like the commit success, is handled similarly - // below. - et, isEnding := ba.GetArg(roachpb.EndTransaction) - if isEnding && !et.(*roachpb.EndTransactionRequest).Commit { - tc.cleanupTxnLocked(ctx, aborted) - } +// setTxnAnchorKey sets the key at which to anchor the transaction record. The +// transaction anchor key defaults to the first key written in a transaction. +func (tc *TxnCoordSender) setTxnAnchorKeyLocked(key roachpb.Key) error { + if len(tc.mu.txn.Key) != 0 { + return errors.Errorf("transaction anchor key already set") } + tc.mu.txn.Key = key + return nil +} - return pErr +// SetSystemConfigTrigger is part of the client.TxnSender interface. +func (tc *TxnCoordSender) SetSystemConfigTrigger() error { + tc.mu.Lock() + defer tc.mu.Unlock() + if !tc.mu.systemConfigTrigger { + tc.mu.systemConfigTrigger = true + // The system-config trigger must be run on the system-config range which + // means any transaction with the trigger set needs to be anchored to the + // system-config range. + return tc.setTxnAnchorKeyLocked(keys.SystemConfigSpan.Key) + } + return nil +} + +// TxnStatus is part of the client.TxnSender interface. +func (tc *TxnCoordSender) TxnStatus() roachpb.TransactionStatus { + tc.mu.Lock() + defer tc.mu.Unlock() + return tc.mu.txn.Status +} + +// SetUserPriority is part of the client.TxnSender interface. +func (tc *TxnCoordSender) SetUserPriority(pri roachpb.UserPriority) error { + tc.mu.Lock() + defer tc.mu.Unlock() + + // Negative priorities come from txn.InternalSetPriority. + if tc.mu.active && pri > 0 { + return errors.Errorf("cannot change the user priority of a running transaction") + } + tc.mu.userPriority = pri + tc.mu.txn.Priority = roachpb.MakePriority(pri) + return nil +} + +// SetDebugName is part of the client.TxnSender interface. +func (tc *TxnCoordSender) SetDebugName(name string) { + tc.mu.Lock() + defer tc.mu.Unlock() + + if tc.mu.txn.Name == name { + return + } + + if tc.mu.active { + panic("cannot change the debug name of a running transaction") + } + tc.mu.txn.Name = name +} + +// SetIsolation is part of the client.TxnSender interface. +func (tc *TxnCoordSender) SetIsolation(isolation enginepb.IsolationType) error { + tc.mu.Lock() + defer tc.mu.Unlock() + + if tc.mu.txn.Isolation == isolation { + return nil + } + if tc.mu.active { + return errors.Errorf("cannot change the isolation level of a running transaction") + } + tc.mu.txn.Isolation = isolation + return nil +} + +// OrigTimestamp is part of the client.TxnSender interface. +func (tc *TxnCoordSender) OrigTimestamp() hlc.Timestamp { + tc.mu.Lock() + defer tc.mu.Unlock() + return tc.mu.txn.OrigTimestamp +} + +// CommitTimestamp is part of the client.TxnSender interface. +func (tc *TxnCoordSender) CommitTimestamp() hlc.Timestamp { + tc.mu.Lock() + defer tc.mu.Unlock() + tc.mu.txn.OrigTimestampWasObserved = true + return tc.mu.txn.OrigTimestamp +} + +// SetFixedTimestamp is part of the client.TxnSender interface. +func (tc *TxnCoordSender) SetFixedTimestamp(ctx context.Context, ts hlc.Timestamp) { + tc.mu.Lock() + tc.mu.txn.Timestamp = ts + tc.mu.txn.OrigTimestamp = ts + tc.mu.txn.MaxTimestamp = ts + tc.mu.txn.OrigTimestampWasObserved = true + tc.mu.Unlock() } -// updateStats updates transaction metrics after a transaction finishes. -func (tc *TxnCoordSender) updateStats(duration, restarts int64, status roachpb.TransactionStatus) { - tc.metrics.Durations.RecordValue(duration) - tc.metrics.Restarts.RecordValue(restarts) - switch status { - case roachpb.ABORTED: - tc.metrics.Aborts.Inc(1) - case roachpb.PENDING: - // NOTE(andrei): Getting a PENDING status here is possible if the heartbeat - // loop has stopped because the stopper is quiescing. - case roachpb.COMMITTED: - tc.metrics.Commits.Inc(1) +// ManualRestart is part of the client.TxnSender interface. +func (tc *TxnCoordSender) ManualRestart( + ctx context.Context, pri roachpb.UserPriority, ts hlc.Timestamp, +) { + tc.mu.Lock() + defer tc.mu.Unlock() + + if tc.mu.txnState == txnFinalized { + log.Fatalf(ctx, "ManualRestart called on finalized txn: %s", tc.mu.txn) + } + + // Invalidate any writes performed by any workers after the retry updated + // the txn's proto but before we synchronized (some of these writes might + // have been performed at the wrong epoch). + tc.mu.txn.Restart(pri, 0 /* upgradePriority */, ts) + + for _, reqInt := range tc.interceptorStack { + reqInt.epochBumpedLocked() } + + // The txn might have entered the txnError state after the epoch was bumped. + // Reset the state for the retry. + tc.mu.txnState = txnPending +} + +// IsSerializablePushAndRefreshNotPossible is part of the client.TxnSender interface. +func (tc *TxnCoordSender) IsSerializablePushAndRefreshNotPossible() bool { + tc.mu.Lock() + defer tc.mu.Unlock() + + origTimestamp := tc.mu.txn.OrigTimestamp + origTimestamp.Forward(tc.mu.txn.RefreshedTimestamp) + isTxnPushed := tc.mu.txn.Timestamp != origTimestamp + // We check OrigTimestampWasObserved here because, if that's set, refreshing + // of reads is not performed. + return tc.mu.txn.Isolation == enginepb.SERIALIZABLE && + isTxnPushed && tc.mu.txn.OrigTimestampWasObserved +} + +// Epoch is part of the client.TxnSender interface. +func (tc *TxnCoordSender) Epoch() uint32 { + return tc.mu.txn.Epoch +} + +// SerializeTxn is part of the client.TxnSender interface. +func (tc *TxnCoordSender) SerializeTxn() *roachpb.Transaction { + tc.mu.Lock() + defer tc.mu.Unlock() + cpy := tc.mu.txn.Clone() + return &cpy } diff --git a/pkg/kv/txn_coord_sender_server_test.go b/pkg/kv/txn_coord_sender_server_test.go index b5fc7eee938e..00b6879ef3ba 100644 --- a/pkg/kv/txn_coord_sender_server_test.go +++ b/pkg/kv/txn_coord_sender_server_test.go @@ -42,12 +42,12 @@ import ( // that it has already been aborted (by a 3rd party). That is, we don't wait for // the client to find out before the intents are removed. // This relies on the TxnCoordSender's heartbeat loop to notice the changed -// transaction status and do an async abort. Note that, as of June 2018, -// subsequent requests sent through the TxnCoordSender return -// TransactionAbortedErrors. On those errors, the contract is that the -// client.Txn creates a new transaction internally and switches the -// TxnCoordSender instance. The expectation is that the old transaction has been -// cleaned up by that point. +// transaction status and do an async abort. +// After the heartbeat loop finds out about the abort, subsequent requests sent +// through the TxnCoordSender return TransactionAbortedErrors. On those errors, +// the contract is that the client.Txn creates a new transaction internally and +// switches the TxnCoordSender instance. The expectation is that the old +// transaction has been cleaned up by that point. func TestHeartbeatFindsOutAboutAbortedTransaction(t *testing.T) { defer leaktest.AfterTest(t)() diff --git a/pkg/kv/txn_coord_sender_test.go b/pkg/kv/txn_coord_sender_test.go index 28f6e852de49..c52875b1f2df 100644 --- a/pkg/kv/txn_coord_sender_test.go +++ b/pkg/kv/txn_coord_sender_test.go @@ -26,6 +26,7 @@ import ( "github.com/pkg/errors" "github.com/stretchr/testify/require" + "golang.org/x/sync/errgroup" "github.com/cockroachdb/cockroach/pkg/internal/client" "github.com/cockroachdb/cockroach/pkg/roachpb" @@ -39,6 +40,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/metric" "github.com/cockroachdb/cockroach/pkg/util/stop" + "github.com/cockroachdb/cockroach/pkg/util/syncutil" "github.com/cockroachdb/cockroach/pkg/util/tracing" "github.com/cockroachdb/cockroach/pkg/util/uuid" ) @@ -68,6 +70,13 @@ func makeTS(walltime int64, logical int32) hlc.Timestamp { } } +// isTracking returns true if the heartbeat loop is running. +func (tc *TxnCoordSender) isTracking() bool { + tc.mu.Lock() + defer tc.mu.Unlock() + return tc.interceptorAlloc.txnHeartbeat.mu.txnEnd != nil +} + // Test that the Transaction.Writing flag is set after performing any writes. func TestTxnCoordSenderSetWritingFlag(t *testing.T) { defer leaktest.AfterTest(t)() @@ -78,7 +87,7 @@ func TestTxnCoordSenderSetWritingFlag(t *testing.T) { if err := txn.Put(context.TODO(), roachpb.Key("a"), []byte("value")); err != nil { t.Fatal(err) } - if !txn.Proto().Writing { + if !txn.Serialize().Writing { t.Fatal("txn is not marked as writing") } } @@ -102,41 +111,18 @@ func TestTxnCoordSenderBeginTransaction(t *testing.T) { if err := txn.Put(context.Background(), key, []byte("value")); err != nil { t.Fatal(err) } - if txn.Proto().Name != "test txn" { - t.Errorf("expected txn name to be %q; got %q", "test txn", txn.Proto().Name) - } - if txn.Proto().Priority != 10 { - t.Errorf("expected txn priority 10; got %d", txn.Proto().Priority) + proto := txn.Serialize() + if proto.Name != "test txn" { + t.Errorf("expected txn name to be %q; got %q", "test txn", proto.Name) } - if !bytes.Equal(txn.Proto().Key, key) { - t.Errorf("expected txn Key to match %q != %q", key, txn.Proto().Key) - } - if txn.Proto().Isolation != enginepb.SNAPSHOT { - t.Errorf("expected txn isolation to be SNAPSHOT; got %s", txn.Proto().Isolation) - } -} - -// TestTxnCoordSenderBeginTransactionMinPriority verifies that when starting -// a new transaction, a non-zero priority is treated as a minimum value. -func TestTxnCoordSenderBeginTransactionMinPriority(t *testing.T) { - defer leaktest.AfterTest(t)() - s := createTestDB(t) - defer s.Stop() - - txn := client.NewTxn(s.DB, 0 /* gatewayNodeID */, client.RootTxn) - - // Put request will create a new transaction. - key := roachpb.Key("key") - txn.InternalSetPriority(10) - txn.Proto().Priority = 11 - if err := txn.SetIsolation(enginepb.SNAPSHOT); err != nil { - t.Fatal(err) + if proto.Priority != 10 { + t.Errorf("expected txn priority 10; got %d", proto.Priority) } - if err := txn.Put(context.Background(), key, []byte("value")); err != nil { - t.Fatal(err) + if !bytes.Equal(proto.Key, key) { + t.Errorf("expected txn Key to match %q != %q", key, proto.Key) } - if prio := txn.Proto().Priority; prio != 11 { - t.Errorf("expected txn priority 11; got %d", prio) + if proto.Isolation != enginepb.SNAPSHOT { + t.Errorf("expected txn isolation to be SNAPSHOT; got %s", proto.Isolation) } } @@ -386,10 +372,7 @@ func TestTxnCoordSenderHeartbeat(t *testing.T) { // This relies on the heartbeat loop stopping once it figures out that the txn // has been aborted. testutils.SucceedsSoon(t, func() error { - tc.mu.Lock() - done := tc.mu.txnEnd == nil - tc.mu.Unlock() - if !done { + if tc.isTracking() { return fmt.Errorf("transaction is not aborted") } return nil @@ -404,7 +387,7 @@ func TestTxnCoordSenderHeartbeat(t *testing.T) { // getTxn fetches the requested key and returns the transaction info. func getTxn(ctx context.Context, txn *client.Txn) (*roachpb.Transaction, *roachpb.Error) { - txnMeta := txn.Proto().TxnMeta + txnMeta := txn.Serialize().TxnMeta qt := &roachpb.QueryTxnRequest{ RequestHeader: roachpb.RequestHeader{ Key: txnMeta.Key, @@ -428,10 +411,7 @@ func getTxn(ctx context.Context, txn *client.Txn) (*roachpb.Transaction, *roachp func verifyCleanup(key roachpb.Key, eng engine.Engine, t *testing.T, coords ...*TxnCoordSender) { testutils.SucceedsSoon(t, func() error { for _, coord := range coords { - coord.mu.Lock() - hb := coord.mu.txnEnd != nil - coord.mu.Unlock() - if hb { + if coord.isTracking() { return fmt.Errorf("expected no heartbeat") } } @@ -667,10 +647,7 @@ func TestTxnCoordSenderGCWithAmbiguousResultErr(t *testing.T) { testutils.SucceedsSoon(t, func() error { // Locking the TxnCoordSender to prevent a data race. - tc.mu.Lock() - done := tc.mu.txnEnd == nil - tc.mu.Unlock() - if !done { + if tc.isTracking() { return errors.Errorf("expected garbage collection") } return nil @@ -828,27 +805,28 @@ func TestTxnCoordSenderTxnUpdatedOnError(t *testing.T) { if test.name != "nil" && err == nil { t.Fatalf("expected an error") } - txnReset := origTxnProto.ID != txn.Proto().ID + proto := txn.Serialize() + txnReset := origTxnProto.ID != proto.ID if txnReset != test.expNewTransaction { t.Fatalf("expected txn reset: %t and got: %t", test.expNewTransaction, txnReset) } - if txn.Proto().Epoch != test.expEpoch { + if proto.Epoch != test.expEpoch { t.Errorf("expected epoch = %d; got %d", - test.expEpoch, txn.Proto().Epoch) + test.expEpoch, proto.Epoch) } - if txn.Proto().Priority != test.expPri { + if proto.Priority != test.expPri { t.Errorf("expected priority = %d; got %d", - test.expPri, txn.Proto().Priority) + test.expPri, proto.Priority) } - if txn.Proto().Timestamp != test.expTS { + if proto.Timestamp != test.expTS { t.Errorf("expected timestamp to be %s; got %s", - test.expTS, txn.Proto().Timestamp) + test.expTS, proto.Timestamp) } - if txn.Proto().OrigTimestamp != test.expOrigTS { + if proto.OrigTimestamp != test.expOrigTS { t.Errorf("expected orig timestamp to be %s; got %s", - test.expOrigTS, txn.Proto().OrigTimestamp) + test.expOrigTS, proto.OrigTimestamp) } - if ns := txn.Proto().ObservedTimestamps; (len(ns) != 0) != test.nodeSeen { + if ns := proto.ObservedTimestamps; (len(ns) != 0) != test.nodeSeen { t.Errorf("expected nodeSeen=%t, but list of hosts is %v", test.nodeSeen, ns) } @@ -856,35 +834,6 @@ func TestTxnCoordSenderTxnUpdatedOnError(t *testing.T) { } } -// TestTxnCoordIdempotentCleanup verifies that cleanupTxnLocked is idempotent. -func TestTxnCoordIdempotentCleanup(t *testing.T) { - defer leaktest.AfterTest(t)() - s := createTestDB(t) - defer s.Stop() - - txn := client.NewTxn(s.DB, 0 /* gatewayNodeID */, client.RootTxn) - tc := txn.Sender().(*TxnCoordSender) - ba := txn.NewBatch() - ba.Put(roachpb.Key("a"), []byte("value")) - if err := txn.Run(context.TODO(), ba); err != nil { - t.Fatal(err) - } - - tc.mu.Lock() - // Clean up twice successively. - tc.cleanupTxnLocked(context.Background(), aborted) - tc.cleanupTxnLocked(context.Background(), aborted) - tc.mu.Unlock() - - // For good measure, try to commit (which cleans up once more if it - // succeeds, which it may not if the previous cleanup has already - // terminated the heartbeat goroutine) - ba = txn.NewBatch() - ba.AddRawRequest(&roachpb.EndTransactionRequest{Commit: true}) - err := txn.Run(context.TODO(), ba) - assertTransactionAbortedError(t, err) -} - // TestTxnMultipleCoord checks that multiple txn coordinators can be // used for reads by a single transaction, and their state can be combined. func TestTxnMultipleCoord(t *testing.T) { @@ -903,7 +852,7 @@ func TestTxnMultipleCoord(t *testing.T) { } // New create a second, leaf coordinator. - txn2 := client.NewTxnWithProto(s.DB, 0 /* gatewayNodeID */, client.LeafTxn, *txn.Proto()) + txn2 := client.NewTxnWithProto(s.DB, 0 /* gatewayNodeID */, client.LeafTxn, *txn.Serialize()) // Start the second transaction. key2 := roachpb.Key("b") @@ -951,7 +900,7 @@ func TestTxnCoordSenderErrorWithIntent(t *testing.T) { {*roachpb.NewErrorf("testError"), "testError"}, } for i, test := range testCases { - func() { + t.Run("", func(t *testing.T) { var senderFn client.SenderFunc = func(_ context.Context, ba roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { txn := ba.Txn.Clone() txn.Writing = true @@ -971,7 +920,6 @@ func TestTxnCoordSenderErrorWithIntent(t *testing.T) { var ba roachpb.BatchRequest key := roachpb.Key("test") - ba.Add(&roachpb.BeginTransactionRequest{RequestHeader: roachpb.RequestHeader{Key: key}}) ba.Add(&roachpb.PutRequest{RequestHeader: roachpb.RequestHeader{Key: key}}) ba.Add(&roachpb.EndTransactionRequest{}) txn := roachpb.MakeTransaction("test", key, 0, 0, clock.Now(), 0) @@ -982,7 +930,7 @@ func TestTxnCoordSenderErrorWithIntent(t *testing.T) { if !testutils.IsPError(pErr, test.errMsg) { t.Errorf("%d: error did not match %s: %v", i, test.errMsg, pErr) } - }() + }) } } @@ -1212,7 +1160,7 @@ func TestTxnAbortCount(t *testing.T) { }); !testutils.IsError(err, intentionalErrText) { t.Fatalf("unexpected error: %v", err) } - checkTxnMetrics(t, metrics, "abort txn", 0, 0, 0, 1, 0) + checkTxnMetrics(t, metrics, "abort txn", 0, 0, 0, 1 /* aborts */, 0) } func TestTxnRestartCount(t *testing.T) { @@ -1221,16 +1169,17 @@ func TestTxnRestartCount(t *testing.T) { readKey := []byte("read") writeKey := []byte("write") value := []byte("value") + ctx := context.Background() for _, expRestart := range []bool{true, false} { - t.Run(fmt.Sprintf("expected restart: %t", expRestart), func(t *testing.T) { + t.Run(fmt.Sprintf("expected restart:%t", expRestart), func(t *testing.T) { s, metrics, cleanupFn := setupMetricsTest(t) defer cleanupFn() // Start a transaction and do a GET. This forces a timestamp to be // chosen for the transaction. txn := client.NewTxn(s.DB, 0 /* gatewayNodeID */, client.RootTxn) - if _, err := txn.Get(context.TODO(), readKey); err != nil { + if _, err := txn.Get(ctx, readKey); err != nil { t.Fatal(err) } @@ -1238,7 +1187,7 @@ func TestTxnRestartCount(t *testing.T) { // transaction, at a higher timestamp, which will necessitate a // txn restart when the original read key span is updated. if expRestart { - if err := s.DB.Put(context.TODO(), readKey, value); err != nil { + if err := s.DB.Put(ctx, readKey, value); err != nil { t.Fatal(err) } } @@ -1246,32 +1195,31 @@ func TestTxnRestartCount(t *testing.T) { // Outside of the transaction, read the same key as will be // written within the transaction. This means that future // attempts to write will forward the txn timestamp. - if _, err := s.DB.Get(context.TODO(), writeKey); err != nil { + if _, err := s.DB.Get(ctx, writeKey); err != nil { t.Fatal(err) } // This put will lay down an intent, txn timestamp will increase // beyond OrigTimestamp. - if err := txn.Put(context.TODO(), writeKey, value); err != nil { + if err := txn.Put(ctx, writeKey, value); err != nil { t.Fatal(err) } - if !txn.Proto().OrigTimestamp.Less(txn.Proto().Timestamp) { - t.Errorf("expected timestamp to increase: %s", txn.Proto()) + proto := txn.Serialize() + if !proto.OrigTimestamp.Less(proto.Timestamp) { + t.Errorf("expected timestamp to increase: %s", proto) } // Wait for heartbeat to start. tc := txn.Sender().(*TxnCoordSender) testutils.SucceedsSoon(t, func() error { - tc.mu.Lock() - defer tc.mu.Unlock() - if tc.mu.txnEnd == nil { + if !tc.isTracking() { return errors.New("expected heartbeat to start") } return nil }) // Commit (should cause restart metric to increase). - err := txn.CommitOrCleanup(context.TODO()) + err := txn.CommitOrCleanup(ctx) if expRestart { assertTransactionRetryError(t, err) checkTxnMetrics(t, metrics, "restart txn", 0, 0, 0, 1, 1) @@ -1365,6 +1313,8 @@ func TestAbortTransactionOnCommitErrors(t *testing.T) { _ context.Context, ba roachpb.BatchRequest, ) (*roachpb.BatchResponse, *roachpb.Error) { br := ba.CreateReply() + txn := ba.Txn.Clone() + br.Txn = &txn if _, hasBT := ba.GetArg(roachpb.BeginTransaction); hasBT { if _, ok := ba.Requests[1].GetInner().(*roachpb.PutRequest); !ok { @@ -1457,7 +1407,10 @@ func (s *mockSender) Send( } } // If none of the matchers triggered, just create an empty reply. - return ba.CreateReply(), nil + br := ba.CreateReply() + txn := ba.Txn.Clone() + br.Txn = &txn + return br, nil } // Test that a rollback sent to the TxnCoordSender stops the heartbeat loop even @@ -1492,7 +1445,7 @@ func TestRollbackErrorStopsHeartbeat(t *testing.T) { txn := client.NewTxn(db, roachpb.NodeID(1), client.RootTxn) txnHeader := roachpb.Header{ - Txn: txn.Proto(), + Txn: txn.Serialize(), } if _, pErr := client.SendWrappedWith( ctx, txn, txnHeader, &roachpb.PutRequest{ @@ -1503,7 +1456,7 @@ func TestRollbackErrorStopsHeartbeat(t *testing.T) { ); pErr != nil { t.Fatal(pErr) } - if !txn.Sender().(*TxnCoordSender).IsTracking() { + if !txn.Sender().(*TxnCoordSender).isTracking() { t.Fatalf("expected TxnCoordSender to be tracking after the write") } @@ -1515,23 +1468,20 @@ func TestRollbackErrorStopsHeartbeat(t *testing.T) { } testutils.SucceedsSoon(t, func() error { - if txn.Sender().(*TxnCoordSender).IsTracking() { + if txn.Sender().(*TxnCoordSender).isTracking() { return fmt.Errorf("still tracking") } return nil }) } -// Test that intent tracking and the heartbeat loop behave correctly for +// Test that intent tracking behaves correctly for // transactions that attempt to run a batch containing both a BeginTransaction // and an EndTransaction. Since in case of an error it's not easy to determine // whether any intents have been laid down (i.e. in case the batch was split by // the DistSender and then there was mixed success for the sub-batches, or in -// case a retriable error is returned), the test verifies that the heartbeat -// loop is still going after the error and that the intents are properly tracked -// and attached to a subsequent EndTransaction. -// There was a time when the TxnCoordSender was cutting corners for these -// wannabe 1PC batches, but that proved to be a bad idea. +// case a retriable error is returned), the test verifies that all possible +// intents are properly tracked and attached to a subsequent EndTransaction. func TestOnePCErrorTracking(t *testing.T) { defer leaktest.AfterTest(t)() ctx := context.Background() @@ -1586,7 +1536,7 @@ func TestOnePCErrorTracking(t *testing.T) { txn := client.NewTxn(db, roachpb.NodeID(1), client.RootTxn) txnHeader := roachpb.Header{ - Txn: txn.Proto(), + Txn: txn.Serialize(), } b := txn.NewBatch() b.Put(key, "test value") @@ -1594,10 +1544,6 @@ func TestOnePCErrorTracking(t *testing.T) { t.Fatal(err) } - if !txn.Sender().(*TxnCoordSender).IsTracking() { - t.Fatalf("expected TxnCoordSender to be tracking after the write") - } - // Now send a rollback and verify that the TxnCoordSender attaches the intent // to it. if _, pErr := client.SendWrappedWith( @@ -1609,7 +1555,7 @@ func TestOnePCErrorTracking(t *testing.T) { // As always, check that the rollback we just sent stops the heartbeat loop. testutils.SucceedsSoon(t, func() error { - if txn.Sender().(*TxnCoordSender).IsTracking() { + if txn.Sender().(*TxnCoordSender).isTracking() { return fmt.Errorf("still tracking") } return nil @@ -1667,6 +1613,662 @@ func TestIntentTrackingBeforeBeginTransaction(t *testing.T) { } } +// TestCommitReadOnlyTransaction verifies that a read-only does not send an +// EndTransactionRequest. +func TestCommitReadOnlyTransaction(t *testing.T) { + defer leaktest.AfterTest(t)() + ctx := context.Background() + clock := hlc.NewClock(hlc.UnixNano, time.Nanosecond) + ambient := log.AmbientContext{Tracer: tracing.NewTracer()} + sender := &mockSender{} + stopper := stop.NewStopper() + defer stopper.Stop(ctx) + + var calls []roachpb.Method + sender.match(func(ba roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { + calls = append(calls, ba.Methods()...) + return nil, nil + }) + + factory := NewTxnCoordSenderFactory( + TxnCoordSenderFactoryConfig{ + AmbientCtx: ambient, + Clock: clock, + Stopper: stopper, + }, + sender, + ) + testutils.RunTrueAndFalse(t, "explicit txn", func(t *testing.T, explicitTxn bool) { + testutils.RunTrueAndFalse(t, "with get", func(t *testing.T, withGet bool) { + calls = nil + db := client.NewDB(testutils.MakeAmbientCtx(), factory, clock) + if err := db.Txn(ctx, func(ctx context.Context, txn *client.Txn) error { + b := txn.NewBatch() + if withGet { + b.Get("foo") + } + if explicitTxn { + return txn.CommitInBatch(ctx, b) + } + return txn.Run(ctx, b) + }); err != nil { + t.Fatal(err) + } + + expectedCalls := []roachpb.Method(nil) + if withGet { + expectedCalls = append(expectedCalls, roachpb.Get) + } + if !reflect.DeepEqual(expectedCalls, calls) { + t.Fatalf("expected %s, got %s", expectedCalls, calls) + } + }) + }) +} + +// TestCommitMutatingTransaction verifies that a transaction is committed +// upon successful invocation of the retryable func. +func TestCommitMutatingTransaction(t *testing.T) { + defer leaktest.AfterTest(t)() + ctx := context.Background() + clock := hlc.NewClock(hlc.UnixNano, time.Nanosecond) + ambient := log.AmbientContext{Tracer: tracing.NewTracer()} + sender := &mockSender{} + stopper := stop.NewStopper() + defer stopper.Stop(ctx) + + var calls []roachpb.Method + sender.match(func(ba roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { + calls = append(calls, ba.Methods()...) + if bt, ok := ba.GetArg(roachpb.BeginTransaction); ok && !bt.Header().Key.Equal(roachpb.Key("a")) { + t.Errorf("expected begin transaction key to be \"a\"; got %s", bt.Header().Key) + } + if et, ok := ba.GetArg(roachpb.EndTransaction); ok && !et.(*roachpb.EndTransactionRequest).Commit { + t.Errorf("expected commit to be true") + } + return nil, nil + }) + + factory := NewTxnCoordSenderFactory( + TxnCoordSenderFactoryConfig{ + AmbientCtx: ambient, + Clock: clock, + Stopper: stopper, + }, + sender, + ) + + // Test all transactional write methods. + testArgs := []struct { + f func(ctx context.Context, txn *client.Txn) error + expMethod roachpb.Method + // pointWrite is set if the method is a "point write", which means that it + // will be pipelined and we should expect a QueryIntent request at commit + // time. + pointWrite bool + }{ + { + f: func(ctx context.Context, txn *client.Txn) error { return txn.Put(ctx, "a", "b") }, + expMethod: roachpb.Put, + pointWrite: true, + }, + { + f: func(ctx context.Context, txn *client.Txn) error { return txn.CPut(ctx, "a", "b", nil) }, + expMethod: roachpb.ConditionalPut, + pointWrite: true, + }, + { + f: func(ctx context.Context, txn *client.Txn) error { + _, err := txn.Inc(ctx, "a", 1) + return err + }, + expMethod: roachpb.Increment, + pointWrite: true, + }, + { + f: func(ctx context.Context, txn *client.Txn) error { return txn.Del(ctx, "a") }, + expMethod: roachpb.Delete, + pointWrite: true, + }, + { + f: func(ctx context.Context, txn *client.Txn) error { return txn.DelRange(ctx, "a", "b") }, + expMethod: roachpb.DeleteRange, + pointWrite: false, + }, + } + for i, test := range testArgs { + t.Run(test.expMethod.String(), func(t *testing.T) { + calls = nil + db := client.NewDB(testutils.MakeAmbientCtx(), factory, clock) + if err := db.Txn(ctx, test.f); err != nil { + t.Fatalf("%d: unexpected error on commit: %s", i, err) + } + expectedCalls := []roachpb.Method{roachpb.BeginTransaction, test.expMethod} + if test.pointWrite { + expectedCalls = append(expectedCalls, roachpb.QueryIntent) + } + expectedCalls = append(expectedCalls, roachpb.EndTransaction) + if !reflect.DeepEqual(expectedCalls, calls) { + t.Fatalf("%d: expected %s, got %s", i, expectedCalls, calls) + } + }) + } +} + +// TestTxnInsertBeginTransaction verifies that a begin transaction +// request is inserted just before the first mutating command. +func TestTxnInsertBeginTransaction(t *testing.T) { + defer leaktest.AfterTest(t)() + ctx := context.Background() + clock := hlc.NewClock(hlc.UnixNano, time.Nanosecond) + ambient := log.AmbientContext{Tracer: tracing.NewTracer()} + sender := &mockSender{} + stopper := stop.NewStopper() + defer stopper.Stop(ctx) + + var calls []roachpb.Method + sender.match(func(ba roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { + calls = append(calls, ba.Methods()...) + if bt, ok := ba.GetArg(roachpb.BeginTransaction); ok && !bt.Header().Key.Equal(roachpb.Key("a")) { + t.Errorf("expected begin transaction key to be \"a\"; got %s", bt.Header().Key) + } + if et, ok := ba.GetArg(roachpb.EndTransaction); ok && !et.(*roachpb.EndTransactionRequest).Commit { + t.Errorf("expected commit to be true") + } + return nil, nil + }) + + factory := NewTxnCoordSenderFactory( + TxnCoordSenderFactoryConfig{ + AmbientCtx: ambient, + Clock: clock, + Stopper: stopper, + }, + sender, + ) + + db := client.NewDB(testutils.MakeAmbientCtx(), factory, clock) + if err := db.Txn(context.TODO(), func(ctx context.Context, txn *client.Txn) error { + if _, err := txn.Get(ctx, "foo"); err != nil { + return err + } + return txn.Put(ctx, "a", "b") + }); err != nil { + t.Fatalf("unexpected error on commit: %s", err) + } + expectedCalls := []roachpb.Method{ + roachpb.Get, + roachpb.BeginTransaction, + roachpb.Put, + roachpb.QueryIntent, + roachpb.EndTransaction} + if !reflect.DeepEqual(expectedCalls, calls) { + t.Fatalf("expected %s, got %s", expectedCalls, calls) + } +} + +// TestBeginTransactionErrorIndex verifies that the error index is cleared +// when a BeginTransaction command causes an error. +func TestBeginTransactionErrorIndex(t *testing.T) { + defer leaktest.AfterTest(t)() + ctx := context.Background() + clock := hlc.NewClock(hlc.UnixNano, time.Nanosecond) + ambient := log.AmbientContext{Tracer: tracing.NewTracer()} + sender := &mockSender{} + stopper := stop.NewStopper() + defer stopper.Stop(ctx) + + sender.match(func(ba roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { + pErr := roachpb.NewError(&roachpb.WriteIntentError{}) + pErr.SetErrorIndex(0) + return nil, pErr + }) + + factory := NewTxnCoordSenderFactory( + TxnCoordSenderFactoryConfig{ + AmbientCtx: ambient, + Clock: clock, + Stopper: stopper, + }, + sender, + ) + + db := client.NewDB(testutils.MakeAmbientCtx(), factory, clock) + _ = db.Txn(ctx, func(ctx context.Context, txn *client.Txn) error { + b := txn.NewBatch() + b.Put("a", "b") + err := getOneErr(txn.Run(ctx, b), b) + if err == nil { + t.Fatal("missing err") + } + pErr := b.MustPErr() + // Verify that the original error type is preserved, but the error index is unset. + if _, ok := pErr.GetDetail().(*roachpb.WriteIntentError); !ok { + t.Fatalf("unexpected error %s", pErr) + } + if pErr.Index != nil { + t.Errorf("error index must not be set, but got %d", pErr.Index) + } + return err + }) +} + +// getOneErr returns the error for a single-request Batch that was run. +// runErr is the error returned by Run, b is the Batch that was passed to Run. +func getOneErr(runErr error, b *client.Batch) error { + if runErr != nil && len(b.Results) > 0 { + return b.Results[0].Err + } + return runErr +} + +// TestAbortReadOnlyTransaction verifies that aborting a read-only +// transaction does not prompt an EndTransaction call. +func TestAbortReadOnlyTransaction(t *testing.T) { + defer leaktest.AfterTest(t)() + ctx := context.Background() + clock := hlc.NewClock(hlc.UnixNano, time.Nanosecond) + ambient := log.AmbientContext{Tracer: tracing.NewTracer()} + sender := &mockSender{} + stopper := stop.NewStopper() + defer stopper.Stop(ctx) + + var calls []roachpb.Method + sender.match(func(ba roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { + calls = append(calls, ba.Methods()...) + return nil, nil + }) + + factory := NewTxnCoordSenderFactory( + TxnCoordSenderFactoryConfig{ + AmbientCtx: ambient, + Clock: clock, + Stopper: stopper, + }, + sender, + ) + db := client.NewDB(testutils.MakeAmbientCtx(), factory, clock) + if err := db.Txn(context.TODO(), func(ctx context.Context, txn *client.Txn) error { + return errors.New("foo") + }); err == nil { + t.Fatal("expected error on abort") + } + + if calls != nil { + t.Fatalf("expected no calls, got %s", calls) + } +} + +// TestEndWriteRestartReadOnlyTransaction verifies that if +// a transaction writes, then restarts and turns read-only, +// an explicit EndTransaction call is still sent if retry- +// able didn't, regardless of whether there is an error +// or not. +func TestEndWriteRestartReadOnlyTransaction(t *testing.T) { + defer leaktest.AfterTest(t)() + ctx := context.Background() + clock := hlc.NewClock(hlc.UnixNano, time.Nanosecond) + ambient := log.AmbientContext{Tracer: tracing.NewTracer()} + sender := &mockSender{} + stopper := stop.NewStopper() + defer stopper.Stop(ctx) + + var calls []roachpb.Method + sender.match(func(ba roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { + calls = append(calls, ba.Methods()...) + if _, ok := ba.GetArg(roachpb.Put); ok { + return nil, roachpb.NewErrorWithTxn(roachpb.NewTransactionRetryError(roachpb.RETRY_SERIALIZABLE), ba.Txn) + } + return nil, nil + }) + + factory := NewTxnCoordSenderFactory( + TxnCoordSenderFactoryConfig{ + AmbientCtx: ambient, + Clock: clock, + Stopper: stopper, + TestingKnobs: ClientTestingKnobs{ + // Disable span refresh, otherwise it kicks and retries batches by + // itself. + MaxTxnRefreshAttempts: -1, + }, + }, + sender, + ) + db := client.NewDB(testutils.MakeAmbientCtx(), factory, clock) + expCalls := []roachpb.Method{roachpb.BeginTransaction, roachpb.Put, roachpb.EndTransaction} + + testutils.RunTrueAndFalse(t, "success", func(t *testing.T, success bool) { + calls = nil + firstIter := true + if err := db.Txn(ctx, func(ctx context.Context, txn *client.Txn) error { + if firstIter { + firstIter = false + if err := txn.Put(ctx, "consider", "phlebas"); err == nil { + t.Fatal("missing injected retriable error") + } + } + if !success { + return errors.New("aborting on purpose") + } + return nil + }); err == nil != success { + t.Fatalf("expected error: %t, got error: %v", !success, err) + } + if !reflect.DeepEqual(expCalls, calls) { + t.Fatalf("expected %v, got %v", expCalls, calls) + } + }) +} + +// TestTransactionKeyNotChangedInRestart verifies that if the transaction +// already has a key (we're in a restart), the key in the begin transaction +// request is not changed. +func TestTransactionKeyNotChangedInRestart(t *testing.T) { + defer leaktest.AfterTest(t)() + ctx := context.Background() + clock := hlc.NewClock(hlc.UnixNano, time.Nanosecond) + ambient := log.AmbientContext{Tracer: tracing.NewTracer()} + sender := &mockSender{} + stopper := stop.NewStopper() + defer stopper.Stop(ctx) + + keys := []string{"first", "second"} + attempt := 0 + sender.match(func(ba roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { + // Ignore the final EndTxnRequest. + if _, ok := ba.GetArg(roachpb.EndTransaction); ok { + br := ba.CreateReply() + txn := ba.Txn.Clone() + br.Txn = &txn + return nil, nil + } + + // Attempt 0 should have a BeginTxnRequest, and a PutRequest. + // Attempt 1 should have a PutRequest. + if attempt == 0 { + if _, ok := ba.GetArg(roachpb.BeginTransaction); !ok { + t.Fatalf("failed to find a begin transaction request: %v", ba) + } + } + if _, ok := ba.GetArg(roachpb.Put); !ok { + t.Fatalf("failed to find a put request: %v", ba) + } + + // In the first attempt, the transaction key is the key of the first write command. + // This key is retained between restarts, so we see the same key in the second attempt. + if expectedKey := []byte(keys[0]); !bytes.Equal(expectedKey, ba.Txn.Key) { + t.Fatalf("expected transaction key %v, got %v", expectedKey, ba.Txn.Key) + } + + if attempt == 0 { + return nil, roachpb.NewErrorWithTxn(roachpb.NewTransactionRetryError(roachpb.RETRY_SERIALIZABLE), ba.Txn) + } + return nil, nil + }) + factory := NewTxnCoordSenderFactory( + TxnCoordSenderFactoryConfig{ + AmbientCtx: ambient, + Clock: clock, + Stopper: stopper, + }, + sender, + ) + db := client.NewDB(testutils.MakeAmbientCtx(), factory, clock) + + if err := db.Txn(context.TODO(), func(ctx context.Context, txn *client.Txn) error { + defer func() { attempt++ }() + b := txn.NewBatch() + b.Put(keys[attempt], "b") + return txn.Run(ctx, b) + }); err != nil { + t.Errorf("unexpected error on commit: %s", err) + } + minimumAttempts := 2 + if attempt < minimumAttempts { + t.Errorf("expected attempt count >= %d, got %d", minimumAttempts, attempt) + } +} + +// TestSequenceNumbers verifies Requests are given sequence numbers and that +// they are incremented on successive commands. +func TestSequenceNumbers(t *testing.T) { + defer leaktest.AfterTest(t)() + ctx := context.Background() + clock := hlc.NewClock(hlc.UnixNano, time.Nanosecond) + ambient := log.AmbientContext{Tracer: tracing.NewTracer()} + sender := &mockSender{} + stopper := stop.NewStopper() + defer stopper.Stop(ctx) + + var expSequence int32 + sender.match(func(ba roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { + for _, ru := range ba.Requests { + args := ru.GetInner() + if args.Method() == roachpb.QueryIntent { + // QueryIntent requests don't have sequence numbers. + continue + } + expSequence++ + if seq := args.Header().Sequence; expSequence != seq { + t.Errorf("expected Request sequence %d; got %d. request: %T", + expSequence, seq, args) + } + } + if expSequence != ba.Txn.Sequence { + t.Errorf("expected header sequence %d; got %d", expSequence, ba.Txn.Sequence) + } + br := ba.CreateReply() + br.Txn = ba.Txn + return br, nil + }) + factory := NewTxnCoordSenderFactory( + TxnCoordSenderFactoryConfig{ + AmbientCtx: ambient, + Clock: clock, + Stopper: stopper, + }, + sender, + ) + db := client.NewDB(testutils.MakeAmbientCtx(), factory, clock) + txn := client.NewTxn(db, 0 /* gatewayNodeID */, client.RootTxn) + + for i := 0; i < 5; i++ { + var ba roachpb.BatchRequest + for j := 0; j < i; j++ { + ba.Add(roachpb.NewPut(roachpb.Key("a"), roachpb.MakeValueFromString("foo")).(*roachpb.PutRequest)) + } + if _, pErr := txn.Send(ctx, ba); pErr != nil { + t.Fatal(pErr) + } + } +} + +// TestConcurrentTxnRequests verifies that multiple requests can be executed on +// a transaction at the same time from multiple goroutines. It makes sure that +// exactly one BeginTxnRequest and one EndTxnRequest are sent. +func TestConcurrentTxnRequests(t *testing.T) { + defer leaktest.AfterTest(t)() + ctx := context.Background() + clock := hlc.NewClock(hlc.UnixNano, time.Nanosecond) + ambient := log.AmbientContext{Tracer: tracing.NewTracer()} + sender := &mockSender{} + stopper := stop.NewStopper() + defer stopper.Stop(ctx) + + var callCountsMu syncutil.Mutex + callCounts := make(map[roachpb.Method]int) + sender.match(func(ba roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { + callCountsMu.Lock() + for _, m := range ba.Methods() { + callCounts[m]++ + } + callCountsMu.Unlock() + return nil, nil + }) + factory := NewTxnCoordSenderFactory( + TxnCoordSenderFactoryConfig{ + AmbientCtx: ambient, + Clock: clock, + Stopper: stopper, + }, + sender, + ) + db := client.NewDB(testutils.MakeAmbientCtx(), factory, clock) + + const keys = "abcdefghijklmnopqrstuvwxyz" + const value = "value" + if err := db.Txn(context.TODO(), func(ctx context.Context, txn *client.Txn) error { + g, gCtx := errgroup.WithContext(ctx) + for _, keyChar := range keys { + key := string(keyChar) + g.Go(func() error { + return txn.Put(gCtx, key, value) + }) + } + return g.Wait() + }); err != nil { + t.Fatal(err) + } + + expectedCallCounts := map[roachpb.Method]int{ + roachpb.BeginTransaction: 1, + roachpb.Put: 26, + roachpb.QueryIntent: 26, + roachpb.EndTransaction: 1, + } + if !reflect.DeepEqual(expectedCallCounts, callCounts) { + t.Errorf("expected %v, got %v", expectedCallCounts, callCounts) + } +} + +// TestTxnRequestTxnTimestamp verifies response txn timestamp is +// always upgraded on successive requests. +func TestTxnRequestTxnTimestamp(t *testing.T) { + defer leaktest.AfterTest(t)() + ctx := context.Background() + manual := hlc.NewManualClock(123) + clock := hlc.NewClock(manual.UnixNano, time.Nanosecond) + ambient := log.AmbientContext{Tracer: tracing.NewTracer()} + sender := &mockSender{} + stopper := stop.NewStopper() + defer stopper.Stop(ctx) + + factory := NewTxnCoordSenderFactory( + TxnCoordSenderFactoryConfig{ + AmbientCtx: ambient, + Clock: clock, + Stopper: stopper, + }, + sender, + ) + db := client.NewDB(testutils.MakeAmbientCtx(), factory, clock) + + curReq := 0 + requests := []struct { + expRequestTS, responseTS hlc.Timestamp + }{ + {hlc.Timestamp{WallTime: 5, Logical: 0}, hlc.Timestamp{WallTime: 10, Logical: 0}}, + {hlc.Timestamp{WallTime: 10, Logical: 0}, hlc.Timestamp{WallTime: 10, Logical: 1}}, + {hlc.Timestamp{WallTime: 10, Logical: 1}, hlc.Timestamp{WallTime: 10, Logical: 0}}, + {hlc.Timestamp{WallTime: 10, Logical: 1}, hlc.Timestamp{WallTime: 20, Logical: 1}}, + {hlc.Timestamp{WallTime: 20, Logical: 1}, hlc.Timestamp{WallTime: 20, Logical: 1}}, + {hlc.Timestamp{WallTime: 20, Logical: 1}, hlc.Timestamp{WallTime: 19, Logical: 0}}, + {hlc.Timestamp{WallTime: 20, Logical: 1}, hlc.Timestamp{WallTime: 20, Logical: 1}}, + } + + sender.match(func(ba roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { + req := requests[curReq] + if req.expRequestTS != ba.Txn.Timestamp { + return nil, roachpb.NewErrorf("%d: expected ts %s got %s", + curReq, req.expRequestTS, ba.Txn.Timestamp) + } + + txnClone := ba.Txn.Clone() + br := ba.CreateReply() + br.Txn = &txnClone + br.Txn.Timestamp.Forward(requests[curReq].responseTS) + return br, nil + }) + + manual.Set(requests[0].expRequestTS.WallTime) + + if err := db.Txn(ctx, func(ctx context.Context, txn *client.Txn) error { + for curReq = range requests { + if _, err := txn.Get(ctx, "k"); err != nil { + return err + } + } + return nil + }); err != nil { + t.Fatal(err) + } +} + +// TestReadOnlyTxnObeysDeadline tests that read-only transactions obey the +// deadline. Read-only transactions have their EndTransaction elided, so the +// enforcement of the deadline is done in the client. +func TestReadOnlyTxnObeysDeadline(t *testing.T) { + defer leaktest.AfterTest(t)() + ctx := context.Background() + manual := hlc.NewManualClock(123) + clock := hlc.NewClock(manual.UnixNano, time.Nanosecond) + ambient := log.AmbientContext{Tracer: tracing.NewTracer()} + sender := &mockSender{} + stopper := stop.NewStopper() + defer stopper.Stop(ctx) + + sender.match(func(ba roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { + if _, ok := ba.GetArg(roachpb.Get); ok { + manual.Increment(100) + txnClone := ba.Txn.Clone() + br := ba.CreateReply() + br.Txn = &txnClone + br.Txn.Timestamp.Forward(clock.Now()) + return br, nil + } + return nil, nil + }) + factory := NewTxnCoordSenderFactory( + TxnCoordSenderFactoryConfig{ + AmbientCtx: ambient, + Clock: clock, + Stopper: stopper, + }, + sender, + ) + db := client.NewDB(testutils.MakeAmbientCtx(), factory, clock) + + // We're going to run two tests: one where the EndTransaction is by itself in + // a batch, one where it is not. As of June 2018, the EndTransaction is elided + // in different ways in the two cases. + + t.Run("standalone commit", func(t *testing.T) { + txn := client.NewTxn(db, 0 /* gatewayNodeID */, client.RootTxn) + // Set a deadline. We'll generate a retriable error with a higher timestamp. + txn.UpdateDeadlineMaybe(ctx, clock.Now()) + if _, err := txn.Get(ctx, "k"); err != nil { + t.Fatal(err) + } + if err := txn.Commit(ctx); !testutils.IsError( + err, "deadline exceeded before transaction finalization") { + t.Fatal(err) + } + }) + + t.Run("commit in batch", func(t *testing.T) { + txn := client.NewTxn(db, 0 /* gatewayNodeID */, client.RootTxn) + // Set a deadline. We'll generate a retriable error with a higher timestamp. + txn.UpdateDeadlineMaybe(ctx, clock.Now()) + b := txn.NewBatch() + b.Get("k") + if err := txn.CommitInBatch(ctx, b); !testutils.IsError( + err, "deadline exceeded before transaction finalization") { + t.Fatal(err) + } + }) +} + // TestTxnCoordSenderPipelining verifies that transactional pipelining of writes // is enabled by default in a transaction and is disabled after // DisablePipelining is called. It also verifies that DisablePipelining returns @@ -1733,3 +2335,50 @@ func TestTxnCoordSenderPipelining(t *testing.T) { } } } + +// Test that a txn's anchor is set to the first write key in batches mixing +// reads with writes. +func TestAnchorKey(t *testing.T) { + defer leaktest.AfterTest(t)() + + ctx := context.Background() + manual := hlc.NewManualClock(123) + clock := hlc.NewClock(manual.UnixNano, time.Nanosecond) + ambient := log.AmbientContext{Tracer: tracing.NewTracer()} + stopper := stop.NewStopper() + defer stopper.Stop(ctx) + + key1 := roachpb.Key("a") + key2 := roachpb.Key("b") + + var senderFn client.SenderFunc = func( + ctx context.Context, ba roachpb.BatchRequest, + ) (*roachpb.BatchResponse, *roachpb.Error) { + if !roachpb.Key(ba.Txn.Key).Equal(key2) { + t.Fatalf("expected anchor %q, got %q", key2, ba.Txn.Key) + } + br := ba.CreateReply() + txn := ba.Txn.Clone() + br.Txn = &txn + return br, nil + } + + factory := NewTxnCoordSenderFactory( + TxnCoordSenderFactoryConfig{ + AmbientCtx: ambient, + Clock: clock, + Stopper: stopper, + }, + senderFn, + ) + db := client.NewDB(testutils.MakeAmbientCtx(), factory, clock) + + if err := db.Txn(ctx, func(ctx context.Context, txn *client.Txn) error { + ba := txn.NewBatch() + ba.Get(key1) + ba.Put(key2, "val") + return txn.Run(ctx, ba) + }); err != nil { + t.Fatal(err) + } +} diff --git a/pkg/kv/txn_interceptor_heartbeat.go b/pkg/kv/txn_interceptor_heartbeat.go new file mode 100644 index 000000000000..a665cca068d5 --- /dev/null +++ b/pkg/kv/txn_interceptor_heartbeat.go @@ -0,0 +1,528 @@ +// Copyright 2018 The Cockroach Authors. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or +// implied. See the License for the specific language governing +// permissions and limitations under the License. + +package kv + +import ( + "context" + "sync" + "time" + + opentracing "github.com/opentracing/opentracing-go" + + "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/util/hlc" + "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/cockroachdb/cockroach/pkg/util/stop" +) + +// txnHeartbeat is a txnInterceptor in charge of the txn's heartbeat loop. +// The heartbeat loop is started upon the first write. txnHeartbeat is also in +// charge of prepending a BeginTransaction to the first write batch and possibly +// eliding EndTransaction requests on read-only transactions. +// +// txnHeartbeat should only be used for root transactions; leafs don't perform +// writes and don't need any of the functionality here. +type txnHeartbeat struct { + log.AmbientContext + + wrapped lockedSender + clock *hlc.Clock + heartbeatInterval time.Duration + metrics *TxnMetrics + + // stopper is the TxnCoordSender's stopper. Used to stop the heartbeat loop + // when quiescing. + stopper *stop.Stopper + + // asyncAbortCallbackLocked is called when the heartbeat loop shuts itself + // down because it has detected the transaction to be aborted. The intention + // is to notify the TxnCoordSender to shut itself down. + asyncAbortCallbackLocked func(context.Context) + + // mu contains state protected by the TxnCoordSender's mutex. + mu struct { + sync.Locker + + // txnEnd is closed when the transaction is aborted or committed, terminating + // the heartbeat loop. Nil if the heartbeat loop is not running. + txnEnd chan struct{} + + // txn is a reference to the TxnCoordSender's proto. + txn *roachpb.Transaction + + // finalErr, if set, will be returned by all subsequent SendLocked() calls, + // except rollbacks. + finalErr *roachpb.Error + + // needBeginTxn dictates whether a BeginTxn request is to be prepended to a + // write batch. It starts as set and then gets unset when the BeginTxn is + // sent. It gets reset on epoch increment, as it's possible that the + // retriable error was generated by the BeginTxn batch and the transaction + // record has not been written. + // We could be smarter about not resetting this if there's ever been a + // successful BeginTxn (in which case we know that there is a txn record) + // 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. + needBeginTxn bool + + // everSentBeginTxn is set once a BeginTransactionRequest (out of possibly + // many) was sent to the server. If a BeginTxn was ever sent, then an + // EndTransaction needs to eventually be sent and cannot be elided. + // Note that simply looking at txnEnd == nil to see if a heartbeat loop is + // currently running is not always sufficient for deciding whether an + // EndTransaction can be elided - we want to allow multiple rollback attempts + // to be sent and the first one stops the heartbeat loop. + everSentBeginTxn bool + } +} + +// init initializes the txnHeartbeat. This method exists instead of a +// constructor because txnHeartbeats live in a pool in the TxnCoordSender. +func (h *txnHeartbeat) init( + mu sync.Locker, + txn *roachpb.Transaction, + clock *hlc.Clock, + heartbeatInterval time.Duration, + metrics *TxnMetrics, + stopper *stop.Stopper, + asyncAbortCallbackLocked func(context.Context), +) { + h.stopper = stopper + h.clock = clock + h.heartbeatInterval = heartbeatInterval + h.metrics = metrics + h.mu.Locker = mu + h.mu.txn = txn + h.mu.needBeginTxn = true + h.asyncAbortCallbackLocked = asyncAbortCallbackLocked +} + +// SendLocked is part of the txnInteceptor interface. +func (h *txnHeartbeat) SendLocked( + ctx context.Context, ba roachpb.BatchRequest, +) (*roachpb.BatchResponse, *roachpb.Error) { + // If finalErr is set, we reject everything but rollbacks. + if h.mu.finalErr != nil { + singleRollback := ba.IsSingleEndTransactionRequest() && + !ba.Requests[0].GetInner().(*roachpb.EndTransactionRequest).Commit + if !singleRollback { + return nil, h.mu.finalErr + } + } + + firstWriteIdx, pErr := firstWriteIndex(&ba) + if pErr != nil { + return nil, pErr + } + haveTxnWrite := firstWriteIdx != -1 + et, haveEndTxn := ba.GetArg(roachpb.EndTransaction) + var etReq *roachpb.EndTransactionRequest + if haveEndTxn { + etReq = et.(*roachpb.EndTransactionRequest) + } + + needBeginTxn := haveTxnWrite && h.mu.needBeginTxn + if needBeginTxn { + h.mu.needBeginTxn = false + h.mu.everSentBeginTxn = true + // From now on, all requests need to be checked against the AbortCache on + // the server side. We also conservatively update the current request, + // although I'm not sure if that's necessary. + h.mu.txn.Writing = true + ba.Txn.Writing = true + + // Set txn key based on the key of the first transactional write if not + // already set. If we're in a restart, make sure we keep the anchor key the + // same. + if len(h.mu.txn.Key) == 0 { + anchor := ba.Requests[firstWriteIdx].GetInner().Header().Key + h.mu.txn.Key = anchor + // Put the anchor also in the ba's copy of the txn, since this batch was + // prepared before we had an anchor. + ba.Txn.Key = anchor + } + // Set the key in the begin transaction request to the txn's anchor key. + bt := &roachpb.BeginTransactionRequest{ + RequestHeader: roachpb.RequestHeader{ + Key: h.mu.txn.Key, + }, + } + + // Inject the new request before the first write position, taking care to + // avoid unnecessary allocations. + oldRequests := ba.Requests + ba.Requests = make([]roachpb.RequestUnion, len(ba.Requests)+1) + copy(ba.Requests, oldRequests[:firstWriteIdx]) + ba.Requests[firstWriteIdx].MustSetInner(bt) + copy(ba.Requests[firstWriteIdx+1:], oldRequests[firstWriteIdx:]) + + // Start the heartbeat loop. + // Note that we don't do it for 1PC txns: they only leave intents around on + // retriable errors if the batch has been split between ranges. We consider + // that unlikely enough so we prefer to not pay for a goroutine. + // + // Note that we don't start the heartbeat loop if the loop is already + // running. That can happen because we send BeginTransaction again after + // retriable errors. + if h.mu.txnEnd == nil && !haveEndTxn { + if err := h.startHeartbeatLoopLocked(ctx); err != nil { + h.mu.finalErr = roachpb.NewError(err) + return nil, h.mu.finalErr + } + } + } + + // See if we can elide an EndTxn. We can elide it for read-only transactions. + lastIndex := int32(len(ba.Requests) - 1) + var elideEndTxn bool + if haveEndTxn { + // Are we writing now or have we written in the past? + elideEndTxn = !h.mu.everSentBeginTxn + if elideEndTxn { + ba.Requests = ba.Requests[:lastIndex] + } else { + // If all the writes were part of old epochs, we can turn the commit into + // a rollback. Besides the rollback being potentially cheaper, this + // transformation is important in situations where it's unclear if the txn + // record exist: if it doesn't, then a commit would return a + // TransactionStatusError where a rollback returns success. + if h.mu.needBeginTxn { + log.VEventf(ctx, 2, "Turning commit in rollback. All writes are part of old epochs.") + etReq.Commit = false + } + } + } + + // Forward the request. + // If we've elided the EndTxn and there's no other requests, we can't send an + // empty batch. + var br *roachpb.BatchResponse + if len(ba.Requests) > 0 { + br, pErr = h.wrapped.SendLocked(ctx, ba) + } else { + br = ba.CreateReply() + txn := ba.Txn.Clone() + br.Txn = &txn + } + + // 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:]...) + } + lastIndex-- + // Handle case where inserted begin txn confused an indexed error. + if pErr != nil && pErr.Index != nil { + idx := pErr.Index.Index + if idx == int32(firstWriteIdx) { + // An error was encountered on begin txn; disallow the indexing. + pErr.Index = nil + } else if idx > int32(firstWriteIdx) { + // An error was encountered after begin txn; decrement index. + pErr.SetErrorIndex(idx - 1) + } + } + } + + if pErr != nil { + return nil, pErr + } + + if elideEndTxn { + // Check if the (read-only) txn was pushed above its timestamp. + // Note that we compare the deadline to br.Txn.Timestamp, not + // h.mu.txn.Timestamp; the last batch might have been the pushed one, so br + // has the most up to date timestamp. + if etReq.Deadline != nil && etReq.Deadline.Less(br.Txn.Timestamp) { + return nil, roachpb.NewErrorWithTxn(roachpb.NewTransactionStatusError( + "deadline exceeded before transaction finalization"), br.Txn) + } + // This normally happens on the server and sent back in response + // headers, but this transaction was optimized away. The caller may + // still inspect the transaction struct, so we manually update it + // here to emulate a true transaction. + var status roachpb.TransactionStatus + if etReq.Commit { + status = roachpb.COMMITTED + } else { + status = roachpb.ABORTED + } + if br.Txn == nil { + txn := ba.Txn.Clone() + br.Txn = &txn + } + br.Txn.Status = status + // Synthesize an EndTransactionResponse. + resp := &roachpb.EndTransactionResponse{} + resp.Txn = br.Txn + br.Add(resp) + } + + return br, nil +} + +// setWrapped is part of the txnInteceptor interface. +func (h *txnHeartbeat) setWrapped(wrapped lockedSender) { + h.wrapped = wrapped +} + +// populateMetaLocked is part of the txnInteceptor interface. +func (h *txnHeartbeat) populateMetaLocked(*roachpb.TxnCoordMeta) {} + +// augmentMetaLocked is part of the txnInteceptor interface. +func (h *txnHeartbeat) augmentMetaLocked(roachpb.TxnCoordMeta) {} + +// epochBumpedLocked is part of the txnInteceptor interface. +func (h *txnHeartbeat) epochBumpedLocked() { + h.mu.needBeginTxn = true +} + +// closeLocked is part of the txnInteceptor interface. +func (h *txnHeartbeat) closeLocked() { + // If the heartbeat loop has already finished, there's nothing more to do. + if h.mu.txnEnd == nil { + return + } + close(h.mu.txnEnd) + h.mu.txnEnd = nil +} + +// startHeartbeatLoopLocked starts a heartbeat loop in a different goroutine. +func (h *txnHeartbeat) startHeartbeatLoopLocked(ctx context.Context) error { + if h.mu.txnEnd != nil { + log.Fatal(ctx, "attempting to start a second heartbeat loop ") + } + + log.VEventf(ctx, 2, "coordinator spawns heartbeat loop") + h.mu.txnEnd = make(chan struct{}) + + // Create a new context so that the heartbeat loop doesn't inherit the + // caller's cancelation. + // We want the loop to run in a span linked to the current one, though, so we + // put our span in the new context and expect RunAsyncTask to fork it + // immediately. + hbCtx := h.AnnotateCtx(context.Background()) + hbCtx = opentracing.ContextWithSpan(hbCtx, opentracing.SpanFromContext(ctx)) + + return h.stopper.RunAsyncTask( + hbCtx, "kv.TxnCoordSender: heartbeat loop", func(ctx context.Context) { + h.heartbeatLoop(ctx) + }) +} + +// heartbeatLoop periodically sends a HeartbeatTxn request to the transaction +// record, stopping in the event the transaction is aborted or committed after +// attempting to resolve the intents. +func (h *txnHeartbeat) heartbeatLoop(ctx context.Context) { + var tickChan <-chan time.Time + { + ticker := time.NewTicker(h.heartbeatInterval) + tickChan = ticker.C + defer ticker.Stop() + } + + var finalErr *roachpb.Error + defer func() { + h.mu.Lock() + // Prevent future SendLocked() calls. + if finalErr != nil { + h.mu.finalErr = finalErr + } + if h.mu.txnEnd != nil { + h.mu.txnEnd = nil + } + h.mu.Unlock() + }() + + var closer <-chan struct{} + { + h.mu.Lock() + closer = h.mu.txnEnd + h.mu.Unlock() + if closer == nil { + return + } + } + // Loop with ticker for periodic heartbeats. + for { + select { + case <-tickChan: + if !h.heartbeat(ctx) { + // This error we're generating here should not be seen by clients. Since + // the transaction is aborted, they should be rejected before they reach + // this interceptor. + finalErr = roachpb.NewErrorf("heartbeat failed fatally") + return + } + case <-closer: + // Transaction finished normally. + finalErr = roachpb.NewErrorf("txnHeartbeat already closed") + return + case <-h.stopper.ShouldQuiesce(): + finalErr = roachpb.NewErrorf("node already quiescing") + return + } + } +} + +// heartbeat sends a HeartbeatTxnRequest to the txn record. +// Errors that carry update txn information (e.g. TransactionAbortedError) will +// update the txn. Other errors are swallowed. +// Returns true if heartbeating should continue, false if the transaction is no +// longer Pending and so there's no point in heartbeating further. +func (h *txnHeartbeat) heartbeat(ctx context.Context) bool { + // Like with the TxnCoordSender, the locking here is peculiar. The lock is not + // held continuously throughout this method: we acquire the lock here and + // then, inside the wrapped.Send() call, the interceptor at the bottom of the + // stack will unlock until it receives a response. + h.mu.Lock() + defer h.mu.Unlock() + + // If the txn is no longer pending, there's nothing for us to heartbeat. + // This h.heartbeat() call could have raced with a response that updated the + // status. That response is supposed to have closed the txnHeartbeat. + if h.mu.txn.Status != roachpb.PENDING { + if h.mu.txnEnd != nil { + log.Fatalf(ctx, "txn status: %s, but heartbeat loop hasn't been signaled to stop", h.mu.txn.Status) + } + return false + } + + // Clone the txn in order to put it in the heartbeat request. + txn := h.mu.txn.Clone() + + if txn.Key == nil { + log.Fatalf(ctx, "attempting to heartbeat txn without anchor key: %v", txn) + } + + ba := roachpb.BatchRequest{} + ba.Txn = &txn + + hb := &roachpb.HeartbeatTxnRequest{ + RequestHeader: roachpb.RequestHeader{ + Key: txn.Key, + }, + Now: h.clock.Now(), + } + ba.Add(hb) + + log.VEvent(ctx, 2, "heartbeat") + br, pErr := h.wrapped.SendLocked(ctx, ba) + + var respTxn *roachpb.Transaction + if pErr != nil { + log.VEventf(ctx, 2, "heartbeat failed: %s", pErr) + + // If the heartbeat request arrived to find a missing transaction record + // then we ignore the error. This is possible if the heartbeat loop was + // started before a BeginTxn request succeeds because of ambiguity in the + // first write request's response. + if tse, ok := pErr.GetDetail().(*roachpb.TransactionStatusError); ok && + tse.Reason == roachpb.TransactionStatusError_REASON_TXN_NOT_FOUND { + return true + } + + if pErr.GetTxn() != nil { + // It is not expected for a 2.1 node to return an error with a transaction + // in it. For one, heartbeats are not supposed to return + // TransactionAbortedErrors. + // TODO(andrei): Remove this in 2.2. + respTxn = pErr.GetTxn() + } else { + return true + } + } else { + respTxn = br.Responses[0].GetInner().(*roachpb.HeartbeatTxnResponse).Txn + } + + // Update our txn. In particular, we need to make sure that the client will + // notice when the txn has been aborted (in which case we'll give them an + // error on their next request). + h.mu.txn.Update(respTxn) + if h.mu.txn.Status != roachpb.PENDING { + if h.mu.txn.Status == roachpb.ABORTED { + log.VEventf(ctx, 1, "Heartbeat detected aborted txn. Cleaning up.") + h.abortTxnAsyncLocked(ctx) + } + return false + } + return true +} + +// abortTxnAsyncLocked send an EndTransaction(commmit=false) asynchronously. +// The asyncAbortCallbackLocked callback is also called. +func (h *txnHeartbeat) abortTxnAsyncLocked(ctx context.Context) { + // Stop the heartbeat loop if it is still running. + if h.mu.txnEnd != nil { + close(h.mu.txnEnd) + h.mu.txnEnd = nil + } + + if h.mu.txn.Status != roachpb.ABORTED { + log.Fatalf(ctx, "abortTxnAsyncLocked called for non-aborted txn: %s", h.mu.txn) + } + h.asyncAbortCallbackLocked(ctx) + txn := h.mu.txn.Clone() + + // NB: We use context.Background() here because we don't want a canceled + // context to interrupt the aborting. + ctx = h.AnnotateCtx(context.Background()) + + // Construct a batch with an EndTransaction request. + ba := roachpb.BatchRequest{} + ba.Header = roachpb.Header{Txn: &txn} + ba.Add(&roachpb.EndTransactionRequest{ + Commit: false, + // Resolved intents should maintain an abort span entry to prevent + // concurrent requests from failing to notice the transaction was aborted. + Poison: true, + }) + + log.VEventf(ctx, 2, "async abort for txn: %s", txn) + if err := h.stopper.RunAsyncTask( + ctx, "txnHeartbeat: aborting txn", func(ctx context.Context) { + // Send the abort request through the interceptor stack. This is important + // because we need the txnIntentCollector to append intents to the + // EndTransaction request. + h.mu.Lock() + defer h.mu.Unlock() + _, pErr := h.wrapped.SendLocked(ctx, ba) + if pErr != nil { + log.VErrEventf(ctx, 1, "async abort failed for %s: %s ", txn, pErr) + } + }, + ); err != nil { + log.Warning(ctx, err) + } +} + +// firstWriteIndex returns the index of the first transactional write in the +// BatchRequest. Returns -1 if the batch has not intention to write. It also +// verifies that if an EndTransactionRequest is included, then it is the last +// request in the batch. +func firstWriteIndex(ba *roachpb.BatchRequest) (int, *roachpb.Error) { + for i, ru := range ba.Requests { + args := ru.GetInner() + if i < len(ba.Requests)-1 /* if not last*/ { + if _, ok := args.(*roachpb.EndTransactionRequest); ok { + return -1, roachpb.NewErrorf("%s sent as non-terminal call", args.Method()) + } + } + if roachpb.IsTransactionWrite(args) { + return i, nil + } + } + return -1, nil +} diff --git a/pkg/kv/txn_interceptor_intent_collector.go b/pkg/kv/txn_interceptor_intent_collector.go index 48114320b966..6885f70044b0 100644 --- a/pkg/kv/txn_interceptor_intent_collector.go +++ b/pkg/kv/txn_interceptor_intent_collector.go @@ -95,12 +95,6 @@ func (ic *txnIntentCollector) SendLocked( et.IntentSpans, distinct = roachpb.MergeSpans(et.IntentSpans) ba.Header.DistinctSpans = distinct && distinctSpans - if len(et.IntentSpans) == 0 { - // If there aren't any intents, then there's factually no - // transaction to end. Read-only txns have all of their state - // in the client. - return nil, roachpb.NewErrorf("cannot commit a read-only transaction") - } if log.V(3) { for _, intent := range et.IntentSpans { log.Infof(ctx, "intent: [%s,%s)", intent.Key, intent.EndKey) diff --git a/pkg/kv/txn_interceptor_metrics.go b/pkg/kv/txn_interceptor_metrics.go new file mode 100644 index 000000000000..901ea0b92cc4 --- /dev/null +++ b/pkg/kv/txn_interceptor_metrics.go @@ -0,0 +1,101 @@ +// Copyright 2018 The Cockroach Authors. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or +// implied. See the License for the specific language governing +// permissions and limitations under the License. + +package kv + +import ( + "context" + + "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/util/hlc" +) + +// txnMetrics is a txnInterceptor in charge of updating some metrics in response +// of transactions going through it. +type txnMetrics struct { + wrapped lockedSender + metrics *TxnMetrics + clock *hlc.Clock + + txn *roachpb.Transaction + txnStartNanos int64 + onePCCommit bool + closed bool +} + +// init initializes the txnMetrics. This method exists instead of a constructor +// because txnMetrics lives in a pool in the TxnCoordSender. +func (m *txnMetrics) init(txn *roachpb.Transaction, clock *hlc.Clock, metrics *TxnMetrics) { + m.clock = clock + m.metrics = metrics + m.txn = txn +} + +// SendLocked is part of the txnInterceptor interface. +func (m *txnMetrics) SendLocked( + ctx context.Context, ba roachpb.BatchRequest, +) (*roachpb.BatchResponse, *roachpb.Error) { + _, hasBegin := ba.GetArg(roachpb.BeginTransaction) + et, hasEnd := ba.GetArg(roachpb.EndTransaction) + m.onePCCommit = hasBegin && hasEnd && et.(*roachpb.EndTransactionRequest).Commit + + if hasBegin { + m.txnStartNanos = m.clock.PhysicalNow() + } + + return m.wrapped.SendLocked(ctx, ba) +} + +// setWrapped is part of the txnInterceptor interface. +func (m *txnMetrics) setWrapped(wrapped lockedSender) { m.wrapped = wrapped } + +// populateMetaLocked is part of the txnInterceptor interface. +func (*txnMetrics) populateMetaLocked(*roachpb.TxnCoordMeta) {} + +// augmentMetaLocked is part of the txnInterceptor interface. +func (*txnMetrics) augmentMetaLocked(roachpb.TxnCoordMeta) {} + +// epochBumpedLocked is part of the txnInterceptor interface. +func (*txnMetrics) epochBumpedLocked() {} + +// closeLocked is part of the txnInterceptor interface. +func (m *txnMetrics) closeLocked() { + if m.closed { + return + } + m.closed = true + + if m.onePCCommit { + m.metrics.Commits1PC.Inc(1) + } + + duration := m.clock.PhysicalNow() - m.txnStartNanos + restarts := int64(m.txn.Epoch) + status := m.txn.Status + + m.metrics.Durations.RecordValue(duration) + m.metrics.Restarts.RecordValue(restarts) + switch status { + case roachpb.ABORTED: + m.metrics.Aborts.Inc(1) + case roachpb.PENDING: + // NOTE(andrei): Getting a PENDING status here is possible when this + // interceptor is closed without a rollback ever succeeding. + // We increment the Aborts metric nevertheless; not sure how these + // transactions should be accounted. + m.metrics.Aborts.Inc(1) + case roachpb.COMMITTED: + m.metrics.Commits.Inc(1) + } +} diff --git a/pkg/kv/txn_interceptor_sequence_nums.go b/pkg/kv/txn_interceptor_sequence_nums.go new file mode 100644 index 000000000000..4fa4401f4047 --- /dev/null +++ b/pkg/kv/txn_interceptor_sequence_nums.go @@ -0,0 +1,91 @@ +// Copyright 2018 The Cockroach Authors. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or +// implied. See the License for the specific language governing +// permissions and limitations under the License. + +package kv + +import ( + "context" + + "github.com/cockroachdb/cockroach/pkg/roachpb" +) + +// txnSeqNumAllocator is a txnInterceptor in charge of allocating sequence +// numbers to all the individual requests in batches. +// +// The sequence number is used for replay and reordering protection. At the +// Store, a sequence number less than or equal to the last observed one (on a +// given key) incurs a transaction restart (if the request is transactional). +// This semantic could be adjusted in the future to provide idempotency for +// replays and re-issues. However, a side effect of providing this property is +// that reorder protection would no longer be provided by the counter, so +// ordering guarantees between requests within the same transaction would need +// to be strengthened elsewhere (e.g. by the transport layer). +type txnSeqNumAllocator struct { + wrapped lockedSender + + seqNumCounter int32 + + // commandCount indicates how many requests have been sent through + // this transaction. Reset on retryable txn errors. + // TODO(andrei): let's get rid of this. It should be maintained + // in the SQL level. + commandCount int32 +} + +// SendLocked is part of the txnInterceptor interface. +func (s *txnSeqNumAllocator) SendLocked( + ctx context.Context, ba roachpb.BatchRequest, +) (*roachpb.BatchResponse, *roachpb.Error) { + for _, ru := range ba.Requests { + // Heartbeats don't get seq nums. They don't need them and they'd cause + // large numbers to be used by other requests. + if ru.GetHeartbeatTxn() != nil { + continue + } + + s.seqNumCounter++ + + oldHeader := ru.GetInner().Header() + oldHeader.Sequence = s.seqNumCounter + ru.GetInner().SetHeader(oldHeader) + } + // For 2.0 compatibility. + ba.Txn.Sequence = s.seqNumCounter + + s.commandCount += int32(len(ba.Requests)) + + return s.wrapped.SendLocked(ctx, ba) +} + +// setWrapped is part of the txnInterceptor interface. +func (s *txnSeqNumAllocator) setWrapped(wrapped lockedSender) { s.wrapped = wrapped } + +// populateMetaLocked is part of the txnInterceptor interface. +func (s *txnSeqNumAllocator) populateMetaLocked(meta *roachpb.TxnCoordMeta) { + meta.CommandCount = s.commandCount +} + +// augmentMetaLocked is part of the txnInterceptor interface. +func (s *txnSeqNumAllocator) augmentMetaLocked(meta roachpb.TxnCoordMeta) { + s.commandCount += meta.CommandCount +} + +// epochBumpedLocked is part of the txnInterceptor interface. +func (s *txnSeqNumAllocator) epochBumpedLocked() { + s.seqNumCounter = 0 + s.commandCount = 0 +} + +// closeLocked is part of the txnInterceptor interface. +func (*txnSeqNumAllocator) closeLocked() {} diff --git a/pkg/kv/txn_interceptor_span_refresher.go b/pkg/kv/txn_interceptor_span_refresher.go index 05cc4789833c..3022eda711ea 100644 --- a/pkg/kv/txn_interceptor_span_refresher.go +++ b/pkg/kv/txn_interceptor_span_refresher.go @@ -93,8 +93,12 @@ func (sr *txnSpanRefresher) SendLocked( } maxAttempts := maxTxnRefreshAttempts - if knob := sr.knobs.MaxTxnRefreshAttempts; knob > 0 { - maxAttempts = knob + if knob := sr.knobs.MaxTxnRefreshAttempts; knob != 0 { + if knob == -1 { + maxAttempts = 0 + } else { + maxAttempts = knob + } } // Send through wrapped lockedSender. Unlocks while sending then re-locks. diff --git a/pkg/kv/txn_test.go b/pkg/kv/txn_test.go index 61afe285033b..d6fd2e60c031 100644 --- a/pkg/kv/txn_test.go +++ b/pkg/kv/txn_test.go @@ -161,13 +161,13 @@ func TestSnapshotIsolationIncrement(t *testing.T) { t.Fatal(err) } - if txn.Proto().Epoch == 0 { + if txn.Epoch() == 0 { close(start) // let someone write into our future // When they're done writing, increment. if err := <-done; err != nil { t.Fatal(err) } - } else if txn.Proto().Epoch > 1 { + } else if txn.Epoch() > 1 { t.Fatal("should experience just one restart") } @@ -181,19 +181,21 @@ func TestSnapshotIsolationIncrement(t *testing.T) { // before anything else happened in the concurrent writer // goroutine). The second iteration of the txn should read the // correct value and commit. - if txn.Proto().Epoch == 0 && !txn.Proto().OrigTimestamp.Less(txn.Proto().Timestamp) { - t.Fatalf("expected orig timestamp less than timestamp: %s", txn.Proto()) + proto := txn.Serialize() + if txn.Epoch() == 0 && !proto.OrigTimestamp.Less(proto.Timestamp) { + t.Fatalf("expected orig timestamp less than timestamp: %s", proto) } ir, err := txn.Inc(ctx, key, 1) if err != nil { t.Fatal(err) } - if vi := ir.ValueInt(); vi != int64(txn.Proto().Epoch+1) { - t.Errorf("expected %d; got %d", txn.Proto().Epoch+1, vi) + proto = txn.Serialize() + if vi := ir.ValueInt(); vi != int64(proto.Epoch+1) { + t.Errorf("expected %d; got %d", proto.Epoch+1, vi) } // Verify that the WriteTooOld boolean is set on the txn. - if (txn.Proto().Epoch == 0) != txn.Proto().WriteTooOld { - t.Fatalf("expected write too old=%t; got %t", (txn.Proto().Epoch == 0), txn.Proto().WriteTooOld) + if (txn.Epoch() == 0) != proto.WriteTooOld { + t.Fatalf("expected write too old=%t; got %t", (proto.Epoch == 0), proto.WriteTooOld) } return nil }); err != nil { @@ -233,13 +235,13 @@ func TestSnapshotIsolationLostUpdate(t *testing.T) { if err != nil { t.Fatal(err) } - if txn.Proto().Epoch == 0 { + if txn.Epoch() == 0 { close(start) // let someone write into our future // When they're done, write based on what we read. if err := <-done; err != nil { t.Fatal(err) } - } else if txn.Proto().Epoch > 1 { + } else if txn.Epoch() > 1 { t.Fatal("should experience just one restart") } @@ -253,8 +255,9 @@ func TestSnapshotIsolationLostUpdate(t *testing.T) { } } // Verify that the WriteTooOld boolean is set on the txn. - if (txn.Proto().Epoch == 0) != txn.Proto().WriteTooOld { - t.Fatalf("expected write too old set (%t): got %t", (txn.Proto().Epoch == 0), txn.Proto().WriteTooOld) + proto := txn.Serialize() + if (txn.Epoch() == 0) != proto.WriteTooOld { + t.Fatalf("expected write too old set (%t): got %t", (txn.Epoch() == 0), proto.WriteTooOld) } return nil }); err != nil { @@ -325,7 +328,7 @@ func TestPriorityRatchetOnAbortOrPush(t *testing.T) { if iteration == 1 { // Verify our priority has ratcheted to one less than the pusher's priority expPri := int32(roachpb.MaxTxnPriority - 1) - if pri := txn.Proto().Priority; pri != expPri { + if pri := txn.Serialize().Priority; pri != expPri { t.Fatalf("%s: expected priority on retry to ratchet to %d; got %d", key, expPri, pri) } return nil diff --git a/pkg/kv/txnstate_string.go b/pkg/kv/txnstate_string.go new file mode 100644 index 000000000000..13bdd54e3c80 --- /dev/null +++ b/pkg/kv/txnstate_string.go @@ -0,0 +1,16 @@ +// Code generated by "stringer -type=txnState"; DO NOT EDIT. + +package kv + +import "strconv" + +const _txnState_name = "txnPendingtxnErrortxnFinalized" + +var _txnState_index = [...]uint8{0, 10, 18, 30} + +func (i txnState) String() string { + if i < 0 || i >= txnState(len(_txnState_index)-1) { + return "txnState(" + strconv.FormatInt(int64(i), 10) + ")" + } + return _txnState_name[_txnState_index[i]:_txnState_index[i+1]] +} diff --git a/pkg/roachpb/api.pb.go b/pkg/roachpb/api.pb.go index d45543402760..ae9ec9bdb0bd 100644 --- a/pkg/roachpb/api.pb.go +++ b/pkg/roachpb/api.pb.go @@ -154,7 +154,6 @@ StoreNotFoundError UnhandledRetryableError HandledRetryableTxnError - TxnPrevAttemptError TxnAlreadyEncounteredErrorError IntegerOverflowError MixedSuccessError diff --git a/pkg/roachpb/batch_generated.go b/pkg/roachpb/batch_generated.go index 99218f73e4d6..3b9039534398 100644 --- a/pkg/roachpb/batch_generated.go +++ b/pkg/roachpb/batch_generated.go @@ -56,8 +56,6 @@ func (ru ErrorDetail) GetInner() error { return t.StoreNotFound case *ErrorDetail_HandledRetryableTxnError: return t.HandledRetryableTxnError - case *ErrorDetail_TxnAbortedAsyncErr: - return t.TxnAbortedAsyncErr case *ErrorDetail_IntegerOverflow: return t.IntegerOverflow case *ErrorDetail_UnsupportedRequest: @@ -299,8 +297,6 @@ func (ru *ErrorDetail) SetInner(r error) bool { union = &ErrorDetail_StoreNotFound{t} case *HandledRetryableTxnError: union = &ErrorDetail_HandledRetryableTxnError{t} - case *TxnPrevAttemptError: - union = &ErrorDetail_TxnAbortedAsyncErr{t} case *IntegerOverflowError: union = &ErrorDetail_IntegerOverflow{t} case *UnsupportedRequestError: diff --git a/pkg/roachpb/errors.go b/pkg/roachpb/errors.go index 3a6e35ca38e8..ff1c44718b9f 100644 --- a/pkg/roachpb/errors.go +++ b/pkg/roachpb/errors.go @@ -608,16 +608,6 @@ func (e *StoreNotFoundError) message(_ *Error) string { var _ ErrorDetailInterface = &StoreNotFoundError{} -func (e *TxnPrevAttemptError) Error() string { - return e.message(nil) -} - -func (*TxnPrevAttemptError) message(_ *Error) string { - return "response meant for previous incarnation of transaction" -} - -var _ ErrorDetailInterface = &TxnPrevAttemptError{} - func (e *TxnAlreadyEncounteredErrorError) Error() string { return e.message(nil) } diff --git a/pkg/roachpb/errors.pb.go b/pkg/roachpb/errors.pb.go index 1f9fa4d82c1d..987f920ae4fc 100644 --- a/pkg/roachpb/errors.pb.go +++ b/pkg/roachpb/errors.pb.go @@ -518,18 +518,6 @@ func (m *HandledRetryableTxnError) String() string { return proto.Com func (*HandledRetryableTxnError) ProtoMessage() {} func (*HandledRetryableTxnError) Descriptor() ([]byte, []int) { return fileDescriptorErrors, []int{23} } -// A TxnPrevAttemptError indicates a batch response for a request sent through -// client Txn was found that was meant for a previous incarnation of the -// transaction. This can happen in cases where concurrent requests are made -// for a Transaction and one of the requests results in a Txn abort. -type TxnPrevAttemptError struct { -} - -func (m *TxnPrevAttemptError) Reset() { *m = TxnPrevAttemptError{} } -func (m *TxnPrevAttemptError) String() string { return proto.CompactTextString(m) } -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. @@ -540,7 +528,7 @@ func (m *TxnAlreadyEncounteredErrorError) Reset() { *m = TxnAlreadyEncou func (m *TxnAlreadyEncounteredErrorError) String() string { return proto.CompactTextString(m) } func (*TxnAlreadyEncounteredErrorError) ProtoMessage() {} func (*TxnAlreadyEncounteredErrorError) Descriptor() ([]byte, []int) { - return fileDescriptorErrors, []int{25} + return fileDescriptorErrors, []int{24} } // An IntegerOverflowError indicates that an operation was aborted because @@ -554,7 +542,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{26} } +func (*IntegerOverflowError) Descriptor() ([]byte, []int) { return fileDescriptorErrors, []int{25} } // A MixedSuccessError indicates that some portion of the batch // request may have succeeded, but the batch as a whole failed with @@ -566,7 +554,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{27} } +func (*MixedSuccessError) Descriptor() ([]byte, []int) { return fileDescriptorErrors, []int{26} } // A BatchTimestampBeforeGCError indicates that a request's timestamp was // before the GC threshold. @@ -579,7 +567,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{28} + return fileDescriptorErrors, []int{27} } // An IntentMissingError indicates that a QueryIntent request expected @@ -593,7 +581,7 @@ type IntentMissingError struct { func (m *IntentMissingError) Reset() { *m = IntentMissingError{} } func (m *IntentMissingError) String() string { return proto.CompactTextString(m) } func (*IntentMissingError) ProtoMessage() {} -func (*IntentMissingError) Descriptor() ([]byte, []int) { return fileDescriptorErrors, []int{29} } +func (*IntentMissingError) Descriptor() ([]byte, []int) { return fileDescriptorErrors, []int{28} } // A MergeInProgressError indicates that the request could not be completed // because the replica is being merged into its left-hand neighbor. The request @@ -606,7 +594,7 @@ type MergeInProgressError struct { func (m *MergeInProgressError) Reset() { *m = MergeInProgressError{} } func (m *MergeInProgressError) String() string { return proto.CompactTextString(m) } func (*MergeInProgressError) ProtoMessage() {} -func (*MergeInProgressError) Descriptor() ([]byte, []int) { return fileDescriptorErrors, []int{30} } +func (*MergeInProgressError) Descriptor() ([]byte, []int) { return fileDescriptorErrors, []int{29} } // ErrorDetail is a union type containing all available errors. type ErrorDetail struct { @@ -633,7 +621,6 @@ type ErrorDetail struct { // *ErrorDetail_AmbiguousResult // *ErrorDetail_StoreNotFound // *ErrorDetail_HandledRetryableTxnError - // *ErrorDetail_TxnAbortedAsyncErr // *ErrorDetail_IntegerOverflow // *ErrorDetail_UnsupportedRequest // *ErrorDetail_MixedSuccess @@ -647,7 +634,7 @@ type ErrorDetail struct { 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{31} } +func (*ErrorDetail) Descriptor() ([]byte, []int) { return fileDescriptorErrors, []int{30} } type isErrorDetail_Value interface { isErrorDetail_Value() @@ -722,9 +709,6 @@ type ErrorDetail_StoreNotFound struct { type ErrorDetail_HandledRetryableTxnError struct { HandledRetryableTxnError *HandledRetryableTxnError `protobuf:"bytes,28,opt,name=handled_retryable_txn_error,json=handledRetryableTxnError,oneof"` } -type ErrorDetail_TxnAbortedAsyncErr struct { - TxnAbortedAsyncErr *TxnPrevAttemptError `protobuf:"bytes,30,opt,name=txn_aborted_async_err,json=txnAbortedAsyncErr,oneof"` -} type ErrorDetail_IntegerOverflow struct { IntegerOverflow *IntegerOverflowError `protobuf:"bytes,31,opt,name=integer_overflow,json=integerOverflow,oneof"` } @@ -769,7 +753,6 @@ func (*ErrorDetail_ReplicaTooOld) isErrorDetail_Value() {} func (*ErrorDetail_AmbiguousResult) isErrorDetail_Value() {} func (*ErrorDetail_StoreNotFound) isErrorDetail_Value() {} func (*ErrorDetail_HandledRetryableTxnError) isErrorDetail_Value() {} -func (*ErrorDetail_TxnAbortedAsyncErr) isErrorDetail_Value() {} func (*ErrorDetail_IntegerOverflow) isErrorDetail_Value() {} func (*ErrorDetail_UnsupportedRequest) isErrorDetail_Value() {} func (*ErrorDetail_MixedSuccess) isErrorDetail_Value() {} @@ -939,13 +922,6 @@ func (m *ErrorDetail) GetHandledRetryableTxnError() *HandledRetryableTxnError { return nil } -func (m *ErrorDetail) GetTxnAbortedAsyncErr() *TxnPrevAttemptError { - if x, ok := m.GetValue().(*ErrorDetail_TxnAbortedAsyncErr); ok { - return x.TxnAbortedAsyncErr - } - return nil -} - func (m *ErrorDetail) GetIntegerOverflow() *IntegerOverflowError { if x, ok := m.GetValue().(*ErrorDetail_IntegerOverflow); ok { return x.IntegerOverflow @@ -1020,7 +996,6 @@ func (*ErrorDetail) XXX_OneofFuncs() (func(msg proto.Message, b *proto.Buffer) e (*ErrorDetail_AmbiguousResult)(nil), (*ErrorDetail_StoreNotFound)(nil), (*ErrorDetail_HandledRetryableTxnError)(nil), - (*ErrorDetail_TxnAbortedAsyncErr)(nil), (*ErrorDetail_IntegerOverflow)(nil), (*ErrorDetail_UnsupportedRequest)(nil), (*ErrorDetail_MixedSuccess)(nil), @@ -1145,11 +1120,6 @@ func _ErrorDetail_OneofMarshaler(msg proto.Message, b *proto.Buffer) error { if err := b.EncodeMessage(x.HandledRetryableTxnError); err != nil { return err } - case *ErrorDetail_TxnAbortedAsyncErr: - _ = b.EncodeVarint(30<<3 | proto.WireBytes) - if err := b.EncodeMessage(x.TxnAbortedAsyncErr); err != nil { - return err - } case *ErrorDetail_IntegerOverflow: _ = b.EncodeVarint(31<<3 | proto.WireBytes) if err := b.EncodeMessage(x.IntegerOverflow); err != nil { @@ -1371,14 +1341,6 @@ func _ErrorDetail_OneofUnmarshaler(msg proto.Message, tag, wire int, b *proto.Bu err := b.DecodeMessage(msg) m.Value = &ErrorDetail_HandledRetryableTxnError{msg} return true, err - case 30: // value.txn_aborted_async_err - if wire != proto.WireBytes { - return true, proto.ErrInternalBadWireType - } - msg := new(TxnPrevAttemptError) - err := b.DecodeMessage(msg) - m.Value = &ErrorDetail_TxnAbortedAsyncErr{msg} - return true, err case 31: // value.integer_overflow if wire != proto.WireBytes { return true, proto.ErrInternalBadWireType @@ -1554,11 +1516,6 @@ func _ErrorDetail_OneofSizer(msg proto.Message) (n int) { n += proto.SizeVarint(28<<3 | proto.WireBytes) n += proto.SizeVarint(uint64(s)) n += s - case *ErrorDetail_TxnAbortedAsyncErr: - s := proto.Size(x.TxnAbortedAsyncErr) - n += proto.SizeVarint(30<<3 | proto.WireBytes) - n += proto.SizeVarint(uint64(s)) - n += s case *ErrorDetail_IntegerOverflow: s := proto.Size(x.IntegerOverflow) n += proto.SizeVarint(31<<3 | proto.WireBytes) @@ -1611,7 +1568,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{32} } +func (*ErrPosition) Descriptor() ([]byte, []int) { return fileDescriptorErrors, []int{31} } // Error is a generic representation including a string message // and information about retryability. @@ -1641,7 +1598,7 @@ type Error struct { func (m *Error) Reset() { *m = Error{} } func (*Error) ProtoMessage() {} -func (*Error) Descriptor() ([]byte, []int) { return fileDescriptorErrors, []int{33} } +func (*Error) Descriptor() ([]byte, []int) { return fileDescriptorErrors, []int{32} } func init() { proto.RegisterType((*NotLeaseHolderError)(nil), "cockroach.roachpb.NotLeaseHolderError") @@ -1668,7 +1625,6 @@ func init() { proto.RegisterType((*StoreNotFoundError)(nil), "cockroach.roachpb.StoreNotFoundError") 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") @@ -2280,27 +2236,6 @@ func (this *HandledRetryableTxnError) Equal(that interface{}) bool { } return true } -func (this *TxnPrevAttemptError) Equal(that interface{}) bool { - if that == nil { - return this == nil - } - - that1, ok := that.(*TxnPrevAttemptError) - if !ok { - that2, ok := that.(TxnPrevAttemptError) - if ok { - that1 = &that2 - } else { - return false - } - } - if that1 == nil { - return this == nil - } else if this == nil { - return false - } - return true -} func (this *TxnAlreadyEncounteredErrorError) Equal(that interface{}) bool { if that == nil { return this == nil @@ -3006,30 +2941,6 @@ func (this *ErrorDetail_HandledRetryableTxnError) Equal(that interface{}) bool { } return true } -func (this *ErrorDetail_TxnAbortedAsyncErr) Equal(that interface{}) bool { - if that == nil { - return this == nil - } - - that1, ok := that.(*ErrorDetail_TxnAbortedAsyncErr) - if !ok { - that2, ok := that.(ErrorDetail_TxnAbortedAsyncErr) - if ok { - that1 = &that2 - } else { - return false - } - } - if that1 == nil { - return this == nil - } else if this == nil { - return false - } - if !this.TxnAbortedAsyncErr.Equal(that1.TxnAbortedAsyncErr) { - return false - } - return true -} func (this *ErrorDetail_IntegerOverflow) Equal(that interface{}) bool { if that == nil { return this == nil @@ -3944,24 +3855,6 @@ func (m *HandledRetryableTxnError) MarshalTo(dAtA []byte) (int, error) { return i, nil } -func (m *TxnPrevAttemptError) 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 *TxnPrevAttemptError) MarshalTo(dAtA []byte) (int, error) { - var i int - _ = i - var l int - _ = l - return i, nil -} - func (m *TxnAlreadyEncounteredErrorError) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) @@ -4465,22 +4358,6 @@ func (m *ErrorDetail_HandledRetryableTxnError) MarshalTo(dAtA []byte) (int, erro } return i, nil } -func (m *ErrorDetail_TxnAbortedAsyncErr) MarshalTo(dAtA []byte) (int, error) { - i := 0 - if m.TxnAbortedAsyncErr != nil { - dAtA[i] = 0xf2 - i++ - dAtA[i] = 0x1 - i++ - i = encodeVarintErrors(dAtA, i, uint64(m.TxnAbortedAsyncErr.Size())) - n46, err := m.TxnAbortedAsyncErr.MarshalTo(dAtA[i:]) - if err != nil { - return 0, err - } - i += n46 - } - return i, nil -} func (m *ErrorDetail_IntegerOverflow) MarshalTo(dAtA []byte) (int, error) { i := 0 if m.IntegerOverflow != nil { @@ -4489,11 +4366,11 @@ func (m *ErrorDetail_IntegerOverflow) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintErrors(dAtA, i, uint64(m.IntegerOverflow.Size())) - n47, err := m.IntegerOverflow.MarshalTo(dAtA[i:]) + n46, err := m.IntegerOverflow.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n47 + i += n46 } return i, nil } @@ -4505,11 +4382,11 @@ func (m *ErrorDetail_UnsupportedRequest) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintErrors(dAtA, i, uint64(m.UnsupportedRequest.Size())) - n48, err := m.UnsupportedRequest.MarshalTo(dAtA[i:]) + n47, err := m.UnsupportedRequest.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n48 + i += n47 } return i, nil } @@ -4521,11 +4398,11 @@ func (m *ErrorDetail_MixedSuccess) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintErrors(dAtA, i, uint64(m.MixedSuccess.Size())) - n49, err := m.MixedSuccess.MarshalTo(dAtA[i:]) + n48, err := m.MixedSuccess.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n49 + i += n48 } return i, nil } @@ -4537,11 +4414,11 @@ func (m *ErrorDetail_TimestampBefore) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintErrors(dAtA, i, uint64(m.TimestampBefore.Size())) - n50, err := m.TimestampBefore.MarshalTo(dAtA[i:]) + n49, err := m.TimestampBefore.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n50 + i += n49 } return i, nil } @@ -4553,11 +4430,11 @@ func (m *ErrorDetail_TxnAlreadyEncounteredError) MarshalTo(dAtA []byte) (int, er dAtA[i] = 0x2 i++ i = encodeVarintErrors(dAtA, i, uint64(m.TxnAlreadyEncounteredError.Size())) - n51, err := m.TxnAlreadyEncounteredError.MarshalTo(dAtA[i:]) + n50, err := m.TxnAlreadyEncounteredError.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n51 + i += n50 } return i, nil } @@ -4569,11 +4446,11 @@ func (m *ErrorDetail_IntentMissing) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintErrors(dAtA, i, uint64(m.IntentMissing.Size())) - n52, err := m.IntentMissing.MarshalTo(dAtA[i:]) + n51, err := m.IntentMissing.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n52 + i += n51 } return i, nil } @@ -4585,11 +4462,11 @@ func (m *ErrorDetail_MergeInProgress) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintErrors(dAtA, i, uint64(m.MergeInProgress.Size())) - n53, err := m.MergeInProgress.MarshalTo(dAtA[i:]) + n52, err := m.MergeInProgress.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n53 + i += n52 } return i, nil } @@ -4640,11 +4517,11 @@ func (m *Error) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x22 i++ i = encodeVarintErrors(dAtA, i, uint64(m.UnexposedTxn.Size())) - n54, err := m.UnexposedTxn.MarshalTo(dAtA[i:]) + n53, err := m.UnexposedTxn.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n54 + i += n53 } dAtA[i] = 0x28 i++ @@ -4652,29 +4529,29 @@ func (m *Error) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x32 i++ i = encodeVarintErrors(dAtA, i, uint64(m.Detail.Size())) - n55, err := m.Detail.MarshalTo(dAtA[i:]) + n54, err := m.Detail.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n55 + i += n54 if m.Index != nil { dAtA[i] = 0x3a i++ i = encodeVarintErrors(dAtA, i, uint64(m.Index.Size())) - n56, err := m.Index.MarshalTo(dAtA[i:]) + n55, err := m.Index.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n56 + i += n55 } dAtA[i] = 0x42 i++ i = encodeVarintErrors(dAtA, i, uint64(m.Now.Size())) - n57, err := m.Now.MarshalTo(dAtA[i:]) + n56, err := m.Now.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n57 + i += n56 return i, nil } @@ -4922,12 +4799,6 @@ func (m *HandledRetryableTxnError) Size() (n int) { return n } -func (m *TxnPrevAttemptError) Size() (n int) { - var l int - _ = l - return n -} - func (m *TxnAlreadyEncounteredErrorError) Size() (n int) { var l int _ = l @@ -5189,15 +5060,6 @@ func (m *ErrorDetail_HandledRetryableTxnError) Size() (n int) { } return n } -func (m *ErrorDetail_TxnAbortedAsyncErr) Size() (n int) { - var l int - _ = l - if m.TxnAbortedAsyncErr != nil { - l = m.TxnAbortedAsyncErr.Size() - n += 2 + l + sovErrors(uint64(l)) - } - return n -} func (m *ErrorDetail_IntegerOverflow) Size() (n int) { var l int _ = l @@ -7525,56 +7387,6 @@ func (m *HandledRetryableTxnError) Unmarshal(dAtA []byte) error { } return nil } -func (m *TxnPrevAttemptError) 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: TxnPrevAttemptError: wiretype end group for non-group") - } - if fieldNum <= 0 { - return fmt.Errorf("proto: TxnPrevAttemptError: 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 *TxnAlreadyEncounteredErrorError) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 @@ -8803,38 +8615,6 @@ func (m *ErrorDetail) Unmarshal(dAtA []byte) error { } m.Value = &ErrorDetail_HandledRetryableTxnError{v} iNdEx = postIndex - case 30: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field TxnAbortedAsyncErr", wireType) - } - var msglen int - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowErrors - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - msglen |= (int(b) & 0x7F) << shift - if b < 0x80 { - break - } - } - if msglen < 0 { - return ErrInvalidLengthErrors - } - postIndex := iNdEx + msglen - if postIndex > l { - return io.ErrUnexpectedEOF - } - v := &TxnPrevAttemptError{} - if err := v.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { - return err - } - m.Value = &ErrorDetail_TxnAbortedAsyncErr{v} - iNdEx = postIndex case 31: if wireType != 2 { return fmt.Errorf("proto: wrong wireType = %d for field IntegerOverflow", wireType) @@ -9500,164 +9280,161 @@ var ( func init() { proto.RegisterFile("roachpb/errors.proto", fileDescriptorErrors) } var fileDescriptorErrors = []byte{ - // 2529 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x9c, 0x58, 0xcb, 0x6f, 0x23, 0x59, - 0xd5, 0xb7, 0x63, 0x27, 0x4e, 0x8e, 0xf3, 0x70, 0x6e, 0xa7, 0x33, 0xd5, 0xe9, 0x89, 0x93, 0x49, - 0xcf, 0x7c, 0xd3, 0x33, 0x9f, 0x48, 0x50, 0x86, 0x01, 0x4d, 0xd3, 0x2c, 0xec, 0xd8, 0x69, 0xbb, - 0x93, 0xd8, 0xa1, 0xe2, 0xd0, 0x2f, 0x50, 0xa9, 0x52, 0x75, 0xe3, 0x14, 0x6d, 0x57, 0x99, 0x5b, - 0xb7, 0x12, 0x67, 0xc7, 0x92, 0x25, 0xec, 0xd8, 0x31, 0x12, 0x2b, 0xc4, 0x16, 0xcd, 0x06, 0x89, - 0x75, 0x2f, 0x59, 0x22, 0x24, 0x22, 0x08, 0x1b, 0x24, 0xfe, 0x83, 0x5e, 0x20, 0x74, 0x1f, 0xf5, - 0xb0, 0x5d, 0xe5, 0x36, 0xbd, 0xb2, 0xeb, 0x9c, 0x73, 0x7f, 0xf7, 0xdc, 0xc7, 0x39, 0xe7, 0x77, - 0x2e, 0xac, 0x10, 0x47, 0x37, 0x2e, 0x7a, 0x67, 0x3b, 0x98, 0x10, 0x87, 0xb8, 0xdb, 0x3d, 0xe2, - 0x50, 0x07, 0x2d, 0x1b, 0x8e, 0xf1, 0x9a, 0x6b, 0xb6, 0xa5, 0x7e, 0x0d, 0xf9, 0x86, 0xa6, 0x4e, - 0x75, 0x61, 0xb6, 0xb6, 0xea, 0xcb, 0xba, 0x98, 0xea, 0x11, 0xf9, 0x03, 0x97, 0x3a, 0x44, 0x6f, - 0xe3, 0x1d, 0x6c, 0xb7, 0x2d, 0xdb, 0xff, 0x61, 0x76, 0x97, 0x86, 0xf1, 0x85, 0x34, 0x52, 0x3c, - 0x6a, 0x75, 0x76, 0x2e, 0x3a, 0xc6, 0x0e, 0xb5, 0xba, 0xd8, 0xa5, 0x7a, 0xb7, 0x27, 0x35, 0x2b, - 0x6d, 0xa7, 0xed, 0xf0, 0xbf, 0x3b, 0xec, 0x9f, 0x90, 0x6e, 0x7d, 0x33, 0x05, 0x77, 0x1a, 0x0e, - 0x3d, 0xc4, 0xba, 0x8b, 0x6b, 0x4e, 0xc7, 0xc4, 0xa4, 0xca, 0x5c, 0x46, 0x15, 0xc8, 0x11, 0xdc, - 0xeb, 0x58, 0x86, 0xae, 0xa4, 0x37, 0xd3, 0x0f, 0xf3, 0xbb, 0x1f, 0x6f, 0x8f, 0x78, 0xbf, 0xad, - 0x0a, 0x8b, 0x0a, 0x76, 0x0d, 0x62, 0xf5, 0xa8, 0x43, 0xca, 0xd9, 0x37, 0x37, 0x1b, 0x29, 0xd5, - 0x1f, 0x8a, 0x9e, 0xc0, 0x7c, 0x87, 0x21, 0x6b, 0x17, 0x1c, 0x5a, 0x99, 0x9a, 0x1c, 0x4a, 0xcd, - 0x77, 0x42, 0x9f, 0xd0, 0x97, 0x30, 0x4b, 0x74, 0xbb, 0x8d, 0x35, 0xcb, 0x54, 0x32, 0x9b, 0xe9, - 0x87, 0x99, 0xf2, 0x1a, 0x9b, 0xe9, 0xf6, 0x66, 0x23, 0xa7, 0x32, 0x79, 0xbd, 0xf2, 0x36, 0xfc, - 0xab, 0xe6, 0xb8, 0x6d, 0xdd, 0x44, 0xdb, 0x30, 0xcd, 0x51, 0x94, 0x2c, 0x9f, 0x58, 0x89, 0x99, - 0x98, 0xaf, 0x5c, 0x15, 0x66, 0xe8, 0x01, 0x80, 0xe1, 0xb9, 0xd4, 0xe9, 0x6a, 0x5d, 0xb7, 0xad, - 0x4c, 0x6f, 0xa6, 0x1f, 0xce, 0xc9, 0x25, 0xcd, 0x09, 0xf9, 0x91, 0xdb, 0x7e, 0x94, 0xfd, 0xd7, - 0xd7, 0x1b, 0xe9, 0xad, 0x0f, 0x61, 0xa5, 0xe1, 0x98, 0xf8, 0xd4, 0xd6, 0x2f, 0x75, 0xab, 0xa3, - 0x9f, 0x75, 0x30, 0xdf, 0x38, 0xa9, 0xdd, 0x80, 0x0f, 0x4e, 0x6d, 0xd7, 0xeb, 0xf5, 0x1c, 0x42, - 0xb1, 0xa9, 0xe2, 0x9f, 0x79, 0xd8, 0xa5, 0x51, 0x83, 0x1f, 0x02, 0xe2, 0xde, 0x36, 0x1c, 0xba, - 0xef, 0x78, 0xb6, 0x29, 0x76, 0x3d, 0xba, 0xcc, 0xf4, 0xc4, 0xcb, 0x94, 0x90, 0xbf, 0x99, 0x82, - 0xbb, 0x5c, 0x75, 0x80, 0xaf, 0x8f, 0x2c, 0xb7, 0xab, 0x53, 0xe3, 0x42, 0xc0, 0x7e, 0x01, 0xcb, - 0x44, 0xb8, 0xa0, 0xb9, 0x54, 0x27, 0x54, 0x7b, 0x8d, 0xaf, 0x39, 0xfe, 0x7c, 0x39, 0xf7, 0xf6, - 0x66, 0x23, 0x73, 0x80, 0xaf, 0xd5, 0x25, 0x69, 0x71, 0xc2, 0x0c, 0x0e, 0xf0, 0x35, 0xda, 0x01, - 0x5f, 0xa4, 0x61, 0xdb, 0xe4, 0x43, 0xa6, 0x06, 0x87, 0x2c, 0x48, 0x7d, 0xd5, 0x36, 0xd9, 0x80, - 0x23, 0x28, 0x74, 0xe5, 0xb4, 0xd8, 0xd4, 0xb8, 0x6f, 0xfc, 0xac, 0xf2, 0xbb, 0x5b, 0x71, 0x07, - 0xce, 0xf4, 0x91, 0xe3, 0x5e, 0x0a, 0xc7, 0x72, 0x15, 0x3a, 0x80, 0x25, 0xd7, 0x6b, 0xb7, 0xb1, - 0x4b, 0x03, 0xb4, 0xec, 0xc4, 0x68, 0x8b, 0xc1, 0x50, 0xae, 0x91, 0x3b, 0xf4, 0xef, 0x29, 0xd8, - 0x52, 0xb1, 0x6e, 0x3e, 0xb3, 0xe8, 0x85, 0x65, 0x9f, 0xda, 0x06, 0x26, 0x54, 0xb7, 0x6c, 0x7a, - 0x5d, 0xb7, 0x29, 0x26, 0x97, 0x7a, 0x47, 0x6c, 0xd7, 0x53, 0x58, 0x24, 0x58, 0x37, 0xb5, 0x20, - 0x82, 0x64, 0x08, 0xac, 0x47, 0x26, 0x66, 0x61, 0xb6, 0x7d, 0xd1, 0x31, 0xb6, 0x5b, 0xbe, 0x91, - 0xbc, 0x28, 0x0b, 0x6c, 0x68, 0x20, 0x44, 0x2a, 0x20, 0xdc, 0xb7, 0x5c, 0x6a, 0xd9, 0xed, 0x08, - 0xde, 0xd4, 0xe4, 0x78, 0xcb, 0xfe, 0xf0, 0x10, 0xb3, 0x0c, 0x0b, 0x5d, 0xbd, 0x1f, 0x81, 0xcb, - 0x4c, 0x00, 0xa7, 0xce, 0x77, 0xf5, 0x7e, 0x88, 0xf1, 0x0a, 0xee, 0x38, 0x67, 0x2e, 0x26, 0x97, - 0x38, 0xb2, 0x4e, 0x57, 0xc9, 0x6e, 0x66, 0x12, 0x02, 0xb4, 0x29, 0xad, 0x87, 0xfd, 0x43, 0xce, - 0xb0, 0xc2, 0x0d, 0x63, 0xa0, 0x45, 0x74, 0xdb, 0xd5, 0x0d, 0x6a, 0x39, 0x76, 0xe9, 0x8c, 0x87, - 0x42, 0x34, 0x06, 0x74, 0x58, 0x89, 0x18, 0x1c, 0x7b, 0xae, 0xbc, 0xae, 0x7b, 0x00, 0x3d, 0xcf, - 0xbd, 0xc0, 0x58, 0xa3, 0x7d, 0x5b, 0xee, 0x7d, 0x31, 0xc6, 0xa5, 0xc8, 0x60, 0x3f, 0x4a, 0xc5, - 0xb8, 0x56, 0xdf, 0x96, 0x53, 0x9c, 0xc3, 0xdd, 0x88, 0x95, 0x8a, 0x29, 0xb9, 0x16, 0x73, 0x3c, - 0x81, 0x19, 0x82, 0x75, 0xd7, 0x11, 0xf8, 0x8b, 0xbb, 0x9f, 0x8d, 0xc7, 0xe7, 0x23, 0x55, 0x3e, - 0x40, 0x4e, 0x25, 0x87, 0xcb, 0x79, 0x8a, 0xb0, 0x3a, 0x60, 0xdd, 0xeb, 0xe8, 0xd7, 0xd1, 0xa5, - 0xfe, 0x29, 0x3d, 0x60, 0x70, 0x42, 0x75, 0xea, 0xb9, 0xc2, 0x93, 0x55, 0xc8, 0xb0, 0x64, 0x93, - 0x8e, 0x24, 0x1b, 0x26, 0x40, 0x8d, 0xc0, 0xc3, 0x29, 0xee, 0xe1, 0xb7, 0xc7, 0x7b, 0x18, 0x81, - 0xdc, 0x8e, 0x73, 0x74, 0xeb, 0xbb, 0x30, 0x23, 0xe4, 0x08, 0xc1, 0xa2, 0x5a, 0x2d, 0x9d, 0x34, - 0x1b, 0xda, 0x69, 0xe3, 0xa0, 0xd1, 0x7c, 0xd6, 0x28, 0xa4, 0x90, 0x02, 0x2b, 0x52, 0xd6, 0x7a, - 0xde, 0xd0, 0x1a, 0xcd, 0x96, 0xb6, 0xdf, 0x3c, 0x6d, 0x54, 0x0a, 0x69, 0xb9, 0x80, 0x17, 0x50, - 0x78, 0x46, 0x2c, 0x8a, 0x59, 0xa4, 0xd8, 0x22, 0x93, 0xa1, 0xaf, 0x20, 0x67, 0xf1, 0x4f, 0x57, - 0x49, 0xf3, 0x7b, 0x73, 0x2f, 0xc6, 0x45, 0x31, 0xc0, 0x2f, 0x0c, 0xd2, 0x5e, 0x80, 0x3e, 0xcd, - 0xce, 0x4e, 0x15, 0x32, 0x5b, 0xbf, 0x4f, 0x4b, 0xec, 0x96, 0xe3, 0x34, 0x3b, 0x32, 0x13, 0x96, - 0x60, 0xee, 0xbd, 0xc2, 0x2f, 0x1c, 0x85, 0x1a, 0x50, 0xd0, 0x0d, 0xea, 0xe9, 0x9d, 0xf7, 0x0b, - 0xbc, 0x25, 0x31, 0x38, 0x10, 0xcb, 0x8d, 0x58, 0x03, 0xd4, 0xec, 0xb1, 0x84, 0x6e, 0x11, 0xec, - 0xb6, 0xfa, 0x76, 0xf4, 0x94, 0x5f, 0xc0, 0xca, 0x9e, 0x63, 0x9b, 0x16, 0x3b, 0x8f, 0x7d, 0xdd, - 0xea, 0xc8, 0xeb, 0x8e, 0xbe, 0x0f, 0xf3, 0xd2, 0x93, 0x4b, 0xbd, 0xe3, 0x61, 0xb9, 0x9e, 0xb8, - 0x6a, 0xf4, 0x23, 0xa6, 0x57, 0xf3, 0xc2, 0x9a, 0x7f, 0x48, 0xe8, 0x3f, 0xa4, 0x01, 0x89, 0x52, - 0x85, 0x7f, 0x8a, 0x0d, 0x3f, 0x90, 0x50, 0x11, 0x72, 0x5d, 0xec, 0xba, 0x7a, 0x1b, 0x0f, 0x5c, - 0x20, 0x5f, 0x88, 0x1e, 0xc3, 0x9c, 0x4c, 0xd2, 0xd8, 0x94, 0x8b, 0x4f, 0x2c, 0x82, 0xfe, 0x0e, - 0x06, 0x03, 0xd0, 0x23, 0x98, 0xf5, 0xb3, 0x8f, 0xcc, 0x31, 0xef, 0x1a, 0x1c, 0xd8, 0x4b, 0xb7, - 0xbf, 0x07, 0x73, 0x27, 0xd8, 0x9e, 0xcc, 0xd9, 0x81, 0x4b, 0x71, 0x05, 0x2b, 0xa5, 0xee, 0x99, - 0xd5, 0xf6, 0x1c, 0xcf, 0x55, 0xb1, 0xeb, 0x75, 0xe8, 0x64, 0x0b, 0xfe, 0x0a, 0xf2, 0x57, 0x44, - 0xef, 0xf5, 0xb0, 0xa9, 0x61, 0x42, 0xc6, 0x2c, 0x99, 0xc3, 0xa9, 0x20, 0x8d, 0xab, 0xc4, 0x3f, - 0xc3, 0x75, 0x56, 0x44, 0xcf, 0xe9, 0x13, 0xe2, 0x78, 0xbd, 0x0a, 0xee, 0xe0, 0xa1, 0x9c, 0x85, - 0x61, 0x55, 0x52, 0x95, 0x3d, 0x87, 0x10, 0xaf, 0xc7, 0x8e, 0x5a, 0x78, 0xf6, 0x11, 0xcc, 0x71, - 0xb6, 0xa7, 0x0d, 0x47, 0xf3, 0x2c, 0x17, 0x1f, 0xb9, 0x6d, 0xb4, 0x05, 0x73, 0x3d, 0xe2, 0x18, - 0xd8, 0x75, 0xe5, 0x69, 0xcc, 0x06, 0x79, 0xcb, 0x17, 0x07, 0x37, 0x09, 0xc9, 0x69, 0xa2, 0x41, - 0xf1, 0x03, 0x00, 0xc9, 0xac, 0x7c, 0x82, 0x30, 0x5d, 0x2e, 0x4a, 0x82, 0x30, 0x27, 0xed, 0x39, - 0x45, 0x08, 0x3f, 0xd8, 0x71, 0x8a, 0xbf, 0x66, 0xc8, 0x3c, 0x4e, 0xa8, 0x43, 0x46, 0x99, 0x07, - 0xa3, 0x97, 0x71, 0xcc, 0x83, 0x5b, 0x0b, 0xe6, 0x21, 0xff, 0xaa, 0x39, 0x6e, 0x1b, 0x40, 0x1e, - 0x31, 0xb6, 0x73, 0xa1, 0xdb, 0x66, 0x87, 0x71, 0x1d, 0x4a, 0xae, 0x03, 0x3a, 0x84, 0x76, 0x21, - 0xdb, 0xab, 0x12, 0x32, 0xe6, 0xca, 0x73, 0x3b, 0xb9, 0x0f, 0xdc, 0x76, 0xeb, 0x6f, 0x69, 0x50, - 0x6a, 0x43, 0x68, 0x7e, 0xa4, 0x25, 0xa6, 0xcb, 0x57, 0x30, 0x43, 0xfb, 0x36, 0x73, 0x5f, 0xb0, - 0x94, 0x0a, 0x53, 0xfd, 0xf5, 0x66, 0xe3, 0x8b, 0xb6, 0x45, 0x2f, 0xbc, 0xb3, 0x6d, 0xc3, 0xe9, - 0xee, 0x04, 0x93, 0x9b, 0x67, 0xe1, 0xff, 0x9d, 0xde, 0xeb, 0xf6, 0x0e, 0x67, 0xcd, 0x9e, 0x67, - 0x99, 0xdb, 0xa7, 0xa7, 0xf5, 0xca, 0xed, 0xcd, 0xc6, 0x74, 0xab, 0x6f, 0xd7, 0x2b, 0xea, 0x34, - 0xed, 0xdb, 0x75, 0x13, 0xed, 0x43, 0x9e, 0x86, 0xa9, 0x56, 0xc6, 0xc2, 0x64, 0x25, 0x29, 0x3a, - 0x50, 0x6e, 0xd7, 0x7d, 0xb8, 0xd3, 0xea, 0xdb, 0xc7, 0x04, 0x5f, 0x96, 0x28, 0xc5, 0xdd, 0xde, - 0x00, 0x31, 0xfc, 0x14, 0x36, 0x5a, 0x7d, 0xbb, 0xd4, 0x61, 0x34, 0xe2, 0xba, 0x6a, 0x1b, 0x8e, - 0xc7, 0xb8, 0x89, 0xbc, 0x84, 0x51, 0xc3, 0x5f, 0xa5, 0x61, 0x85, 0x25, 0xd7, 0x36, 0x26, 0xcd, - 0x4b, 0x4c, 0xce, 0x3b, 0xce, 0x95, 0xd8, 0xa1, 0x7b, 0x90, 0x89, 0xe1, 0x77, 0x4c, 0x86, 0x3e, - 0x83, 0x05, 0xc3, 0x23, 0x04, 0xdb, 0x54, 0x66, 0xa2, 0x29, 0x7e, 0xd4, 0xc2, 0xd3, 0x79, 0xa9, - 0xe2, 0x69, 0x07, 0x7d, 0x0b, 0x96, 0x2c, 0xdb, 0x20, 0xb8, 0x1b, 0x1a, 0x67, 0x22, 0xc6, 0x8b, - 0x81, 0x32, 0x9a, 0xa5, 0x8e, 0x60, 0xf9, 0xc8, 0xea, 0x63, 0xf3, 0xc4, 0x33, 0xd8, 0x75, 0xf6, - 0xaf, 0x40, 0x4e, 0x46, 0xd9, 0xbb, 0x6e, 0x81, 0xea, 0x1b, 0x4a, 0xb8, 0xdf, 0xa5, 0xe1, 0x7e, - 0x99, 0x71, 0xc2, 0x30, 0x37, 0xe3, 0x73, 0x87, 0xe0, 0x27, 0x7b, 0x41, 0x91, 0x68, 0xbd, 0x57, - 0x91, 0x08, 0x79, 0x10, 0x83, 0xb8, 0x20, 0xd8, 0x65, 0x0d, 0xca, 0xff, 0x52, 0x1d, 0xc2, 0x51, - 0xd2, 0xd7, 0xe7, 0x80, 0x44, 0xa9, 0x3b, 0xb2, 0x5c, 0xd7, 0xb2, 0xdb, 0xc2, 0xc3, 0xc7, 0x30, - 0x7f, 0x45, 0x1c, 0xbb, 0xad, 0x89, 0xc2, 0x27, 0x9d, 0x4c, 0xae, 0x93, 0x6a, 0x9e, 0x9b, 0x8b, - 0x8f, 0xb0, 0xd3, 0x38, 0xc2, 0xa4, 0x8d, 0xeb, 0xf6, 0x31, 0x71, 0xda, 0xc4, 0xdf, 0x57, 0xa9, - 0xfd, 0xcf, 0x0a, 0xe4, 0xf9, 0x77, 0x05, 0x53, 0xdd, 0xea, 0x20, 0x15, 0x0a, 0xb6, 0x43, 0xb5, - 0x81, 0xb6, 0x4b, 0xcc, 0xfa, 0x7f, 0x31, 0xb3, 0xc6, 0xb4, 0x7e, 0xb5, 0x94, 0xba, 0x68, 0x0f, - 0x88, 0x51, 0x13, 0x96, 0x44, 0x5b, 0xc2, 0x90, 0xcf, 0x59, 0xd2, 0x90, 0x5b, 0xf5, 0x49, 0x12, - 0x15, 0x1f, 0x48, 0x2e, 0x35, 0xc6, 0x8a, 0xa3, 0x52, 0xf4, 0x1c, 0x90, 0x00, 0x7c, 0x8d, 0xaf, - 0x35, 0x9f, 0xf8, 0xcb, 0xb0, 0x7a, 0x98, 0x84, 0x39, 0xdc, 0xd6, 0xd4, 0x52, 0x6a, 0x81, 0x0c, - 0x29, 0xd0, 0xcf, 0xd3, 0xb0, 0xc9, 0xc9, 0xfb, 0x15, 0xe7, 0xf8, 0x9a, 0x17, 0x92, 0x7c, 0x7e, - 0x06, 0x8c, 0xe5, 0xcb, 0x3e, 0xe2, 0xcb, 0xd8, 0x36, 0xf4, 0x5d, 0xdd, 0x41, 0x2d, 0xa5, 0xae, - 0x93, 0x71, 0x56, 0xe8, 0x27, 0x70, 0x27, 0x12, 0xf3, 0x9a, 0x2e, 0x88, 0x2f, 0xef, 0x26, 0xf3, - 0xbb, 0x9f, 0x8f, 0x4f, 0x1a, 0x51, 0x96, 0x5c, 0x4b, 0xa9, 0x88, 0x8e, 0xa8, 0x50, 0x0b, 0x0a, - 0x51, 0x78, 0xc6, 0x78, 0x95, 0x19, 0x8e, 0xfd, 0xe9, 0x78, 0xec, 0x80, 0x60, 0xd7, 0x52, 0xea, - 0x12, 0x1d, 0x94, 0xa3, 0x67, 0xb0, 0x1c, 0x45, 0x25, 0x2c, 0xef, 0x2a, 0xb9, 0xc4, 0x03, 0x89, - 0x25, 0xd5, 0xec, 0x40, 0xe8, 0x90, 0x02, 0xbd, 0x84, 0xe8, 0x22, 0x58, 0xff, 0x49, 0x3d, 0x57, - 0x99, 0xe5, 0xc8, 0x9f, 0x4d, 0x4c, 0x69, 0x6b, 0x29, 0x35, 0xea, 0x9f, 0xd0, 0xa0, 0x1a, 0x8b, - 0x2e, 0x8b, 0x62, 0x3f, 0xba, 0xe6, 0x38, 0xea, 0x83, 0x18, 0xd4, 0x61, 0xee, 0x5a, 0x4b, 0xb1, - 0x48, 0x0b, 0x64, 0xa8, 0x0e, 0x0b, 0x02, 0x89, 0x3a, 0x8e, 0xc6, 0x52, 0x01, 0x8c, 0x87, 0x8a, - 0x54, 0xe5, 0x00, 0x4a, 0xc8, 0x58, 0xb0, 0x38, 0x3d, 0x8d, 0x48, 0x86, 0xc8, 0x5b, 0x98, 0x7c, - 0x62, 0xb0, 0x8c, 0x52, 0x49, 0x16, 0x2c, 0x4e, 0x54, 0xca, 0x0e, 0xdc, 0xf0, 0x59, 0xa5, 0x76, - 0xce, 0x69, 0xa5, 0x32, 0x9f, 0x78, 0xe0, 0x71, 0x04, 0x94, 0x1d, 0xb8, 0x31, 0x28, 0x47, 0x0d, - 0x58, 0x14, 0x39, 0x82, 0x48, 0x42, 0xa9, 0x2c, 0x24, 0x7a, 0x39, 0x4a, 0x3c, 0x99, 0x97, 0x9d, - 0xa8, 0x94, 0x79, 0x69, 0x3b, 0x26, 0xd6, 0xbc, 0xf0, 0x41, 0x44, 0x59, 0x4c, 0xf4, 0x32, 0xee, - 0xe9, 0x84, 0x79, 0x69, 0x0f, 0xca, 0x19, 0x7d, 0x70, 0xb1, 0x6d, 0x2a, 0x4b, 0x1c, 0xe9, 0xc3, - 0x18, 0xa4, 0x80, 0x5e, 0xd6, 0x52, 0x2a, 0xb7, 0x15, 0xc9, 0xe5, 0x9c, 0x6a, 0x6d, 0x46, 0xe1, - 0x34, 0x53, 0x70, 0x38, 0xa5, 0x30, 0x26, 0xb9, 0xc4, 0xd0, 0x3d, 0x91, 0x5c, 0x06, 0x15, 0xec, - 0x2e, 0xfb, 0xfc, 0xcb, 0x08, 0xd8, 0x9f, 0xb2, 0x9c, 0x78, 0x97, 0xe3, 0x99, 0x22, 0xbb, 0xcb, - 0x64, 0x58, 0xc3, 0x73, 0xac, 0xc4, 0xf6, 0xef, 0x20, 0x4a, 0xce, 0xb1, 0x23, 0xdc, 0x90, 0xe7, - 0xd8, 0xa8, 0x74, 0x38, 0xf0, 0x08, 0xef, 0x49, 0x95, 0xd5, 0x49, 0x02, 0x2f, 0xd2, 0xbf, 0x0e, - 0x05, 0x9e, 0xd0, 0xb0, 0xc3, 0xd6, 0x7d, 0x76, 0xae, 0x11, 0x4e, 0xcf, 0x95, 0xb5, 0xc4, 0xc3, - 0x8e, 0x23, 0xf2, 0xec, 0xb0, 0xf5, 0x41, 0x39, 0xdb, 0x02, 0xc1, 0x41, 0xc3, 0x32, 0x73, 0x3f, - 0x71, 0x0b, 0x46, 0x39, 0x2c, 0xdb, 0x02, 0x37, 0x2a, 0x45, 0x1d, 0xb8, 0x2f, 0x59, 0xa9, 0x48, - 0x68, 0xec, 0x4a, 0xb1, 0x80, 0xd4, 0x38, 0x23, 0x57, 0x3e, 0xe4, 0xe0, 0xff, 0x1f, 0x03, 0x9e, - 0xc4, 0x3e, 0x6b, 0x29, 0x55, 0xb9, 0x48, 0x62, 0xa6, 0xaf, 0xe0, 0x2e, 0xc3, 0x96, 0xf9, 0x5e, - 0xd3, 0xdd, 0x6b, 0xdb, 0xe0, 0x4d, 0x48, 0x31, 0xb1, 0xfc, 0xc6, 0xd0, 0x40, 0x9e, 0xf5, 0xfb, - 0x7e, 0xb6, 0x2f, 0x31, 0x90, 0x2a, 0x21, 0x6c, 0xc7, 0x2d, 0x41, 0xf6, 0x34, 0x47, 0xb2, 0x3d, - 0x65, 0x23, 0x71, 0xc7, 0xe3, 0x78, 0x21, 0xdb, 0x71, 0x6b, 0x50, 0xce, 0x4a, 0x95, 0x17, 0x3e, - 0x53, 0x6a, 0xb2, 0xf3, 0x53, 0x36, 0x13, 0x4b, 0x55, 0xc2, 0xa3, 0x26, 0x73, 0xda, 0x1b, 0x51, - 0xa1, 0x03, 0x58, 0xe8, 0x32, 0x3a, 0xa8, 0xb9, 0x82, 0x0f, 0x2a, 0x1f, 0x25, 0xbe, 0xff, 0x8e, - 0xd0, 0xc6, 0x5a, 0x4a, 0x9d, 0xef, 0x46, 0x84, 0xe8, 0x15, 0x14, 0x82, 0x3e, 0x5e, 0x3b, 0xe3, - 0x3c, 0x50, 0xd9, 0xe2, 0x78, 0xdb, 0x31, 0x78, 0x63, 0x68, 0x23, 0x2f, 0x7f, 0x83, 0x1a, 0x74, - 0x05, 0xeb, 0xfc, 0xec, 0x04, 0xed, 0xd6, 0x70, 0xc8, 0xbb, 0xe5, 0x5d, 0x79, 0xc0, 0x67, 0xda, - 0x8d, 0x3f, 0xc3, 0x71, 0x6c, 0xbd, 0x96, 0x52, 0xd7, 0x68, 0xa2, 0x09, 0x4b, 0xc3, 0xa2, 0x78, - 0x31, 0x1a, 0xc4, 0x78, 0xa3, 0xf2, 0x71, 0xe2, 0x95, 0x1f, 0xe5, 0x97, 0xec, 0xca, 0x5b, 0x51, - 0x29, 0x3a, 0x85, 0xe5, 0x2e, 0x23, 0x8b, 0x9a, 0x65, 0x6b, 0x3d, 0x49, 0x17, 0x95, 0x4f, 0x12, - 0x2f, 0x4a, 0x1c, 0xb1, 0x64, 0xfb, 0xd3, 0x1d, 0x94, 0x0b, 0xae, 0x59, 0xce, 0xc1, 0x34, 0xef, - 0x04, 0x9e, 0x66, 0x67, 0xef, 0x15, 0xd6, 0x9e, 0x66, 0x67, 0xd7, 0x0b, 0xc5, 0xad, 0x1d, 0xce, - 0x3f, 0x8f, 0x1d, 0x97, 0x57, 0x17, 0xb4, 0x06, 0xd3, 0x96, 0x6d, 0xe2, 0xbe, 0x6c, 0x4f, 0x05, - 0x5b, 0x16, 0x22, 0xc9, 0x58, 0xbf, 0xc9, 0xc0, 0xf4, 0x64, 0xcd, 0xfc, 0x8f, 0x07, 0x99, 0x14, - 0xc1, 0xfc, 0xf5, 0x9a, 0xf3, 0xc4, 0xc5, 0xd8, 0x1d, 0x1a, 0xc8, 0x61, 0xdc, 0xd8, 0x7f, 0xa5, - 0xa4, 0x23, 0x1a, 0xb4, 0x07, 0x0b, 0x9e, 0x8d, 0xfb, 0x3d, 0xc7, 0xc5, 0x26, 0x2f, 0xd3, 0xd9, - 0x49, 0xda, 0x3a, 0x75, 0x3e, 0x18, 0xc4, 0x8a, 0xf3, 0x0e, 0xe4, 0x1d, 0x62, 0xb5, 0x2d, 0x5b, - 0x63, 0xa5, 0x8b, 0x93, 0xbc, 0xe9, 0xf2, 0x22, 0x9b, 0xf3, 0xed, 0xcd, 0xc6, 0x0c, 0x2b, 0x73, - 0xf5, 0x8a, 0x0a, 0xc2, 0x84, 0x7d, 0xa1, 0xc7, 0x30, 0x63, 0x72, 0xa6, 0x2e, 0x49, 0x5b, 0x31, - 0xa9, 0x19, 0x12, 0x7c, 0xde, 0x7f, 0xc4, 0x13, 0x63, 0xd0, 0x77, 0xfc, 0xdd, 0xcd, 0x8d, 0x1b, - 0xec, 0x1f, 0x86, 0xdc, 0x77, 0xf4, 0x25, 0x64, 0x6c, 0xe7, 0x4a, 0x92, 0xae, 0x89, 0xda, 0x1b, - 0x66, 0xff, 0x68, 0xf6, 0xd7, 0x5f, 0x6f, 0xa4, 0xc2, 0x37, 0x99, 0xcf, 0xff, 0x98, 0x1e, 0x7a, - 0xe5, 0x0c, 0xde, 0x44, 0xc5, 0xf3, 0x61, 0x4b, 0x7d, 0xa1, 0x8d, 0x3c, 0x2c, 0x7e, 0x00, 0x77, - 0x84, 0xe6, 0x99, 0x5a, 0x6f, 0x55, 0xb5, 0x56, 0xb3, 0xa9, 0x35, 0x0f, 0x2b, 0x85, 0x34, 0x5a, - 0x05, 0x24, 0x14, 0x95, 0xea, 0x61, 0xb5, 0x55, 0xd5, 0xd4, 0x52, 0xe3, 0x49, 0xb5, 0x30, 0x15, - 0xca, 0x4f, 0xaa, 0x6a, 0xbd, 0x74, 0x58, 0x7f, 0x59, 0x2a, 0x1f, 0x56, 0x0b, 0x19, 0x74, 0x0f, - 0xee, 0x0a, 0xf9, 0x71, 0xf3, 0xe4, 0xa4, 0x5e, 0x3e, 0xac, 0x6a, 0x6a, 0xf5, 0xf8, 0xb0, 0xf4, - 0xa2, 0x90, 0x45, 0xeb, 0x70, 0x4f, 0xa8, 0x4a, 0x27, 0x2f, 0x1a, 0x7b, 0x72, 0xa6, 0xfd, 0x52, - 0xfd, 0xf0, 0x54, 0xad, 0x16, 0xa6, 0xd7, 0xb2, 0xbf, 0xf8, 0x6d, 0x31, 0xf5, 0xf9, 0x23, 0x40, - 0xa3, 0xd7, 0x03, 0xcd, 0x42, 0xb6, 0xd1, 0x6c, 0x54, 0x0b, 0x29, 0x94, 0x87, 0x5c, 0xb9, 0xb4, - 0x77, 0xd0, 0xdc, 0xdf, 0x2f, 0xa4, 0xd1, 0x02, 0xcc, 0xd5, 0x8f, 0x8e, 0xaa, 0x95, 0x7a, 0xa9, - 0x55, 0x2d, 0x4c, 0x95, 0x3f, 0x7a, 0xf3, 0x8f, 0x62, 0xea, 0xcd, 0x6d, 0x31, 0xfd, 0xe7, 0xdb, - 0x62, 0xfa, 0x2f, 0xb7, 0xc5, 0xf4, 0xdf, 0x6f, 0x8b, 0xe9, 0x5f, 0xfe, 0xb3, 0x98, 0x7a, 0x99, - 0x93, 0xdb, 0xfe, 0xdf, 0x00, 0x00, 0x00, 0xff, 0xff, 0xd7, 0xc8, 0x43, 0xb4, 0xf3, 0x1b, 0x00, - 0x00, + // 2490 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x9c, 0x58, 0xcf, 0x73, 0x1b, 0x4b, + 0xf1, 0x97, 0x2c, 0xd9, 0xb2, 0x5b, 0xfe, 0x21, 0x4f, 0x1c, 0xbf, 0x8d, 0x13, 0x4b, 0x8e, 0xf3, + 0xde, 0xf7, 0x25, 0xf9, 0x16, 0x36, 0xe5, 0x10, 0xa8, 0x17, 0xc2, 0x41, 0xb2, 0xe4, 0x48, 0xb1, + 0x2d, 0x85, 0xb5, 0x4c, 0x7e, 0x41, 0x6d, 0xad, 0x77, 0xc7, 0xf2, 0x12, 0x69, 0x57, 0xcc, 0xce, + 0xda, 0xf2, 0x8d, 0x23, 0x47, 0xb8, 0x71, 0xe3, 0x55, 0x71, 0xa0, 0x28, 0xae, 0xd4, 0xbb, 0x50, + 0xc5, 0x39, 0x47, 0x8e, 0x14, 0x55, 0xb8, 0xc0, 0x5c, 0xa8, 0xe2, 0x3f, 0xc8, 0x89, 0x9a, 0x1f, + 0xab, 0x5d, 0x49, 0xbb, 0x8a, 0xc8, 0x49, 0xda, 0xee, 0x9e, 0xcf, 0xf4, 0x4c, 0x4f, 0x77, 0x7f, + 0x66, 0x60, 0x85, 0x38, 0xba, 0x71, 0xd6, 0x3d, 0xd9, 0xc6, 0x84, 0x38, 0xc4, 0xdd, 0xea, 0x12, + 0x87, 0x3a, 0x68, 0xd9, 0x70, 0x8c, 0x77, 0x5c, 0xb3, 0x25, 0xf5, 0x6b, 0xc8, 0x37, 0x34, 0x75, + 0xaa, 0x0b, 0xb3, 0xb5, 0x55, 0x5f, 0xd6, 0xc1, 0x54, 0x0f, 0xc9, 0xef, 0xb9, 0xd4, 0x21, 0x7a, + 0x0b, 0x6f, 0x63, 0xbb, 0x65, 0xd9, 0xfe, 0x0f, 0xb3, 0x3b, 0x37, 0x8c, 0x47, 0xd2, 0x48, 0xf1, + 0xa8, 0xd5, 0xde, 0x3e, 0x6b, 0x1b, 0xdb, 0xd4, 0xea, 0x60, 0x97, 0xea, 0x9d, 0xae, 0xd4, 0xac, + 0xb4, 0x9c, 0x96, 0xc3, 0xff, 0x6e, 0xb3, 0x7f, 0x42, 0xba, 0xf9, 0xcd, 0x14, 0xdc, 0xa8, 0x3b, + 0xf4, 0x00, 0xeb, 0x2e, 0xae, 0x3a, 0x6d, 0x13, 0x93, 0x0a, 0x73, 0x19, 0x95, 0x21, 0x43, 0x70, + 0xb7, 0x6d, 0x19, 0xba, 0x92, 0xdc, 0x48, 0xde, 0xcf, 0xee, 0x7c, 0xbe, 0x35, 0xe2, 0xfd, 0x96, + 0x2a, 0x2c, 0xca, 0xd8, 0x35, 0x88, 0xd5, 0xa5, 0x0e, 0x29, 0xa5, 0xdf, 0x5f, 0x15, 0x12, 0xaa, + 0x3f, 0x14, 0x3d, 0x83, 0xf9, 0x36, 0x43, 0xd6, 0xce, 0x38, 0xb4, 0x32, 0x35, 0x39, 0x94, 0x9a, + 0x6d, 0x07, 0x3e, 0xa1, 0xc7, 0x30, 0x4b, 0x74, 0xbb, 0x85, 0x35, 0xcb, 0x54, 0x52, 0x1b, 0xc9, + 0xfb, 0xa9, 0xd2, 0x1a, 0x9b, 0xe9, 0xfa, 0xaa, 0x90, 0x51, 0x99, 0xbc, 0x56, 0xfe, 0x10, 0xfc, + 0x55, 0x33, 0xdc, 0xb6, 0x66, 0xa2, 0x2d, 0x98, 0xe6, 0x28, 0x4a, 0x9a, 0x4f, 0xac, 0x44, 0x4c, + 0xcc, 0x57, 0xae, 0x0a, 0x33, 0x74, 0x0f, 0xc0, 0xf0, 0x5c, 0xea, 0x74, 0xb4, 0x8e, 0xdb, 0x52, + 0xa6, 0x37, 0x92, 0xf7, 0xe7, 0xe4, 0x92, 0xe6, 0x84, 0xfc, 0xd0, 0x6d, 0x3d, 0x49, 0xff, 0xfb, + 0xeb, 0x42, 0x72, 0xf3, 0x0e, 0xac, 0xd4, 0x1d, 0x13, 0x1f, 0xdb, 0xfa, 0xb9, 0x6e, 0xb5, 0xf5, + 0x93, 0x36, 0xe6, 0x1b, 0x27, 0xb5, 0x05, 0xf8, 0xec, 0xd8, 0x76, 0xbd, 0x6e, 0xd7, 0x21, 0x14, + 0x9b, 0x2a, 0xfe, 0x99, 0x87, 0x5d, 0x1a, 0x36, 0xf8, 0x21, 0x20, 0xee, 0x6d, 0xdd, 0xa1, 0x7b, + 0x8e, 0x67, 0x9b, 0x62, 0xd7, 0xc3, 0xcb, 0x4c, 0x4e, 0xbc, 0x4c, 0x09, 0xf9, 0x9b, 0x29, 0xb8, + 0xc9, 0x55, 0xfb, 0xf8, 0xf2, 0xd0, 0x72, 0x3b, 0x3a, 0x35, 0xce, 0x04, 0xec, 0x23, 0x58, 0x26, + 0xc2, 0x05, 0xcd, 0xa5, 0x3a, 0xa1, 0xda, 0x3b, 0x7c, 0xc9, 0xf1, 0xe7, 0x4b, 0x99, 0x0f, 0x57, + 0x85, 0xd4, 0x3e, 0xbe, 0x54, 0x97, 0xa4, 0xc5, 0x11, 0x33, 0xd8, 0xc7, 0x97, 0x68, 0x1b, 0x7c, + 0x91, 0x86, 0x6d, 0x93, 0x0f, 0x99, 0x1a, 0x1c, 0xb2, 0x20, 0xf5, 0x15, 0xdb, 0x64, 0x03, 0x0e, + 0x21, 0xd7, 0x91, 0xd3, 0x62, 0x53, 0xe3, 0xbe, 0xf1, 0x58, 0x65, 0x77, 0x36, 0xa3, 0x02, 0xce, + 0xf4, 0xa1, 0x70, 0x2f, 0x05, 0x63, 0xb9, 0x0a, 0xed, 0xc3, 0x92, 0xeb, 0xb5, 0x5a, 0xd8, 0xa5, + 0x7d, 0xb4, 0xf4, 0xc4, 0x68, 0x8b, 0xfd, 0xa1, 0x5c, 0x23, 0x77, 0xe8, 0x3f, 0x53, 0xb0, 0xa9, + 0x62, 0xdd, 0x7c, 0x69, 0xd1, 0x33, 0xcb, 0x3e, 0xb6, 0x0d, 0x4c, 0xa8, 0x6e, 0xd9, 0xf4, 0xb2, + 0x66, 0x53, 0x4c, 0xce, 0xf5, 0xb6, 0xd8, 0xae, 0xe7, 0xb0, 0x48, 0xb0, 0x6e, 0x6a, 0xfd, 0x0c, + 0x92, 0x29, 0xb0, 0x1e, 0x9a, 0x98, 0xa5, 0xd9, 0xd6, 0x59, 0xdb, 0xd8, 0x6a, 0xfa, 0x46, 0xf2, + 0xa0, 0x2c, 0xb0, 0xa1, 0x7d, 0x21, 0x52, 0x01, 0xe1, 0x9e, 0xe5, 0x52, 0xcb, 0x6e, 0x85, 0xf0, + 0xa6, 0x26, 0xc7, 0x5b, 0xf6, 0x87, 0x07, 0x98, 0x25, 0x58, 0xe8, 0xe8, 0xbd, 0x10, 0x5c, 0x6a, + 0x02, 0x38, 0x75, 0xbe, 0xa3, 0xf7, 0x02, 0x8c, 0xb7, 0x70, 0xc3, 0x39, 0x71, 0x31, 0x39, 0xc7, + 0xa1, 0x75, 0xba, 0x4a, 0x7a, 0x23, 0x15, 0x93, 0xa0, 0x0d, 0x69, 0x3d, 0xec, 0x1f, 0x72, 0x86, + 0x15, 0x6e, 0x90, 0x03, 0x4d, 0xa2, 0xdb, 0xae, 0x6e, 0x50, 0xcb, 0xb1, 0x8b, 0x27, 0x3c, 0x15, + 0xc2, 0x39, 0xa0, 0xc3, 0x4a, 0xc8, 0xe0, 0x85, 0xe7, 0xca, 0xe3, 0xba, 0x0b, 0xd0, 0xf5, 0xdc, + 0x33, 0x8c, 0x35, 0xda, 0xb3, 0xe5, 0xde, 0xe7, 0x23, 0x5c, 0x0a, 0x0d, 0xf6, 0xb3, 0x54, 0x8c, + 0x6b, 0xf6, 0x6c, 0x39, 0xc5, 0x29, 0xdc, 0x0c, 0x59, 0xa9, 0x98, 0x92, 0x4b, 0x31, 0xc7, 0x33, + 0x98, 0x21, 0x58, 0x77, 0x1d, 0x81, 0xbf, 0xb8, 0xf3, 0x60, 0x3c, 0x3e, 0x1f, 0xa9, 0xf2, 0x01, + 0x72, 0x2a, 0x39, 0x5c, 0xce, 0x93, 0x87, 0xd5, 0x01, 0xeb, 0x6e, 0x5b, 0xbf, 0x0c, 0x2f, 0xf5, + 0xcf, 0xc9, 0x01, 0x83, 0x23, 0xaa, 0x53, 0xcf, 0x15, 0x9e, 0xac, 0x42, 0x8a, 0x15, 0x9b, 0x64, + 0xa8, 0xd8, 0x30, 0x01, 0xaa, 0xf7, 0x3d, 0x9c, 0xe2, 0x1e, 0x7e, 0x7b, 0xbc, 0x87, 0x21, 0xc8, + 0xad, 0x28, 0x47, 0x37, 0xbf, 0x0b, 0x33, 0x42, 0x8e, 0x10, 0x2c, 0xaa, 0x95, 0xe2, 0x51, 0xa3, + 0xae, 0x1d, 0xd7, 0xf7, 0xeb, 0x8d, 0x97, 0xf5, 0x5c, 0x02, 0x29, 0xb0, 0x22, 0x65, 0xcd, 0x57, + 0x75, 0xad, 0xde, 0x68, 0x6a, 0x7b, 0x8d, 0xe3, 0x7a, 0x39, 0x97, 0x94, 0x0b, 0x78, 0x0d, 0xb9, + 0x97, 0xc4, 0xa2, 0x98, 0x65, 0x8a, 0x2d, 0x2a, 0x19, 0xfa, 0x0a, 0x32, 0x16, 0xff, 0x74, 0x95, + 0x24, 0x3f, 0x37, 0xb7, 0x22, 0x5c, 0x14, 0x03, 0xfc, 0xc6, 0x20, 0xed, 0x05, 0xe8, 0xf3, 0xf4, + 0xec, 0x54, 0x2e, 0xb5, 0xf9, 0x87, 0xa4, 0xc4, 0x6e, 0x3a, 0x4e, 0xa3, 0x2d, 0x2b, 0x61, 0x11, + 0xe6, 0x3e, 0x29, 0xfd, 0x82, 0x51, 0xa8, 0x0e, 0x39, 0xdd, 0xa0, 0x9e, 0xde, 0xfe, 0xb4, 0xc4, + 0x5b, 0x12, 0x83, 0xfb, 0x62, 0xb9, 0x11, 0x6b, 0x80, 0x1a, 0x5d, 0x56, 0xd0, 0x2d, 0x82, 0xdd, + 0x66, 0xcf, 0x0e, 0x47, 0xf9, 0x35, 0xac, 0xec, 0x3a, 0xb6, 0x69, 0xb1, 0x78, 0xec, 0xe9, 0x56, + 0x5b, 0x1e, 0x77, 0xf4, 0x7d, 0x98, 0x97, 0x9e, 0x9c, 0xeb, 0x6d, 0x0f, 0xcb, 0xf5, 0x44, 0x75, + 0xa3, 0x1f, 0x31, 0xbd, 0x9a, 0x15, 0xd6, 0xfc, 0x43, 0x42, 0xff, 0x31, 0x09, 0x48, 0xb4, 0x2a, + 0xfc, 0x53, 0x6c, 0xf8, 0x89, 0x84, 0xf2, 0x90, 0xe9, 0x60, 0xd7, 0xd5, 0x5b, 0x78, 0xe0, 0x00, + 0xf9, 0x42, 0xf4, 0x14, 0xe6, 0x64, 0x91, 0xc6, 0xa6, 0x5c, 0x7c, 0x6c, 0x13, 0xf4, 0x77, 0xb0, + 0x3f, 0x00, 0x3d, 0x81, 0x59, 0xbf, 0xfa, 0xc8, 0x1a, 0xf3, 0xb1, 0xc1, 0x7d, 0x7b, 0xe9, 0xf6, + 0xf7, 0x60, 0xee, 0x08, 0xdb, 0x93, 0x39, 0x3b, 0x70, 0x28, 0x2e, 0x60, 0xa5, 0xd8, 0x39, 0xb1, + 0x5a, 0x9e, 0xe3, 0xb9, 0x2a, 0x76, 0xbd, 0x36, 0x9d, 0x6c, 0xc1, 0x5f, 0x41, 0xf6, 0x82, 0xe8, + 0xdd, 0x2e, 0x36, 0x35, 0x4c, 0xc8, 0x98, 0x25, 0x73, 0x38, 0x15, 0xa4, 0x71, 0x85, 0xf8, 0x31, + 0x5c, 0x67, 0x4d, 0xf4, 0x94, 0x3e, 0x23, 0x8e, 0xd7, 0x2d, 0xe3, 0x36, 0x1e, 0xaa, 0x59, 0x18, + 0x56, 0x25, 0x55, 0xd9, 0x75, 0x08, 0xf1, 0xba, 0x2c, 0xd4, 0xc2, 0xb3, 0xbb, 0x30, 0xc7, 0xd9, + 0x9e, 0x36, 0x9c, 0xcd, 0xb3, 0x5c, 0x7c, 0xe8, 0xb6, 0xd0, 0x26, 0xcc, 0x75, 0x89, 0x63, 0x60, + 0xd7, 0x95, 0xd1, 0x98, 0xed, 0xd7, 0x2d, 0x5f, 0xdc, 0x3f, 0x49, 0x48, 0x4e, 0x13, 0x4e, 0x8a, + 0x1f, 0x00, 0x48, 0x66, 0xe5, 0x13, 0x84, 0xe9, 0x52, 0x5e, 0x12, 0x84, 0x39, 0x69, 0xcf, 0x29, + 0x42, 0xf0, 0xc1, 0xc2, 0x29, 0xfe, 0x9a, 0x01, 0xf3, 0x38, 0xa2, 0x0e, 0x19, 0x65, 0x1e, 0x8c, + 0x5e, 0x46, 0x31, 0x0f, 0x6e, 0x2d, 0x98, 0x87, 0xfc, 0xab, 0x66, 0xb8, 0x6d, 0x1f, 0xf2, 0x90, + 0xb1, 0x9d, 0x33, 0xdd, 0x36, 0xdb, 0x8c, 0xeb, 0x50, 0x72, 0xd9, 0xa7, 0x43, 0x68, 0x07, 0xd2, + 0xdd, 0x0a, 0x21, 0x63, 0x8e, 0x3c, 0xb7, 0x93, 0xfb, 0xc0, 0x6d, 0x37, 0xff, 0x9e, 0x04, 0xa5, + 0x3a, 0x84, 0xe6, 0x67, 0x5a, 0x6c, 0xb9, 0x7c, 0x0b, 0x33, 0xb4, 0x67, 0x33, 0xf7, 0x05, 0x4b, + 0x29, 0x33, 0xd5, 0xdf, 0xae, 0x0a, 0x8f, 0x5a, 0x16, 0x3d, 0xf3, 0x4e, 0xb6, 0x0c, 0xa7, 0xb3, + 0xdd, 0x9f, 0xdc, 0x3c, 0x09, 0xfe, 0x6f, 0x77, 0xdf, 0xb5, 0xb6, 0x39, 0x6b, 0xf6, 0x3c, 0xcb, + 0xdc, 0x3a, 0x3e, 0xae, 0x95, 0xaf, 0xaf, 0x0a, 0xd3, 0xcd, 0x9e, 0x5d, 0x2b, 0xab, 0xd3, 0xb4, + 0x67, 0xd7, 0x4c, 0xb4, 0x07, 0x59, 0x1a, 0x94, 0x5a, 0x99, 0x0b, 0x93, 0xb5, 0xa4, 0xf0, 0x40, + 0xb9, 0x5d, 0x5f, 0x42, 0xa1, 0xd9, 0xb3, 0x8b, 0x6d, 0xc6, 0x14, 0x2e, 0x2b, 0xb6, 0xe1, 0x78, + 0x8c, 0x7e, 0xc8, 0x73, 0x16, 0x3e, 0x6c, 0xbf, 0x4a, 0xc2, 0x0a, 0xab, 0x9f, 0x2d, 0x4c, 0x1a, + 0xe7, 0x98, 0x9c, 0xb6, 0x9d, 0x0b, 0xb1, 0x09, 0xb7, 0x20, 0x15, 0x41, 0xe1, 0x98, 0x0c, 0x3d, + 0x80, 0x05, 0xc3, 0x23, 0x04, 0xdb, 0x54, 0x16, 0x9b, 0x29, 0x1e, 0x4d, 0xe1, 0xcc, 0xbc, 0x54, + 0xf1, 0xca, 0x82, 0xbe, 0x05, 0x4b, 0x96, 0x6d, 0x10, 0xdc, 0x09, 0x8c, 0x53, 0x21, 0xe3, 0xc5, + 0xbe, 0x32, 0x5c, 0x88, 0x0e, 0x61, 0xf9, 0xd0, 0xea, 0x61, 0xf3, 0xc8, 0x33, 0xd8, 0x89, 0xf5, + 0xa3, 0x9c, 0x91, 0x89, 0xf4, 0xb1, 0x40, 0xab, 0xbe, 0xa1, 0x84, 0xfb, 0x7d, 0x12, 0x6e, 0x97, + 0x18, 0xed, 0x0b, 0xca, 0x2f, 0x3e, 0x75, 0x08, 0x7e, 0xb6, 0xdb, 0xef, 0x03, 0xcd, 0x4f, 0xea, + 0x03, 0x01, 0xd5, 0x61, 0x10, 0x67, 0x04, 0xbb, 0xec, 0x0e, 0xf2, 0xbf, 0x34, 0x80, 0x60, 0x94, + 0xf4, 0xf5, 0x15, 0x20, 0xd1, 0xcd, 0x0e, 0x2d, 0xd7, 0xb5, 0xec, 0x96, 0xf0, 0xf0, 0x29, 0xcc, + 0x5f, 0x10, 0xc7, 0x6e, 0x69, 0xa2, 0xb7, 0x49, 0x27, 0xe3, 0x5b, 0xa1, 0x9a, 0xe5, 0xe6, 0xe2, + 0x23, 0xb8, 0x4c, 0x1c, 0x62, 0xd2, 0xc2, 0x35, 0xfb, 0x05, 0x71, 0x5a, 0xc4, 0xdf, 0x57, 0xa9, + 0xfd, 0xdd, 0x0a, 0x64, 0xf9, 0x77, 0x19, 0x53, 0xdd, 0x6a, 0x23, 0x15, 0x72, 0xb6, 0x43, 0xb5, + 0x81, 0x9b, 0x95, 0x98, 0xf5, 0xff, 0x22, 0x66, 0x8d, 0xb8, 0xdd, 0x55, 0x13, 0xea, 0xa2, 0x3d, + 0x20, 0x46, 0x0d, 0x58, 0x12, 0x37, 0x0f, 0x86, 0x7c, 0xca, 0xea, 0x82, 0xdc, 0xaa, 0x2f, 0xe2, + 0xd8, 0xf6, 0x40, 0xfd, 0xa8, 0x32, 0xe2, 0x1b, 0x96, 0xa2, 0x57, 0x80, 0x04, 0xe0, 0x3b, 0x7c, + 0xa9, 0xf9, 0xdc, 0x5e, 0x66, 0xce, 0xfd, 0x38, 0xcc, 0xe1, 0x9b, 0x4b, 0x35, 0xa1, 0xe6, 0xc8, + 0x90, 0x02, 0xfd, 0x3c, 0x09, 0x1b, 0x9c, 0x9f, 0x5f, 0x70, 0x1a, 0xaf, 0x79, 0x01, 0x8f, 0xe7, + 0x31, 0x60, 0x44, 0x5e, 0x5e, 0x15, 0x1e, 0x47, 0xde, 0x34, 0x3f, 0x76, 0x01, 0xa8, 0x26, 0xd4, + 0x75, 0x32, 0xce, 0x0a, 0xfd, 0x04, 0x6e, 0x84, 0xd2, 0x5a, 0xd3, 0x05, 0xb7, 0xe5, 0x17, 0xc6, + 0xec, 0xce, 0xc3, 0xf1, 0x75, 0x21, 0x4c, 0x84, 0xab, 0x09, 0x15, 0xd1, 0x11, 0x15, 0x6a, 0x42, + 0x2e, 0x0c, 0xcf, 0x48, 0xad, 0x32, 0xc3, 0xb1, 0xbf, 0x1c, 0x8f, 0xdd, 0xe7, 0xd0, 0xd5, 0x84, + 0xba, 0x44, 0x07, 0xe5, 0xe8, 0x25, 0x2c, 0x87, 0x51, 0x09, 0x2b, 0xad, 0x4a, 0x26, 0x36, 0x20, + 0x91, 0xbc, 0x99, 0x05, 0x84, 0x0e, 0x29, 0xd0, 0x1b, 0x08, 0x2f, 0x82, 0x5d, 0x31, 0xa9, 0xe7, + 0x2a, 0xb3, 0x1c, 0xf9, 0xc1, 0xc4, 0xac, 0xb5, 0x9a, 0x50, 0xc3, 0xfe, 0x09, 0x0d, 0xaa, 0xb2, + 0xec, 0xb2, 0x28, 0xf6, 0xb3, 0x6b, 0x8e, 0xa3, 0xde, 0x8b, 0x40, 0x1d, 0xa6, 0xa7, 0xd5, 0x04, + 0xcb, 0xb4, 0xbe, 0x0c, 0xd5, 0x60, 0x41, 0x20, 0x51, 0xc7, 0xd1, 0x58, 0x29, 0x80, 0xf1, 0x50, + 0xa1, 0xc6, 0xdb, 0x87, 0x12, 0x32, 0x96, 0x2c, 0x4e, 0x57, 0x23, 0x92, 0x04, 0xf2, 0x5b, 0x4a, + 0x36, 0x36, 0x59, 0x46, 0xd9, 0x22, 0x4b, 0x16, 0x27, 0x2c, 0x65, 0x01, 0x37, 0x7c, 0xe2, 0xa8, + 0x9d, 0x72, 0xe6, 0xa8, 0xcc, 0xc7, 0x06, 0x3c, 0x8a, 0x63, 0xb2, 0x80, 0x1b, 0x83, 0x72, 0x54, + 0x87, 0x45, 0x51, 0x23, 0x88, 0xe4, 0x8c, 0xca, 0x42, 0xac, 0x97, 0xa3, 0xdc, 0x92, 0x79, 0xd9, + 0x0e, 0x4b, 0x99, 0x97, 0xb6, 0x63, 0x62, 0xcd, 0x0b, 0xde, 0x3c, 0x94, 0xc5, 0x58, 0x2f, 0xa3, + 0x5e, 0x47, 0x98, 0x97, 0xf6, 0xa0, 0x9c, 0x31, 0x04, 0x17, 0xdb, 0xa6, 0xb2, 0xc4, 0x91, 0xee, + 0x44, 0x20, 0xf5, 0x19, 0x64, 0x35, 0xa1, 0x72, 0x5b, 0x51, 0x5c, 0x4e, 0xa9, 0xd6, 0x62, 0x2c, + 0x4d, 0x33, 0x05, 0x4d, 0x53, 0x72, 0x63, 0x8a, 0x4b, 0x04, 0xa3, 0x13, 0xc5, 0x65, 0x50, 0xc1, + 0xce, 0xb2, 0x4f, 0xb1, 0x8c, 0x3e, 0xc1, 0x53, 0x96, 0x63, 0xcf, 0x72, 0x34, 0x19, 0x64, 0x67, + 0x99, 0x0c, 0x6b, 0x78, 0x8d, 0x95, 0xd8, 0xfe, 0x19, 0x44, 0xf1, 0x35, 0x76, 0x84, 0xfe, 0xf1, + 0x1a, 0x1b, 0x96, 0x0e, 0x27, 0x1e, 0xe1, 0xd7, 0x4e, 0x65, 0x75, 0x92, 0xc4, 0x0b, 0x5d, 0x51, + 0x87, 0x12, 0x4f, 0x68, 0x58, 0xb0, 0x75, 0x9f, 0x80, 0x6b, 0x84, 0x33, 0x70, 0x65, 0x2d, 0x36, + 0xd8, 0x51, 0x5c, 0x9d, 0x05, 0x5b, 0x1f, 0x94, 0xb3, 0x2d, 0x10, 0x34, 0x33, 0x68, 0x33, 0xb7, + 0x63, 0xb7, 0x60, 0x94, 0xa6, 0xb2, 0x2d, 0x70, 0xc3, 0x52, 0xd4, 0x86, 0xdb, 0x92, 0x78, 0x8a, + 0x82, 0xc6, 0x8e, 0x14, 0x4b, 0x48, 0x8d, 0x93, 0x6e, 0xe5, 0x0e, 0x07, 0xff, 0xff, 0x08, 0xf0, + 0x38, 0x82, 0x59, 0x4d, 0xa8, 0xca, 0x59, 0x1c, 0xf9, 0x6c, 0x42, 0xce, 0x12, 0x7c, 0x4c, 0x73, + 0x24, 0x21, 0x53, 0x0a, 0xb1, 0x9b, 0x12, 0x45, 0xdd, 0xd8, 0xa6, 0x58, 0x83, 0x72, 0xd6, 0x4d, + 0xbc, 0xe0, 0xb1, 0x50, 0x93, 0xf7, 0x2f, 0x65, 0x23, 0xb6, 0x9b, 0xc4, 0x3c, 0x2d, 0xb2, 0x6e, + 0xe2, 0x8d, 0xa8, 0xd0, 0x3e, 0x2c, 0x74, 0x18, 0x63, 0xd3, 0x5c, 0x41, 0xd9, 0x94, 0xbb, 0xb1, + 0xaf, 0xb0, 0x23, 0xcc, 0xae, 0x9a, 0x50, 0xe7, 0x3b, 0x21, 0x21, 0x7a, 0x0b, 0xb9, 0xfe, 0x6d, + 0x5a, 0x3b, 0xe1, 0x54, 0x4d, 0xd9, 0xe4, 0x78, 0x5b, 0x11, 0x78, 0x63, 0x98, 0x1d, 0xef, 0x50, + 0x83, 0x1a, 0x74, 0x01, 0xeb, 0x2c, 0x74, 0xba, 0x60, 0xc6, 0x1a, 0x0e, 0xa8, 0xb1, 0x0c, 0xe7, + 0x3d, 0x3e, 0xd3, 0x4e, 0xd4, 0xd1, 0x1e, 0x4f, 0xa8, 0xab, 0x09, 0x75, 0x8d, 0xc6, 0x9a, 0xb0, + 0x4a, 0x29, 0xfa, 0x0b, 0x63, 0x2a, 0x8c, 0xda, 0x29, 0x9f, 0xc7, 0x9e, 0xca, 0x51, 0x0a, 0xc8, + 0x4e, 0xa5, 0x15, 0x96, 0xa2, 0x63, 0x58, 0xee, 0x30, 0x3e, 0xa7, 0x59, 0xb6, 0xd6, 0x95, 0x8c, + 0x4e, 0xf9, 0x22, 0xf6, 0xa0, 0x44, 0x71, 0x3f, 0xb6, 0x3f, 0x9d, 0x41, 0xb9, 0xa0, 0x83, 0xa5, + 0x0c, 0x4c, 0x73, 0xb2, 0xfe, 0x3c, 0x3d, 0x7b, 0x2b, 0xb7, 0xf6, 0x3c, 0x3d, 0xbb, 0x9e, 0xcb, + 0x3f, 0x4f, 0xcf, 0xe6, 0x73, 0x85, 0xcd, 0x6d, 0x4e, 0x14, 0x5f, 0x38, 0x2e, 0x6f, 0x03, 0x68, + 0x0d, 0xa6, 0x2d, 0xdb, 0xc4, 0x3d, 0x79, 0x55, 0x14, 0xb4, 0x56, 0x88, 0x24, 0xb5, 0xfc, 0x26, + 0x05, 0xd3, 0x93, 0x5d, 0xac, 0x7f, 0x3c, 0x48, 0x79, 0x08, 0xe6, 0x2f, 0xc9, 0x9c, 0xd0, 0x2d, + 0x46, 0xee, 0xd3, 0x40, 0xb1, 0xe1, 0xc6, 0xfe, 0x8b, 0x21, 0x1d, 0xd1, 0xa0, 0x5d, 0x58, 0xf0, + 0x6c, 0xdc, 0xeb, 0x3a, 0x2e, 0x36, 0x79, 0x3f, 0x4d, 0x4f, 0x72, 0xc5, 0x52, 0xe7, 0xfb, 0x83, + 0x58, 0x17, 0xdd, 0x86, 0xac, 0x43, 0xac, 0x96, 0x65, 0x6b, 0xac, 0xc7, 0x70, 0x36, 0x36, 0x5d, + 0x5a, 0x64, 0x73, 0x7e, 0xb8, 0x2a, 0xcc, 0xb0, 0x7e, 0x54, 0x2b, 0xab, 0x20, 0x4c, 0xd8, 0x17, + 0x7a, 0x0a, 0x33, 0x26, 0xa7, 0xd4, 0x92, 0x5d, 0xe5, 0xe3, 0x6e, 0x2d, 0x82, 0x78, 0xfb, 0x0f, + 0x6a, 0x62, 0x0c, 0xfa, 0x8e, 0xbf, 0xbb, 0x99, 0x71, 0x83, 0xfd, 0x60, 0xc8, 0x7d, 0x47, 0x8f, + 0x21, 0x65, 0x3b, 0x17, 0x92, 0x1d, 0x4d, 0x74, 0x0f, 0x61, 0xf6, 0x4f, 0x66, 0x7f, 0xfd, 0x75, + 0x21, 0x11, 0xbc, 0x8f, 0x3c, 0xfc, 0x53, 0x72, 0xe8, 0xc5, 0xb1, 0xff, 0x3e, 0x29, 0x9e, 0xf2, + 0x9a, 0xea, 0x6b, 0x6d, 0xe4, 0x91, 0xef, 0x33, 0xb8, 0x21, 0x34, 0x2f, 0xd5, 0x5a, 0xb3, 0xa2, + 0x35, 0x1b, 0x0d, 0xad, 0x71, 0x50, 0xce, 0x25, 0xd1, 0x2a, 0x20, 0xa1, 0x28, 0x57, 0x0e, 0x2a, + 0xcd, 0x8a, 0xa6, 0x16, 0xeb, 0xcf, 0x2a, 0xb9, 0xa9, 0x40, 0x7e, 0x54, 0x51, 0x6b, 0xc5, 0x83, + 0xda, 0x9b, 0x62, 0xe9, 0xa0, 0x92, 0x4b, 0xa1, 0x5b, 0x70, 0x53, 0xc8, 0x5f, 0x34, 0x8e, 0x8e, + 0x6a, 0xa5, 0x83, 0x8a, 0xa6, 0x56, 0x5e, 0x1c, 0x14, 0x5f, 0xe7, 0xd2, 0x68, 0x1d, 0x6e, 0x09, + 0x55, 0xf1, 0xe8, 0x75, 0x7d, 0x57, 0xce, 0xb4, 0x57, 0xac, 0x1d, 0x1c, 0xab, 0x95, 0xdc, 0xf4, + 0x5a, 0xfa, 0x17, 0xbf, 0xcd, 0x27, 0x1e, 0x3e, 0x01, 0x34, 0x7a, 0x3c, 0xd0, 0x2c, 0xa4, 0xeb, + 0x8d, 0x7a, 0x25, 0x97, 0x40, 0x59, 0xc8, 0x94, 0x8a, 0xbb, 0xfb, 0x8d, 0xbd, 0xbd, 0x5c, 0x12, + 0x2d, 0xc0, 0x5c, 0xed, 0xf0, 0xb0, 0x52, 0xae, 0x15, 0x9b, 0x95, 0xdc, 0x54, 0xe9, 0xee, 0xfb, + 0x7f, 0xe6, 0x13, 0xef, 0xaf, 0xf3, 0xc9, 0xbf, 0x5c, 0xe7, 0x93, 0x7f, 0xbd, 0xce, 0x27, 0xff, + 0x71, 0x9d, 0x4f, 0xfe, 0xf2, 0x5f, 0xf9, 0xc4, 0x9b, 0x8c, 0xdc, 0xf6, 0xff, 0x06, 0x00, 0x00, + 0xff, 0xff, 0xfd, 0xba, 0x19, 0x88, 0x7f, 0x1b, 0x00, 0x00, } diff --git a/pkg/roachpb/errors.proto b/pkg/roachpb/errors.proto index 826bdb48cd2b..557af72ad32e 100644 --- a/pkg/roachpb/errors.proto +++ b/pkg/roachpb/errors.proto @@ -341,14 +341,6 @@ message HandledRetryableTxnError { optional Transaction transaction = 3 [(gogoproto.nullable) = false]; } -// A TxnPrevAttemptError indicates a batch response for a request sent through -// client Txn was found that was meant for a previous incarnation of the -// transaction. This can happen in cases where concurrent requests are made -// for a Transaction and one of the requests results in a Txn abort. -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. @@ -407,7 +399,7 @@ message MergeInProgressError { message ErrorDetail { option (gogoproto.equal) = true; - reserved 25, 29; + reserved 25, 29, 30; oneof value { NotLeaseHolderError not_lease_holder = 1; RangeNotFoundError range_not_found = 2; @@ -436,7 +428,6 @@ message ErrorDetail { // needs to be communicated from the TxnCoordSender and Txn to the upper // layers through the Sender interface. HandledRetryableTxnError handled_retryable_txn_error = 28; - TxnPrevAttemptError txn_aborted_async_err = 30; IntegerOverflowError integer_overflow = 31; UnsupportedRequestError unsupported_request = 32; MixedSuccessError mixed_success = 33; diff --git a/pkg/sql/conn_executor.go b/pkg/sql/conn_executor.go index 1e24f7b003d3..06b147972567 100644 --- a/pkg/sql/conn_executor.go +++ b/pkg/sql/conn_executor.go @@ -42,6 +42,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/util" "github.com/cockroachdb/cockroach/pkg/util/envutil" "github.com/cockroachdb/cockroach/pkg/util/fsm" + "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/metric" "github.com/cockroachdb/cockroach/pkg/util/mon" @@ -1575,13 +1576,14 @@ func (ex *connExecutor) synchronizeParallelStmts(ctx context.Context) error { defer ex.state.mu.Unlock() // Sort the errors according to their importance. - curTxn := ex.state.mu.txn.Proto() + curTxnID := ex.state.mu.txn.ID() + curTxnEpoch := ex.state.mu.txn.Epoch() sort.Slice(errs, func(i, j int) bool { errPriority := func(err error) int { switch t := err.(type) { case *roachpb.HandledRetryableTxnError: errTxn := t.Transaction - if errTxn.ID == curTxn.ID && errTxn.Epoch == curTxn.Epoch { + if errTxn.ID == curTxnID && errTxn.Epoch == curTxnEpoch { // A retryable error for the current transaction // incarnation is given the highest priority. return 1 @@ -1590,9 +1592,6 @@ func (ex *connExecutor) synchronizeParallelStmts(ctx context.Context) error { case *roachpb.TxnAlreadyEncounteredErrorError: // Another parallel stmt got an error that caused this one. return 5 - case *roachpb.TxnPrevAttemptError: - // Symptom of concurrent retry. - return 3 default: // Any other error. We sort these behind retryable errors // and errors we know to be their symptoms because it is @@ -1617,10 +1616,7 @@ func (ex *connExecutor) synchronizeParallelStmts(ctx context.Context) error { // these writes might have been performed at the wrong epoch). Note // that we don't need to lock the client.Txn because we're synchronized. // See #17197. - ex.state.mu.txn.BumpEpochAfterConcurrentRetryErrorLocked() - case *roachpb.TxnPrevAttemptError: - log.Fatalf(ctx, "found symptoms of a concurrent retry, but did "+ - "not find the final retry error: %v", errs) + ex.state.mu.txn.ManualRestart(ctx, hlc.Timestamp{}) } return bestErr } diff --git a/pkg/sql/conn_executor_exec.go b/pkg/sql/conn_executor_exec.go index ccb1135fa55f..a4fefec98a93 100644 --- a/pkg/sql/conn_executor_exec.go +++ b/pkg/sql/conn_executor_exec.go @@ -413,7 +413,7 @@ func (ex *connExecutor) execStmtInOpenState( IsCommit: fsm.FromBool(isCommit(stmt.AST)), CanAutoRetry: fsm.FromBool(canAutoRetry), } - txn.Proto().Restart(0 /* userPriority */, 0 /* upgradePriority */, ex.server.cfg.Clock.Now()) + txn.ManualRestart(ctx, ex.server.cfg.Clock.Now()) payload := eventRetriableErrPayload{ err: roachpb.NewHandledRetryableTxnError( "serializable transaction timestamp pushed (detected by connExecutor)", @@ -494,7 +494,7 @@ func (ex *connExecutor) checkTableTwoVersionInvariant(ctx context.Context) error `cannot publish new versions for tables: %v, old versions still in use`, tables), txn.ID(), - *txn.Proto(), + *txn.Serialize(), ) // We cleanup the transaction and create a new transaction after // waiting for the invariant to be satisfied because the wait time @@ -504,7 +504,7 @@ func (ex *connExecutor) checkTableTwoVersionInvariant(ctx context.Context) error // TODO(vivek): Change this to restart a txn while fixing #20526 . All the // table descriptor intents can be laid down here after the invariant // has been checked. - isolation := txn.Proto().Isolation + isolation := txn.Isolation() userPriority := txn.UserPriority() // We cleanup the transaction and create a new transaction wait time // might be extensive and so we'd better get rid of all the intents. diff --git a/pkg/sql/conn_fsm.go b/pkg/sql/conn_fsm.go index 2224fb9a2527..93628dbe6c87 100644 --- a/pkg/sql/conn_fsm.go +++ b/pkg/sql/conn_fsm.go @@ -384,8 +384,7 @@ var TxnStateTransitions = Compile(Pattern{ // rollback to a regular savepoint, clearly we couldn't bump the // timestamp in that case. In the special case of the cockroach_restart // savepoint, it's not clear to me what a user's expectation might be. - state.mu.txn.Proto().Restart( - 0 /* userPriority */, 0 /* upgradePriority */, hlc.Timestamp{}) + state.mu.txn.ManualRestart(args.Ctx, hlc.Timestamp{}) args.Extended.(*txnState).setAdvanceInfo(advanceOne, noRewind, txnRestart) return nil }, diff --git a/pkg/sql/distsql_running.go b/pkg/sql/distsql_running.go index bc7e1234c3a7..a0805042183f 100644 --- a/pkg/sql/distsql_running.go +++ b/pkg/sql/distsql_running.go @@ -119,13 +119,6 @@ func (dsp *DistSQLPlanner) Run( var txnCoordMeta *roachpb.TxnCoordMeta if txn != nil { meta := txn.GetStrippedTxnCoordMeta() - // The txn proto in the TxnCoordSender can get out of sync with the - // txn proto held in client.Txn. We want to make sure all of the - // information makes it into the TxnCoordMeta, so update the proto - // accordingly. - // TODO(andrei): Without this a few tests fail. Remove this hack - // once we merge the txn protos. - meta.Txn = *txn.Proto() txnCoordMeta = &meta } diff --git a/pkg/sql/distsqlplan/aggregator_funcs_test.go b/pkg/sql/distsqlplan/aggregator_funcs_test.go index af94f8111a2a..99f6f3b159a2 100644 --- a/pkg/sql/distsqlplan/aggregator_funcs_test.go +++ b/pkg/sql/distsqlplan/aggregator_funcs_test.go @@ -22,7 +22,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/internal/client" - "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/sql/distsqlrun" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sqlbase" @@ -58,7 +57,7 @@ func runTestFlow( ) sqlbase.EncDatumRows { distSQLSrv := srv.DistSQLServer().(*distsqlrun.ServerImpl) - txnCoordMeta := roachpb.MakeTxnCoordMeta(*txn.Proto()) + txnCoordMeta := txn.GetStrippedTxnCoordMeta() req := distsqlrun.SetupFlowRequest{ Version: distsqlrun.Version, TxnCoordMeta: &txnCoordMeta, diff --git a/pkg/sql/distsqlrun/server_test.go b/pkg/sql/distsqlrun/server_test.go index 064b9c7aa437..eb7e026391ce 100644 --- a/pkg/sql/distsqlrun/server_test.go +++ b/pkg/sql/distsqlrun/server_test.go @@ -23,7 +23,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/gossip" "github.com/cockroachdb/cockroach/pkg/internal/client" - "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/sql/sqlbase" "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" @@ -62,7 +61,7 @@ func TestServer(t *testing.T) { } txn := client.NewTxn(kvDB, s.NodeID(), client.RootTxn) - txnCoordMeta := roachpb.MakeTxnCoordMeta(*txn.Proto()) + txnCoordMeta := txn.GetStrippedTxnCoordMeta() req := &SetupFlowRequest{Version: Version, TxnCoordMeta: &txnCoordMeta} req.Flow = FlowSpec{ diff --git a/pkg/sql/logictest/testdata/logic_test/show_trace b/pkg/sql/logictest/testdata/logic_test/show_trace index 6a3839770345..347d47a5bfaf 100644 --- a/pkg/sql/logictest/testdata/logic_test/show_trace +++ b/pkg/sql/logictest/testdata/logic_test/show_trace @@ -23,10 +23,12 @@ WHERE message LIKE '%SPAN START%' OR message LIKE '%pos%executing%'; 1 [Open pos:?] executing ExecStmt: SELECT 1 sql txn 2 === SPAN START: consuming rows === consuming rows 1 [Open pos:?] executing ExecStmt: COMMIT TRANSACTION sql txn +3 === SPAN START: txn coordinator send === txn coordinator send 0 [NoTxn pos:?] executing ExecStmt: SELECT 2 session recording -3 === SPAN START: sql txn === sql txn -3 [Open pos:?] executing ExecStmt: SELECT 2 sql txn -4 === SPAN START: consuming rows === consuming rows +4 === SPAN START: sql txn === sql txn +4 [Open pos:?] executing ExecStmt: SELECT 2 sql txn +5 === SPAN START: consuming rows === consuming rows +6 === SPAN START: txn coordinator send === txn coordinator send 0 [NoTxn pos:?] executing ExecStmt: SET TRACING = off session recording # Check SHOW KV TRACE FOR SESSION. @@ -43,17 +45,17 @@ WHERE message NOT LIKE '%Z/%' AND tag NOT LIKE '%scExec=%' AND tag NOT LIKE '%IndexBackfiller%' ---- -dist sender querying next range at /Table/2/1/0/"t"/3/1 -dist sender r1: sending batch 1 Get to (n1,s1):1 -dist sender querying next range at /System/"desc-idgen" -dist sender r1: sending batch 1 Inc to (n1,s1):1 -sql txn CPut /Table/2/1/0/"t"/3/1 -> 53 -sql txn CPut /Table/3/1/53/2/1 -> database: users: > > -dist sender querying next range at /Table/SystemConfigSpan/Start -dist sender r1: sending batch 2 CPut, 1 BeginTxn to (n1,s1):1 -sql txn rows affected: 0 -dist sender querying next range at /Table/SystemConfigSpan/Start -dist sender r1: sending batch 1 EndTxn, 7 QueryIntent to (n1,s1):1 +dist sender send querying next range at /Table/2/1/0/"t"/3/1 +dist sender send r1: sending batch 1 Get to (n1,s1):1 +dist sender send querying next range at /System/"desc-idgen" +dist sender send r1: sending batch 1 Inc to (n1,s1):1 +sql txn CPut /Table/2/1/0/"t"/3/1 -> 53 +sql txn CPut /Table/3/1/53/2/1 -> database: users: > > +dist sender send querying next range at /Table/SystemConfigSpan/Start +dist sender send r1: sending batch 2 CPut, 1 BeginTxn to (n1,s1):1 +sql txn rows affected: 0 +dist sender send querying next range at /Table/SystemConfigSpan/Start +dist sender send r1: sending batch 1 EndTxn, 7 QueryIntent to (n1,s1):1 # More KV operations. @@ -68,27 +70,27 @@ WHERE message NOT LIKE '%Z/%' AND tag NOT LIKE '%scExec=%' AND tag NOT LIKE '%IndexBackfiller%' ---- -dist sender querying next range at /Table/2/1/0/"test"/3/1 -dist sender r1: sending batch 1 Get to (n1,s1):1 -dist sender querying next range at /Table/3/1/52/2/1 -dist sender r1: sending batch 1 Get to (n1,s1):1 -dist sender querying next range at /Table/2/1/0/"t"/3/1 -dist sender r1: sending batch 1 Get to (n1,s1):1 -dist sender querying next range at /Table/3/1/53/2/1 -dist sender r1: sending batch 1 Get to (n1,s1):1 -dist sender querying next range at /Table/2/1/53/"kv"/3/1 -dist sender r1: sending batch 1 Get to (n1,s1):1 -dist sender querying next range at /System/"desc-idgen" -dist sender r1: sending batch 1 Inc to (n1,s1):1 -sql txn CPut /Table/2/1/53/"kv"/3/1 -> 54 -sql txn CPut /Table/3/1/54/2/1 -> table: columns: nullable:false hidden:false > columns: nullable:true hidden:false > next_column_id:3 families: next_family_id:1 primary_index: interleave:<> partitioning: type:FORWARD > next_index_id:2 privileges: users: > next_mutation_id:1 format_version:3 state:PUBLIC view_query:"" drop_time:0 replacement_of: > audit_mode:DISABLED > -dist sender querying next range at /Table/SystemConfigSpan/Start -dist sender r1: sending batch 2 CPut, 1 BeginTxn to (n1,s1):1 -dist sender querying next range at /Table/3/1/53/2/1 -dist sender r1: sending batch 1 Get to (n1,s1):1 -sql txn rows affected: 0 -dist sender querying next range at /Table/SystemConfigSpan/Start -dist sender r1: sending batch 1 EndTxn, 7 QueryIntent to (n1,s1):1 +dist sender send querying next range at /Table/2/1/0/"test"/3/1 +dist sender send r1: sending batch 1 Get to (n1,s1):1 +dist sender send querying next range at /Table/3/1/52/2/1 +dist sender send r1: sending batch 1 Get to (n1,s1):1 +dist sender send querying next range at /Table/2/1/0/"t"/3/1 +dist sender send r1: sending batch 1 Get to (n1,s1):1 +dist sender send querying next range at /Table/3/1/53/2/1 +dist sender send r1: sending batch 1 Get to (n1,s1):1 +dist sender send querying next range at /Table/2/1/53/"kv"/3/1 +dist sender send r1: sending batch 1 Get to (n1,s1):1 +dist sender send querying next range at /System/"desc-idgen" +dist sender send r1: sending batch 1 Inc to (n1,s1):1 +sql txn CPut /Table/2/1/53/"kv"/3/1 -> 54 +sql txn CPut /Table/3/1/54/2/1 -> table: columns: nullable:false hidden:false > columns: nullable:true hidden:false > next_column_id:3 families: next_family_id:1 primary_index: interleave:<> partitioning: type:FORWARD > next_index_id:2 privileges: users: > next_mutation_id:1 format_version:3 state:PUBLIC view_query:"" drop_time:0 replacement_of: > audit_mode:DISABLED > +dist sender send querying next range at /Table/SystemConfigSpan/Start +dist sender send r1: sending batch 2 CPut, 1 BeginTxn to (n1,s1):1 +dist sender send querying next range at /Table/3/1/53/2/1 +dist sender send r1: sending batch 1 Get to (n1,s1):1 +sql txn rows affected: 0 +dist sender send querying next range at /Table/SystemConfigSpan/Start +dist sender send r1: sending batch 1 EndTxn, 7 QueryIntent to (n1,s1):1 # We avoid using the full trace output, because that would make the # ensuing trace especially chatty, as it traces the index backfill at @@ -108,22 +110,22 @@ WHERE message NOT LIKE '%Z/%' AND tag NOT LIKE '%scExec=%' AND tag NOT LIKE '%IndexBackfiller%' ---- -dist sender querying next range at /Table/2/1/0/"t"/3/1 -dist sender r1: sending batch 1 Get to (n1,s1):1 -dist sender querying next range at /Table/3/1/53/2/1 -dist sender r1: sending batch 1 Get to (n1,s1):1 -dist sender querying next range at /Table/2/1/53/"kv"/3/1 -dist sender r1: sending batch 1 Get to (n1,s1):1 -dist sender querying next range at /Table/3/1/54/2/1 -dist sender r1: sending batch 1 Get to (n1,s1):1 -dist sender querying next range at /Table/3/1/53/2/1 -dist sender r1: sending batch 1 Get to (n1,s1):1 -sql txn Put /Table/3/1/54/2/1 -> table: columns: nullable:false hidden:false > columns: nullable:true hidden:false > next_column_id:3 families: next_family_id:1 primary_index: interleave:<> partitioning: type:FORWARD > next_index_id:3 privileges: users: > mutations: interleave:<> partitioning: type:FORWARD > state:DELETE_ONLY direction:ADD mutation_id:1 rollback:false > next_mutation_id:2 format_version:3 state:PUBLIC view_query:"" mutationJobs:<...> drop_time:0 replacement_of: > audit_mode:DISABLED > -dist sender querying next range at /Table/3/1/54/2/1 -dist sender r1: sending batch 1 Put to (n1,s1):1 -sql txn rows affected: 0 -dist sender querying next range at /Table/SystemConfigSpan/Start -dist sender r1: sending batch 1 EndTxn, 9 QueryIntent to (n1,s1):1 +dist sender send querying next range at /Table/2/1/0/"t"/3/1 +dist sender send r1: sending batch 1 Get to (n1,s1):1 +dist sender send querying next range at /Table/3/1/53/2/1 +dist sender send r1: sending batch 1 Get to (n1,s1):1 +dist sender send querying next range at /Table/2/1/53/"kv"/3/1 +dist sender send r1: sending batch 1 Get to (n1,s1):1 +dist sender send querying next range at /Table/3/1/54/2/1 +dist sender send r1: sending batch 1 Get to (n1,s1):1 +dist sender send querying next range at /Table/3/1/53/2/1 +dist sender send r1: sending batch 1 Get to (n1,s1):1 +sql txn Put /Table/3/1/54/2/1 -> table: columns: nullable:false hidden:false > columns: nullable:true hidden:false > next_column_id:3 families: next_family_id:1 primary_index: interleave:<> partitioning: type:FORWARD > next_index_id:3 privileges: users: > mutations: interleave:<> partitioning: type:FORWARD > state:DELETE_ONLY direction:ADD mutation_id:1 rollback:false > next_mutation_id:2 format_version:3 state:PUBLIC view_query:"" mutationJobs:<...> drop_time:0 replacement_of: > audit_mode:DISABLED > +dist sender send querying next range at /Table/3/1/54/2/1 +dist sender send r1: sending batch 1 Put to (n1,s1):1 +sql txn rows affected: 0 +dist sender send querying next range at /Table/SystemConfigSpan/Start +dist sender send r1: sending batch 1 EndTxn, 9 QueryIntent to (n1,s1):1 statement ok SET tracing = on,kv,results; INSERT INTO t.kv(k, v) VALUES (1,2); SET tracing = off @@ -131,12 +133,12 @@ SET tracing = on,kv,results; INSERT INTO t.kv(k, v) VALUES (1,2); SET tracing = query TT SELECT operation, message FROM [SHOW KV TRACE FOR SESSION] ---- -sql txn CPut /Table/54/1/1/0 -> /TUPLE/2:2:Int/2 -sql txn InitPut /Table/54/2/2/0 -> /BYTES/0x89 -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 -sql txn fast path completed -sql txn rows affected: 1 +sql txn CPut /Table/54/1/1/0 -> /TUPLE/2:2:Int/2 +sql txn InitPut /Table/54/2/2/0 -> /BYTES/0x89 +dist sender send querying next range at /Table/54/1/1/0 +dist sender send r1: sending batch 1 CPut, 1 BeginTxn, 1 EndTxn, 1 InitPut to (n1,s1):1 +sql txn fast path completed +sql txn rows affected: 1 statement error duplicate key value @@ -146,13 +148,13 @@ query TT set tracing=off; SELECT operation, message FROM [SHOW KV TRACE FOR SESSION] ---- -sql txn CPut /Table/54/1/1/0 -> /TUPLE/2:2:Int/2 -sql txn InitPut /Table/54/2/2/0 -> /BYTES/0x89 -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 -sql txn execution failed after 0 rows: duplicate key value (k)=(1) violates unique constraint "primary" -dist sender querying next range at /Table/54/1/1/0 -dist sender r1: sending batch 1 EndTxn to (n1,s1):1 +sql txn CPut /Table/54/1/1/0 -> /TUPLE/2:2:Int/2 +sql txn InitPut /Table/54/2/2/0 -> /BYTES/0x89 +dist sender send querying next range at /Table/54/1/1/0 +dist sender send r1: sending batch 1 CPut, 1 BeginTxn, 1 EndTxn, 1 InitPut to (n1,s1):1 +sql txn execution failed after 0 rows: duplicate key value (k)=(1) violates unique constraint "primary" +dist sender send querying next range at /Table/54/1/1/0 +dist sender send r1: sending batch 1 EndTxn to (n1,s1):1 statement error duplicate key value SET tracing = on,kv,results; INSERT INTO t.kv(k, v) VALUES (2,2); SET tracing = off @@ -161,13 +163,13 @@ query TT set tracing=off; SELECT operation, message FROM [SHOW KV TRACE FOR SESSION] ---- -sql txn CPut /Table/54/1/2/0 -> /TUPLE/2:2:Int/2 -sql txn InitPut /Table/54/2/2/0 -> /BYTES/0x8a -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 -sql txn execution failed after 0 rows: duplicate key value (v)=(2) violates unique constraint "woo" -dist sender querying next range at /Table/54/1/2/0 -dist sender r1: sending batch 1 EndTxn to (n1,s1):1 +sql txn CPut /Table/54/1/2/0 -> /TUPLE/2:2:Int/2 +sql txn InitPut /Table/54/2/2/0 -> /BYTES/0x8a +dist sender send querying next range at /Table/54/1/2/0 +dist sender send r1: sending batch 1 CPut, 1 BeginTxn, 1 EndTxn, 1 InitPut to (n1,s1):1 +sql txn execution failed after 0 rows: duplicate key value (v)=(2) violates unique constraint "woo" +dist sender send querying next range at /Table/54/1/2/0 +dist sender send r1: sending batch 1 EndTxn to (n1,s1):1 statement ok SET tracing = on,kv,results; UPSERT INTO t.kv(k, v) VALUES (2,3); SET tracing = off @@ -175,15 +177,15 @@ SET tracing = on,kv,results; UPSERT INTO t.kv(k, v) VALUES (2,3); SET tracing = query TT SELECT operation, message FROM [SHOW KV TRACE FOR SESSION] ---- -sql txn Scan /Table/54/1/{2-3} -dist sender querying next range at /Table/54/1/2 -dist sender r1: sending batch 1 Scan to (n1,s1):1 -sql txn CPut /Table/54/1/2/0 -> /TUPLE/2:2:Int/3 -sql txn InitPut /Table/54/2/3/0 -> /BYTES/0x8a -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 -sql txn fast path completed -sql txn rows affected: 1 +sql txn Scan /Table/54/1/{2-3} +dist sender send querying next range at /Table/54/1/2 +dist sender send r1: sending batch 1 Scan to (n1,s1):1 +sql txn CPut /Table/54/1/2/0 -> /TUPLE/2:2:Int/3 +sql txn InitPut /Table/54/2/3/0 -> /BYTES/0x8a +dist sender send querying next range at /Table/54/1/2/0 +dist sender send r1: sending batch 1 CPut, 1 BeginTxn, 1 EndTxn, 1 InitPut to (n1,s1):1 +sql txn fast path completed +sql txn rows affected: 1 statement ok SET tracing = on,kv,results; UPSERT INTO t.kv(k, v) VALUES (1,2); SET tracing = off @@ -191,15 +193,15 @@ SET tracing = on,kv,results; UPSERT INTO t.kv(k, v) VALUES (1,2); SET tracing = query TT SELECT operation, message FROM [SHOW KV TRACE FOR SESSION] ---- -sql txn Scan /Table/54/1/{1-2} -dist sender querying next range at /Table/54/1/1 -dist sender r1: sending batch 1 Scan to (n1,s1):1 -sql txn fetched: /kv/primary/1/v -> /2 -sql txn Put /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 Put, 1 BeginTxn, 1 EndTxn to (n1,s1):1 -sql txn fast path completed -sql txn rows affected: 1 +sql txn Scan /Table/54/1/{1-2} +dist sender send querying next range at /Table/54/1/1 +dist sender send r1: sending batch 1 Scan to (n1,s1):1 +sql txn fetched: /kv/primary/1/v -> /2 +sql txn Put /Table/54/1/1/0 -> /TUPLE/2:2:Int/2 +dist sender send querying next range at /Table/54/1/1/0 +dist sender send r1: sending batch 1 Put, 1 BeginTxn, 1 EndTxn to (n1,s1):1 +sql txn fast path completed +sql txn rows affected: 1 statement error duplicate key value SET tracing = on,kv,results; UPSERT INTO t.kv(k, v) VALUES (2,2); SET tracing = off @@ -208,18 +210,18 @@ query TT set tracing=off; SELECT operation, message FROM [SHOW KV TRACE FOR SESSION] ---- -sql txn Scan /Table/54/1/{2-3} -dist sender querying next range at /Table/54/1/2 -dist sender r1: sending batch 1 Scan to (n1,s1):1 -sql txn fetched: /kv/primary/2/v -> /3 -sql txn Put /Table/54/1/2/0 -> /TUPLE/2:2:Int/2 -sql txn Del /Table/54/2/3/0 -sql txn CPut /Table/54/2/2/0 -> /BYTES/0x8a -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 -sql txn execution failed after 0 rows: duplicate key value (v)=(2) violates unique constraint "woo" -dist sender querying next range at /Table/54/1/2/0 -dist sender r1: sending batch 1 EndTxn to (n1,s1):1 +sql txn Scan /Table/54/1/{2-3} +dist sender send querying next range at /Table/54/1/2 +dist sender send r1: sending batch 1 Scan to (n1,s1):1 +sql txn fetched: /kv/primary/2/v -> /3 +sql txn Put /Table/54/1/2/0 -> /TUPLE/2:2:Int/2 +sql txn Del /Table/54/2/3/0 +sql txn CPut /Table/54/2/2/0 -> /BYTES/0x8a +dist sender send querying next range at /Table/54/1/2/0 +dist sender send r1: sending batch 1 Put, 1 CPut, 1 Del, 1 BeginTxn, 1 EndTxn to (n1,s1):1 +sql txn execution failed after 0 rows: duplicate key value (v)=(2) violates unique constraint "woo" +dist sender send querying next range at /Table/54/1/2/0 +dist sender send r1: sending batch 1 EndTxn to (n1,s1):1 statement ok SET tracing = on,kv,results; CREATE TABLE t.kv2 AS TABLE t.kv; SET tracing = off @@ -232,35 +234,35 @@ WHERE message NOT LIKE '%Z/%' AND tag NOT LIKE '%scExec=%' AND tag NOT LIKE '%IndexBackfiller%' ---- -dist sender querying next range at /Table/2/1/0/"test"/3/1 -dist sender r1: sending batch 1 Get to (n1,s1):1 -dist sender querying next range at /Table/3/1/52/2/1 -dist sender r1: sending batch 1 Get to (n1,s1):1 -dist sender querying next range at /Table/2/1/0/"t"/3/1 -dist sender r1: sending batch 1 Get to (n1,s1):1 -dist sender querying next range at /Table/3/1/53/2/1 -dist sender r1: sending batch 1 Get to (n1,s1):1 -dist sender querying next range at /Table/2/1/53/"kv2"/3/1 -dist sender r1: sending batch 1 Get to (n1,s1):1 -dist sender querying next range at /System/"desc-idgen" -dist sender r1: sending batch 1 Inc to (n1,s1):1 -sql txn CPut /Table/2/1/53/"kv2"/3/1 -> 55 -sql txn CPut /Table/3/1/55/2/1 -> table: columns: nullable:true hidden:false > columns: nullable:true hidden:false > columns: nullable:false default_expr:"unique_rowid()" hidden:true > next_column_id:4 families: next_family_id:1 primary_index: interleave:<> partitioning: type:FORWARD > next_index_id:2 privileges: users: > next_mutation_id:1 format_version:3 state:PUBLIC view_query:"" drop_time:0 replacement_of: > audit_mode:DISABLED > -dist sender querying next range at /Table/SystemConfigSpan/Start -dist sender r1: sending batch 2 CPut, 1 BeginTxn to (n1,s1):1 -dist sender querying next range at /Table/3/1/53/2/1 -dist sender r1: sending batch 1 Get to (n1,s1):1 -sql txn Scan /Table/54/{1-2} -dist sender querying next range at /Table/54/1 -dist sender r1: sending batch 1 Scan to (n1,s1):1 -sql txn fetched: /kv/primary/1/v -> /2 -sql txn CPut /Table/55/1/...PK.../0 -> /TUPLE/1:1:Int/1/1:2:Int/2 -sql txn fetched: /kv/primary/2/v -> /3 -sql txn CPut /Table/55/1/...PK.../0 -> /TUPLE/1:1:Int/2/1:2:Int/3 -dist sender querying next range at /Table/SystemConfigSpan/Start -dist sender r1: sending batch 2 CPut, 1 EndTxn, 7 QueryIntent to (n1,s1):1 -sql txn fast path completed -sql txn rows affected: 2 +dist sender send querying next range at /Table/2/1/0/"test"/3/1 +dist sender send r1: sending batch 1 Get to (n1,s1):1 +dist sender send querying next range at /Table/3/1/52/2/1 +dist sender send r1: sending batch 1 Get to (n1,s1):1 +dist sender send querying next range at /Table/2/1/0/"t"/3/1 +dist sender send r1: sending batch 1 Get to (n1,s1):1 +dist sender send querying next range at /Table/3/1/53/2/1 +dist sender send r1: sending batch 1 Get to (n1,s1):1 +dist sender send querying next range at /Table/2/1/53/"kv2"/3/1 +dist sender send r1: sending batch 1 Get to (n1,s1):1 +dist sender send querying next range at /System/"desc-idgen" +dist sender send r1: sending batch 1 Inc to (n1,s1):1 +sql txn CPut /Table/2/1/53/"kv2"/3/1 -> 55 +sql txn CPut /Table/3/1/55/2/1 -> table: columns: nullable:true hidden:false > columns: nullable:true hidden:false > columns: nullable:false default_expr:"unique_rowid()" hidden:true > next_column_id:4 families: next_family_id:1 primary_index: interleave:<> partitioning: type:FORWARD > next_index_id:2 privileges: users: > next_mutation_id:1 format_version:3 state:PUBLIC view_query:"" drop_time:0 replacement_of: > audit_mode:DISABLED > +dist sender send querying next range at /Table/SystemConfigSpan/Start +dist sender send r1: sending batch 2 CPut, 1 BeginTxn to (n1,s1):1 +dist sender send querying next range at /Table/3/1/53/2/1 +dist sender send r1: sending batch 1 Get to (n1,s1):1 +sql txn Scan /Table/54/{1-2} +dist sender send querying next range at /Table/54/1 +dist sender send r1: sending batch 1 Scan to (n1,s1):1 +sql txn fetched: /kv/primary/1/v -> /2 +sql txn CPut /Table/55/1/...PK.../0 -> /TUPLE/1:1:Int/1/1:2:Int/2 +sql txn fetched: /kv/primary/2/v -> /3 +sql txn CPut /Table/55/1/...PK.../0 -> /TUPLE/1:1:Int/2/1:2:Int/3 +dist sender send querying next range at /Table/SystemConfigSpan/Start +dist sender send r1: sending batch 2 CPut, 1 EndTxn, 7 QueryIntent to (n1,s1):1 +sql txn fast path completed +sql txn rows affected: 2 statement ok SET tracing = on,kv,results; UPDATE t.kv2 SET v = v + 2; SET tracing = off @@ -273,22 +275,22 @@ WHERE message NOT LIKE '%Z/%' AND tag NOT LIKE '%scExec=%' AND tag NOT LIKE '%IndexBackfiller%' ---- -dist sender querying next range at /Table/2/1/53/"kv2"/3/1 -dist sender r1: sending batch 1 Get to (n1,s1):1 -dist sender querying next range at /Table/3/1/55/2/1 -dist sender r1: sending batch 1 Get to (n1,s1):1 -dist sender r1: sending batch 1 EndTxn, 1 QueryIntent to (n1,s1):1 -sql txn Scan /Table/55/{1-2} -dist sender querying next range at /Table/55/1 -dist sender r1: sending batch 1 Scan to (n1,s1):1 -sql txn fetched: /kv2/primary/...PK.../k/v -> /1/2 -sql txn Put /Table/55/1/...PK.../0 -> /TUPLE/1:1:Int/1/1:2:Int/4 -sql txn fetched: /kv2/primary/...PK.../k/v -> /2/3 -sql txn Put /Table/55/1/...PK.../0 -> /TUPLE/1:1:Int/2/1:2:Int/5 -dist sender querying next range at /Table/55/1/...PK.../0 -dist sender r1: sending batch 2 Put, 1 BeginTxn, 1 EndTxn to (n1,s1):1 -sql txn fast path completed -sql txn rows affected: 2 +dist sender send querying next range at /Table/2/1/53/"kv2"/3/1 +dist sender send r1: sending batch 1 Get to (n1,s1):1 +dist sender send querying next range at /Table/3/1/55/2/1 +dist sender send r1: sending batch 1 Get to (n1,s1):1 +dist sender send r1: sending batch 1 EndTxn, 1 QueryIntent to (n1,s1):1 +sql txn Scan /Table/55/{1-2} +dist sender send querying next range at /Table/55/1 +dist sender send r1: sending batch 1 Scan to (n1,s1):1 +sql txn fetched: /kv2/primary/...PK.../k/v -> /1/2 +sql txn Put /Table/55/1/...PK.../0 -> /TUPLE/1:1:Int/1/1:2:Int/4 +sql txn fetched: /kv2/primary/...PK.../k/v -> /2/3 +sql txn Put /Table/55/1/...PK.../0 -> /TUPLE/1:1:Int/2/1:2:Int/5 +dist sender send querying next range at /Table/55/1/...PK.../0 +dist sender send r1: sending batch 2 Put, 1 BeginTxn, 1 EndTxn to (n1,s1):1 +sql txn fast path completed +sql txn rows affected: 2 statement ok SET tracing = on,kv,results; DELETE FROM t.kv2; SET tracing = off @@ -296,14 +298,14 @@ SET tracing = on,kv,results; DELETE FROM t.kv2; SET tracing = off query TT SELECT operation, message FROM [SHOW KV TRACE FOR SESSION] ---- -sql txn Scan /Table/55/{1-2} -dist sender querying next range at /Table/55/1 -dist sender r1: sending batch 1 Scan to (n1,s1):1 -sql txn DelRange /Table/55/1 - /Table/55/2 -dist sender querying next range at /Table/55/1 -dist sender r1: sending batch 1 DelRng, 1 BeginTxn, 1 EndTxn to (n1,s1):1 -sql txn fast path completed -sql txn rows affected: 2 +sql txn Scan /Table/55/{1-2} +dist sender send querying next range at /Table/55/1 +dist sender send r1: sending batch 1 Scan to (n1,s1):1 +sql txn DelRange /Table/55/1 - /Table/55/2 +dist sender send querying next range at /Table/55/1 +dist sender send r1: sending batch 1 DelRng, 1 BeginTxn, 1 EndTxn to (n1,s1):1 +sql txn fast path completed +sql txn rows affected: 2 statement ok SET tracing = on,kv,results; DROP TABLE t.kv2; SET tracing = off @@ -317,32 +319,32 @@ WHERE message NOT LIKE '%Z/%' AND tag NOT LIKE '%scExec=%' AND tag NOT LIKE '%IndexBackfiller%' ---- -dist sender querying next range at /Table/2/1/0/"t"/3/1 -dist sender r1: sending batch 1 Get to (n1,s1):1 -dist sender querying next range at /Table/3/1/53/2/1 -dist sender r1: sending batch 1 Get to (n1,s1):1 -dist sender querying next range at /Table/2/1/53/"kv2"/3/1 -dist sender r1: sending batch 1 Get to (n1,s1):1 -dist sender querying next range at /Table/3/1/55/2/1 -dist sender r1: sending batch 1 Get to (n1,s1):1 -dist sender querying next range at /Table/3/1/53/2/1 -dist sender r1: sending batch 1 Get to (n1,s1):1 -dist sender querying next range at /Table/5/1/55/2/1 -dist sender r1: sending batch 1 Get to (n1,s1):1 -dist sender querying next range at /Table/3/1/55/2/1 -dist sender r1: sending batch 1 Get to (n1,s1):1 -dist sender querying next range at /Table/5/1/53/2/1 -dist sender r1: sending batch 1 Get to (n1,s1):1 -dist sender querying next range at /Table/3/1/53/2/1 -dist sender r1: sending batch 1 Get to (n1,s1):1 -dist sender querying next range at /Table/5/1/0/2/1 -dist sender r1: sending batch 1 Get to (n1,s1):1 -sql txn Put /Table/3/1/55/2/1 -> table: columns: nullable:true hidden:false > columns: nullable:true hidden:false > columns: nullable:false default_expr:"unique_rowid()" hidden:true > next_column_id:4 families: next_family_id:1 primary_index: interleave:<> partitioning: type:FORWARD > next_index_id:2 privileges: users: > next_mutation_id:1 format_version:3 state:DROP draining_names: view_query:"" drop_time:... replacement_of: > audit_mode:DISABLED > -dist sender querying next range at /Table/SystemConfigSpan/Start -dist sender r1: sending batch 1 Put, 1 BeginTxn to (n1,s1):1 -sql txn rows affected: 0 -dist sender querying next range at /Table/SystemConfigSpan/Start -dist sender r1: sending batch 1 EndTxn, 6 QueryIntent to (n1,s1):1 +dist sender send querying next range at /Table/2/1/0/"t"/3/1 +dist sender send r1: sending batch 1 Get to (n1,s1):1 +dist sender send querying next range at /Table/3/1/53/2/1 +dist sender send r1: sending batch 1 Get to (n1,s1):1 +dist sender send querying next range at /Table/2/1/53/"kv2"/3/1 +dist sender send r1: sending batch 1 Get to (n1,s1):1 +dist sender send querying next range at /Table/3/1/55/2/1 +dist sender send r1: sending batch 1 Get to (n1,s1):1 +dist sender send querying next range at /Table/3/1/53/2/1 +dist sender send r1: sending batch 1 Get to (n1,s1):1 +dist sender send querying next range at /Table/5/1/55/2/1 +dist sender send r1: sending batch 1 Get to (n1,s1):1 +dist sender send querying next range at /Table/3/1/55/2/1 +dist sender send r1: sending batch 1 Get to (n1,s1):1 +dist sender send querying next range at /Table/5/1/53/2/1 +dist sender send r1: sending batch 1 Get to (n1,s1):1 +dist sender send querying next range at /Table/3/1/53/2/1 +dist sender send r1: sending batch 1 Get to (n1,s1):1 +dist sender send querying next range at /Table/5/1/0/2/1 +dist sender send r1: sending batch 1 Get to (n1,s1):1 +sql txn Put /Table/3/1/55/2/1 -> table: columns: nullable:true hidden:false > columns: nullable:true hidden:false > columns: nullable:false default_expr:"unique_rowid()" hidden:true > next_column_id:4 families: next_family_id:1 primary_index: interleave:<> partitioning: type:FORWARD > next_index_id:2 privileges: users: > next_mutation_id:1 format_version:3 state:DROP draining_names: view_query:"" drop_time:... replacement_of: > audit_mode:DISABLED > +dist sender send querying next range at /Table/SystemConfigSpan/Start +dist sender send r1: sending batch 1 Put, 1 BeginTxn to (n1,s1):1 +sql txn rows affected: 0 +dist sender send querying next range at /Table/SystemConfigSpan/Start +dist sender send r1: sending batch 1 EndTxn, 6 QueryIntent to (n1,s1):1 statement ok SET tracing = on,kv,results; DELETE FROM t.kv; SET tracing = off @@ -350,19 +352,19 @@ SET tracing = on,kv,results; DELETE FROM t.kv; SET tracing = off query TT SELECT operation, message FROM [SHOW KV TRACE FOR SESSION] ---- -sql txn Scan /Table/54/{1-2} -dist sender querying next range at /Table/54/1 -dist sender r1: sending batch 1 Scan to (n1,s1):1 -sql txn fetched: /kv/primary/1/v -> /2 -sql txn Del /Table/54/2/2/0 -sql txn Del /Table/54/1/1/0 -sql txn fetched: /kv/primary/2/v -> /3 -sql txn Del /Table/54/2/3/0 -sql txn Del /Table/54/1/2/0 -dist sender querying next range at /Table/54/1/1/0 -dist sender r1: sending batch 4 Del, 1 BeginTxn, 1 EndTxn to (n1,s1):1 -sql txn fast path completed -sql txn rows affected: 2 +sql txn Scan /Table/54/{1-2} +dist sender send querying next range at /Table/54/1 +dist sender send r1: sending batch 1 Scan to (n1,s1):1 +sql txn fetched: /kv/primary/1/v -> /2 +sql txn Del /Table/54/2/2/0 +sql txn Del /Table/54/1/1/0 +sql txn fetched: /kv/primary/2/v -> /3 +sql txn Del /Table/54/2/3/0 +sql txn Del /Table/54/1/2/0 +dist sender send querying next range at /Table/54/1/1/0 +dist sender send r1: sending batch 4 Del, 1 BeginTxn, 1 EndTxn to (n1,s1):1 +sql txn fast path completed +sql txn rows affected: 2 statement ok SET tracing = on,kv,results; DROP INDEX t.kv@woo CASCADE; SET tracing = off @@ -376,34 +378,34 @@ WHERE message NOT LIKE '%Z/%' AND tag NOT LIKE '%scExec=%' AND tag NOT LIKE '%IndexBackfiller%' ---- -dist sender querying next range at /Table/2/1/0/"t"/3/1 -dist sender r1: sending batch 1 Get to (n1,s1):1 -dist sender querying next range at /Table/3/1/53/2/1 -dist sender r1: sending batch 1 Get to (n1,s1):1 -dist sender querying next range at /Table/2/1/53/"kv"/3/1 -dist sender r1: sending batch 1 Get to (n1,s1):1 -dist sender querying next range at /Table/3/1/54/2/1 -dist sender r1: sending batch 1 Get to (n1,s1):1 -dist sender querying next range at /Table/3/1/53/2/1 -dist sender r1: sending batch 1 Get to (n1,s1):1 -dist sender querying next range at /Table/2/1/0/"t"/3/1 -dist sender r1: sending batch 1 Get to (n1,s1):1 -dist sender querying next range at /Table/3/1/53/2/1 -dist sender r1: sending batch 1 Get to (n1,s1):1 -dist sender querying next range at /Table/2/1/53/"kv"/3/1 -dist sender r1: sending batch 1 Get to (n1,s1):1 -dist sender querying next range at /Table/3/1/54/2/1 -dist sender r1: sending batch 1 Get to (n1,s1):1 -dist sender querying next range at /Table/3/1/53/2/1 -dist sender r1: sending batch 1 Get to (n1,s1):1 -dist sender querying next range at /Table/3/1/53/2/1 -dist sender r1: sending batch 1 Get to (n1,s1):1 -sql txn Put /Table/3/1/54/2/1 -> table: columns: nullable:false hidden:false > columns: nullable:true hidden:false > next_column_id:3 families: next_family_id:1 primary_index: interleave:<> partitioning: type:FORWARD > next_index_id:3 privileges: users: > mutations: interleave:<> partitioning: type:FORWARD > state:DELETE_AND_WRITE_ONLY direction:DROP mutation_id:2 rollback:false > next_mutation_id:3 format_version:3 state:PUBLIC view_query:"" mutationJobs:<...> drop_time:0 replacement_of: > audit_mode:DISABLED > -dist sender querying next range at /Table/3/1/54/2/1 -dist sender r1: sending batch 1 Put to (n1,s1):1 -sql txn rows affected: 0 -dist sender querying next range at /Table/SystemConfigSpan/Start -dist sender r1: sending batch 1 EndTxn, 9 QueryIntent to (n1,s1):1 +dist sender send querying next range at /Table/2/1/0/"t"/3/1 +dist sender send r1: sending batch 1 Get to (n1,s1):1 +dist sender send querying next range at /Table/3/1/53/2/1 +dist sender send r1: sending batch 1 Get to (n1,s1):1 +dist sender send querying next range at /Table/2/1/53/"kv"/3/1 +dist sender send r1: sending batch 1 Get to (n1,s1):1 +dist sender send querying next range at /Table/3/1/54/2/1 +dist sender send r1: sending batch 1 Get to (n1,s1):1 +dist sender send querying next range at /Table/3/1/53/2/1 +dist sender send r1: sending batch 1 Get to (n1,s1):1 +dist sender send querying next range at /Table/2/1/0/"t"/3/1 +dist sender send r1: sending batch 1 Get to (n1,s1):1 +dist sender send querying next range at /Table/3/1/53/2/1 +dist sender send r1: sending batch 1 Get to (n1,s1):1 +dist sender send querying next range at /Table/2/1/53/"kv"/3/1 +dist sender send r1: sending batch 1 Get to (n1,s1):1 +dist sender send querying next range at /Table/3/1/54/2/1 +dist sender send r1: sending batch 1 Get to (n1,s1):1 +dist sender send querying next range at /Table/3/1/53/2/1 +dist sender send r1: sending batch 1 Get to (n1,s1):1 +dist sender send querying next range at /Table/3/1/53/2/1 +dist sender send r1: sending batch 1 Get to (n1,s1):1 +sql txn Put /Table/3/1/54/2/1 -> table: columns: nullable:false hidden:false > columns: nullable:true hidden:false > next_column_id:3 families: next_family_id:1 primary_index: interleave:<> partitioning: type:FORWARD > next_index_id:3 privileges: users: > mutations: interleave:<> partitioning: type:FORWARD > state:DELETE_AND_WRITE_ONLY direction:DROP mutation_id:2 rollback:false > next_mutation_id:3 format_version:3 state:PUBLIC view_query:"" mutationJobs:<...> drop_time:0 replacement_of: > audit_mode:DISABLED > +dist sender send querying next range at /Table/3/1/54/2/1 +dist sender send r1: sending batch 1 Put to (n1,s1):1 +sql txn rows affected: 0 +dist sender send querying next range at /Table/SystemConfigSpan/Start +dist sender send r1: sending batch 1 EndTxn, 9 QueryIntent to (n1,s1):1 statement ok SET tracing = on,kv,results; DROP TABLE t.kv; SET tracing = off @@ -416,32 +418,32 @@ WHERE message NOT LIKE '%Z/%' AND tag NOT LIKE '%scExec=%' AND tag NOT LIKE '%IndexBackfiller%' ---- -dist sender querying next range at /Table/2/1/0/"t"/3/1 -dist sender r1: sending batch 1 Get to (n1,s1):1 -dist sender querying next range at /Table/3/1/53/2/1 -dist sender r1: sending batch 1 Get to (n1,s1):1 -dist sender querying next range at /Table/2/1/53/"kv"/3/1 -dist sender r1: sending batch 1 Get to (n1,s1):1 -dist sender querying next range at /Table/3/1/54/2/1 -dist sender r1: sending batch 1 Get to (n1,s1):1 -dist sender querying next range at /Table/3/1/53/2/1 -dist sender r1: sending batch 1 Get to (n1,s1):1 -dist sender querying next range at /Table/5/1/54/2/1 -dist sender r1: sending batch 1 Get to (n1,s1):1 -dist sender querying next range at /Table/3/1/54/2/1 -dist sender r1: sending batch 1 Get to (n1,s1):1 -dist sender querying next range at /Table/5/1/53/2/1 -dist sender r1: sending batch 1 Get to (n1,s1):1 -dist sender querying next range at /Table/3/1/53/2/1 -dist sender r1: sending batch 1 Get to (n1,s1):1 -dist sender querying next range at /Table/5/1/0/2/1 -dist sender r1: sending batch 1 Get to (n1,s1):1 -sql txn Put /Table/3/1/54/2/1 -> table: columns: nullable:false hidden:false > columns: nullable:true hidden:false > next_column_id:3 families: next_family_id:1 primary_index: interleave:<> partitioning: type:FORWARD > next_index_id:3 privileges: users: > next_mutation_id:3 format_version:3 state:DROP draining_names: view_query:"" drop_time:... replacement_of: > audit_mode:DISABLED > -dist sender querying next range at /Table/SystemConfigSpan/Start -dist sender r1: sending batch 1 Put, 1 BeginTxn to (n1,s1):1 -sql txn rows affected: 0 -dist sender querying next range at /Table/SystemConfigSpan/Start -dist sender r1: sending batch 1 EndTxn, 6 QueryIntent to (n1,s1):1 +dist sender send querying next range at /Table/2/1/0/"t"/3/1 +dist sender send r1: sending batch 1 Get to (n1,s1):1 +dist sender send querying next range at /Table/3/1/53/2/1 +dist sender send r1: sending batch 1 Get to (n1,s1):1 +dist sender send querying next range at /Table/2/1/53/"kv"/3/1 +dist sender send r1: sending batch 1 Get to (n1,s1):1 +dist sender send querying next range at /Table/3/1/54/2/1 +dist sender send r1: sending batch 1 Get to (n1,s1):1 +dist sender send querying next range at /Table/3/1/53/2/1 +dist sender send r1: sending batch 1 Get to (n1,s1):1 +dist sender send querying next range at /Table/5/1/54/2/1 +dist sender send r1: sending batch 1 Get to (n1,s1):1 +dist sender send querying next range at /Table/3/1/54/2/1 +dist sender send r1: sending batch 1 Get to (n1,s1):1 +dist sender send querying next range at /Table/5/1/53/2/1 +dist sender send r1: sending batch 1 Get to (n1,s1):1 +dist sender send querying next range at /Table/3/1/53/2/1 +dist sender send r1: sending batch 1 Get to (n1,s1):1 +dist sender send querying next range at /Table/5/1/0/2/1 +dist sender send r1: sending batch 1 Get to (n1,s1):1 +sql txn Put /Table/3/1/54/2/1 -> table: columns: nullable:false hidden:false > columns: nullable:true hidden:false > next_column_id:3 families: next_family_id:1 primary_index: interleave:<> partitioning: type:FORWARD > next_index_id:3 privileges: users: > next_mutation_id:3 format_version:3 state:DROP draining_names: view_query:"" drop_time:... replacement_of: > audit_mode:DISABLED > +dist sender send querying next range at /Table/SystemConfigSpan/Start +dist sender send r1: sending batch 1 Put, 1 BeginTxn to (n1,s1):1 +sql txn rows affected: 0 +dist sender send querying next range at /Table/SystemConfigSpan/Start +dist sender send r1: sending batch 1 EndTxn, 6 QueryIntent to (n1,s1):1 # Check that session tracing does not inhibit the fast path for inserts & # friends (the path resulting in 1PC transactions). diff --git a/pkg/sql/sem/builtins/builtins.go b/pkg/sql/sem/builtins/builtins.go index 8f2714f634a4..8d2a5fea6472 100644 --- a/pkg/sql/sem/builtins/builtins.go +++ b/pkg/sql/sem/builtins/builtins.go @@ -2845,7 +2845,8 @@ may increase either contention or retry errors, or both.`, Nanos: int64(ctx.StmtTimestamp.Sub(ctx.TxnTimestamp)), } if elapsed.Compare(minDuration) < 0 { - return nil, ctx.Txn.GenerateForcedRetryableError("forced by crdb_internal.force_retry()") + return nil, ctx.Txn.GenerateForcedRetryableError( + ctx.Ctx(), "forced by crdb_internal.force_retry()") } return tree.DZero, nil }, diff --git a/pkg/sql/sem/tree/eval.go b/pkg/sql/sem/tree/eval.go index 331dedeeced3..dcceb919c799 100644 --- a/pkg/sql/sem/tree/eval.go +++ b/pkg/sql/sem/tree/eval.go @@ -2556,7 +2556,6 @@ func MakeTestingEvalContextWithMon(st *cluster.Settings, monitor *mon.BytesMonit acc := monitor.MakeBoundAccount() ctx.ActiveMemAcc = &acc now := timeutil.Now() - ctx.Txn.Proto().OrigTimestamp = hlc.Timestamp{WallTime: now.Unix()} ctx.SetTxnTimestamp(now) ctx.SetStmtTimestamp(now) return ctx diff --git a/pkg/sql/sem/tree/timeconv_test.go b/pkg/sql/sem/tree/timeconv_test.go index 6f2884f9cf06..ce94deceb5c1 100644 --- a/pkg/sql/sem/tree/timeconv_test.go +++ b/pkg/sql/sem/tree/timeconv_test.go @@ -15,6 +15,7 @@ package tree_test import ( + "context" "testing" "time" @@ -43,13 +44,14 @@ func TestClusterTimestampConversion(t *testing.T) { } clock := hlc.NewClock(hlc.UnixNano, time.Nanosecond) - - factory := client.TxnSenderFactoryFunc(func(client.TxnType) client.TxnSender { - return nil - }) + senderFactory := client.MakeMockTxnSenderFactory( + func(context.Context, *roachpb.Transaction, roachpb.BatchRequest, + ) (*roachpb.BatchResponse, *roachpb.Error) { + panic("unused") + }) db := client.NewDB( testutils.MakeAmbientCtx(), - factory, + senderFactory, clock) for _, d := range testData { diff --git a/pkg/sql/sqlbase/fk.go b/pkg/sql/sqlbase/fk.go index e6ca4d4c90db..0826a4e862f0 100644 --- a/pkg/sql/sqlbase/fk.go +++ b/pkg/sql/sqlbase/fk.go @@ -364,7 +364,7 @@ func (f *fkBatchChecker) runCheck( } return pgerror.NewErrorf(pgerror.CodeForeignKeyViolationError, "foreign key violation: value %s not found in %s@%s %s (txn=%s)", - fkValues, fk.searchTable.Name, fk.searchIdx.Name, fk.searchIdx.ColumnNames[:fk.prefixLen], f.txn.Proto()) + fkValues, fk.searchTable.Name, fk.searchIdx.Name, fk.searchIdx.ColumnNames[:fk.prefixLen], f.txn.ID()) } case CheckDeletes: // If we're deleting, then there's a violation if the scan found something. diff --git a/pkg/sql/trace_test.go b/pkg/sql/trace_test.go index 72083a9fc52e..79b909c5963b 100644 --- a/pkg/sql/trace_test.go +++ b/pkg/sql/trace_test.go @@ -73,7 +73,8 @@ func TestTrace(t *testing.T) { "session recording", "sql txn", "consuming rows", - "dist sender", + "txn coordinator send", + "dist sender send", "/cockroach.roachpb.Internal/Batch", }, }, @@ -124,7 +125,8 @@ func TestTrace(t *testing.T) { "flow", "table reader", "consuming rows", - "dist sender", + "txn coordinator send", + "dist sender send", "/cockroach.roachpb.Internal/Batch", }, // Depending on whether the data is local or not, we may not see these @@ -157,7 +159,8 @@ func TestTrace(t *testing.T) { "session recording", "sql txn", "consuming rows", - "dist sender", + "txn coordinator send", + "dist sender send", "/cockroach.roachpb.Internal/Batch", }, }, @@ -186,7 +189,8 @@ func TestTrace(t *testing.T) { "flow", "table reader", "consuming rows", - "dist sender", + "txn coordinator send", + "dist sender send", "/cockroach.roachpb.Internal/Batch", }, // Depending on whether the data is local or not, we may not see these @@ -215,7 +219,7 @@ func TestTrace(t *testing.T) { "WHERE message LIKE '%1 DelRng%' ORDER BY op") }, expSpans: []string{ - "dist sender", + "dist sender send", "kv.DistSender: sending partial batch", "/cockroach.roachpb.Internal/Batch", }, diff --git a/pkg/sql/txn_state.go b/pkg/sql/txn_state.go index dc1451eb5e53..0a33802aac21 100644 --- a/pkg/sql/txn_state.go +++ b/pkg/sql/txn_state.go @@ -16,7 +16,6 @@ package sql import ( "context" - "fmt" "time" opentracing "github.com/opentracing/opentracing-go" @@ -107,10 +106,6 @@ type txnState struct { // txnAbortCount is incremented whenever the state transitions to // stateAborted. txnAbortCount *metric.Counter - - // inExternalTxn, if set, means that mu.txn is not owned by the txnState. This - // happens for the InternalExecutor. - inExternalTxn bool } // txnType represents the type of a SQL transaction. @@ -202,7 +197,6 @@ func (ts *txnState) resetForNewSQLTxn( ts.mu.txn.SetDebugName(opName) } else { ts.mu.txn = txn - ts.inExternalTxn = true } ts.mu.Unlock() @@ -236,11 +230,6 @@ func (ts *txnState) finishSQLTxn() { panic("No span in context? Was resetForNewSQLTxn() called previously?") } - if !ts.mu.txn.IsFinalized() && !ts.inExternalTxn { - panic(fmt.Sprintf( - "attempting to finishSQLTxn(), but KV txn is not finalized: %+v", ts.mu.txn)) - } - if ts.recordingThreshold > 0 { if r := tracing.GetRecording(ts.sp); r != nil { if elapsed := timeutil.Since(ts.recordingStart); elapsed >= ts.recordingThreshold { diff --git a/pkg/sql/txn_state_test.go b/pkg/sql/txn_state_test.go index f2bcce3dd890..ec36895d95f1 100644 --- a/pkg/sql/txn_state_test.go +++ b/pkg/sql/txn_state_test.go @@ -58,13 +58,11 @@ type testContext struct { 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 nil, nil - }, - ) - }) + factory := client.MakeMockTxnSenderFactory( + func(context.Context, *roachpb.Transaction, roachpb.BatchRequest, + ) (*roachpb.BatchResponse, *roachpb.Error) { + return nil, nil + }) settings := cluster.MakeTestingClusterSettings() ambient := testutils.MakeAmbientCtx() @@ -211,7 +209,6 @@ type expKVTxn struct { tsNanos *int64 origTSNanos *int64 maxTSNanos *int64 - isFinalized *bool } func checkTxn(txn *client.Txn, exp expKVTxn) error { @@ -230,21 +227,19 @@ func checkTxn(txn *client.Txn, exp expKVTxn) error { return errors.Errorf("expected UserPriority: %s, but got: %s", *exp.userPriority, txn.UserPriority()) } - if exp.tsNanos != nil && *exp.tsNanos != txn.Proto().Timestamp.WallTime { + proto := txn.Serialize() + if exp.tsNanos != nil && *exp.tsNanos != proto.Timestamp.WallTime { return errors.Errorf("expected Timestamp: %d, but got: %s", - *exp.tsNanos, txn.Proto().Timestamp) + *exp.tsNanos, proto.Timestamp) } if origTimestamp := txn.OrigTimestamp(); exp.origTSNanos != nil && *exp.origTSNanos != origTimestamp.WallTime { return errors.Errorf("expected OrigTimestamp: %d, but got: %s", *exp.origTSNanos, origTimestamp) } - if exp.maxTSNanos != nil && *exp.maxTSNanos != txn.Proto().MaxTimestamp.WallTime { + if exp.maxTSNanos != nil && *exp.maxTSNanos != proto.MaxTimestamp.WallTime { return errors.Errorf("expected MaxTimestamp: %d, but got: %s", - *exp.maxTSNanos, txn.Proto().MaxTimestamp) - } - if exp.isFinalized != nil && *exp.isFinalized != txn.IsFinalized() { - return errors.Errorf("expected finalized: %t but wasn't", *exp.isFinalized) + *exp.maxTSNanos, proto.MaxTimestamp) } return nil } @@ -275,8 +270,6 @@ func TestTransitions(t *testing.T) { iso := enginepb.SERIALIZABLE pri := roachpb.NormalUserPriority maxTS := testCon.clock.Now().Add(testCon.clock.MaxOffset().Nanoseconds(), 0 /* logical */) - varTrue := true - varFalse := false type test struct { name string @@ -330,7 +323,6 @@ func TestTransitions(t *testing.T) { tsNanos: &now.WallTime, origTSNanos: &now.WallTime, maxTSNanos: &maxTS.WallTime, - isFinalized: &varFalse, }, }, { @@ -354,7 +346,6 @@ func TestTransitions(t *testing.T) { tsNanos: &now.WallTime, origTSNanos: &now.WallTime, maxTSNanos: &maxTS.WallTime, - isFinalized: &varFalse, }, }, // @@ -428,7 +419,7 @@ func TestTransitions(t *testing.T) { err: roachpb.NewHandledRetryableTxnError( "test retriable err", ts.mu.txn.ID(), - *ts.mu.txn.Proto()), + *ts.mu.txn.Serialize()), rewCap: dummyRewCap, } return eventRetriableErr{CanAutoRetry: True, IsCommit: False}, b @@ -439,9 +430,7 @@ func TestTransitions(t *testing.T) { expEv: txnRestart, }, // Expect non-nil txn. - expTxn: &expKVTxn{ - isFinalized: &varFalse, - }, + expTxn: &expKVTxn{}, }, { // Like the above test - get a retriable error while we can auto-retry, @@ -457,7 +446,7 @@ func TestTransitions(t *testing.T) { err: roachpb.NewHandledRetryableTxnError( "test retriable err", ts.mu.txn.ID(), - *ts.mu.txn.Proto()), + *ts.mu.txn.Serialize()), rewCap: dummyRewCap, } return eventRetriableErr{CanAutoRetry: True, IsCommit: True}, b @@ -468,9 +457,7 @@ func TestTransitions(t *testing.T) { expEv: txnRestart, }, // Expect non-nil txn. - expTxn: &expKVTxn{ - isFinalized: &varFalse, - }, + expTxn: &expKVTxn{}, }, { // Get a retriable error when we can no longer auto-retry, but the client @@ -485,7 +472,7 @@ func TestTransitions(t *testing.T) { err: roachpb.NewHandledRetryableTxnError( "test retriable err", ts.mu.txn.ID(), - *ts.mu.txn.Proto()), + *ts.mu.txn.Serialize()), rewCap: rewindCapability{}, } return eventRetriableErr{CanAutoRetry: False, IsCommit: False}, b @@ -496,9 +483,7 @@ func TestTransitions(t *testing.T) { expEv: txnRestart, }, // Expect non-nil txn. - expTxn: &expKVTxn{ - isFinalized: &varFalse, - }, + expTxn: &expKVTxn{}, }, { // Like the above (a retriable error when we can no longer auto-retry, but @@ -517,7 +502,7 @@ func TestTransitions(t *testing.T) { err: roachpb.NewHandledRetryableTxnError( "test retriable err", ts.mu.txn.ID(), - *ts.mu.txn.Proto()), + *ts.mu.txn.Serialize()), rewCap: rewindCapability{}, } return eventRetriableErr{CanAutoRetry: False, IsCommit: True}, b @@ -560,7 +545,7 @@ func TestTransitions(t *testing.T) { err: roachpb.NewHandledRetryableTxnError( "test retriable err", ts.mu.txn.ID(), - *ts.mu.txn.Proto()), + *ts.mu.txn.Serialize()), rewCap: rewindCapability{}, } return eventRetriableErr{CanAutoRetry: False, IsCommit: False}, b @@ -570,9 +555,7 @@ func TestTransitions(t *testing.T) { expCode: skipBatch, expEv: txnAborted, }, - expTxn: &expKVTxn{ - isFinalized: &varTrue, - }, + expTxn: &expKVTxn{}, }, { // Like the above, but this time with an implicit txn: we get a retriable @@ -587,7 +570,7 @@ func TestTransitions(t *testing.T) { err: roachpb.NewHandledRetryableTxnError( "test retriable err", ts.mu.txn.ID(), - *ts.mu.txn.Proto()), + *ts.mu.txn.Serialize()), rewCap: rewindCapability{}, } return eventRetriableErr{CanAutoRetry: False, IsCommit: False}, b @@ -614,9 +597,7 @@ func TestTransitions(t *testing.T) { expCode: skipBatch, expEv: txnAborted, }, - expTxn: &expKVTxn{ - isFinalized: &varTrue, - }, + expTxn: &expKVTxn{}, }, { // We go to CommitWait (after a RELEASE SAVEPOINT). @@ -633,9 +614,7 @@ func TestTransitions(t *testing.T) { expCode: advanceOne, expEv: txnCommit, }, - expTxn: &expKVTxn{ - isFinalized: &varTrue, - }, + expTxn: &expKVTxn{}, }, { // Restarting from Open via ROLLBACK TO SAVEPOINT. @@ -687,9 +666,7 @@ func TestTransitions(t *testing.T) { expCode: advanceOne, expEv: noEvent, }, - expTxn: &expKVTxn{ - isFinalized: &varFalse, - }, + expTxn: &expKVTxn{}, }, // // Tests starting from the RestartWait state. @@ -769,9 +746,7 @@ func TestTransitions(t *testing.T) { expAdv: expAdvance{ expCode: skipBatch, }, - expTxn: &expKVTxn{ - isFinalized: &varTrue, - }, + expTxn: &expKVTxn{}, }, } diff --git a/pkg/storage/client_split_test.go b/pkg/storage/client_split_test.go index 87d24605f5e4..32dcd4183dd0 100644 --- a/pkg/storage/client_split_test.go +++ b/pkg/storage/client_split_test.go @@ -1662,7 +1662,7 @@ func TestStoreSplitTimestampCacheDifferentLeaseHolder(t *testing.T) { } // Verify that the txn's safe timestamp was set. - if txnOld.Proto().RefreshedTimestamp == (hlc.Timestamp{}) { + if txnOld.Serialize().RefreshedTimestamp == (hlc.Timestamp{}) { t.Fatal("expected non-zero refreshed timestamp") } @@ -2442,7 +2442,8 @@ func TestDistributedTxnCleanup(t *testing.T) { if err := txn.Run(ctx, b); err != nil { return err } - txnKey = keys.TransactionKey(txn.Proto().Key, txn.Proto().ID) + proto := txn.Serialize() + txnKey = keys.TransactionKey(proto.Key, proto.ID) // If force=true, we're force-aborting the txn out from underneath. // This simulates txn deadlock or a max priority txn aborting a // normal or min priority txn. @@ -2451,10 +2452,10 @@ func TestDistributedTxnCleanup(t *testing.T) { ba.RangeID = lhs.RangeID ba.Add(&roachpb.PushTxnRequest{ RequestHeader: roachpb.RequestHeader{ - Key: txn.Proto().Key, + Key: proto.Key, }, Now: store.Clock().Now(), - PusheeTxn: txn.Proto().TxnMeta, + PusheeTxn: proto.TxnMeta, PushType: roachpb.PUSH_ABORT, Force: true, }) diff --git a/pkg/storage/log.go b/pkg/storage/log.go index ce93d54b8e30..99313d57c693 100644 --- a/pkg/storage/log.go +++ b/pkg/storage/log.go @@ -56,13 +56,13 @@ func (s *Store) insertRangeLogEvent( } const insertEventTableStmt = ` -INSERT INTO system.rangelog ( - timestamp, "rangeID", "storeID", "eventType", "otherRangeID", info -) -VALUES( - $1, $2, $3, $4, $5, $6 -) -` + INSERT INTO system.rangelog ( + timestamp, "rangeID", "storeID", "eventType", "otherRangeID", info + ) + VALUES( + $1, $2, $3, $4, $5, $6 + ) + ` args := []interface{}{ event.Timestamp, event.RangeID, @@ -116,7 +116,7 @@ func (s *Store) logSplit( return nil } return s.insertRangeLogEvent(ctx, txn, RangeLogEvent{ - Timestamp: selectEventTimestamp(s, txn.Proto().Timestamp), + Timestamp: selectEventTimestamp(s, txn.OrigTimestamp()), RangeID: updatedDesc.RangeID, EventType: RangeLogEventType_split, StoreID: s.StoreID(), @@ -169,7 +169,7 @@ func (s *Store) logChange( } return s.insertRangeLogEvent(ctx, txn, RangeLogEvent{ - Timestamp: selectEventTimestamp(s, txn.Proto().Timestamp), + Timestamp: selectEventTimestamp(s, txn.OrigTimestamp()), RangeID: desc.RangeID, EventType: logType, StoreID: s.StoreID(), diff --git a/pkg/storage/log_test.go b/pkg/storage/log_test.go index 109fc16a9273..0d119dcc458e 100644 --- a/pkg/storage/log_test.go +++ b/pkg/storage/log_test.go @@ -63,7 +63,7 @@ func TestLogSplits(t *testing.T) { } // Generate an explicit split event. - if err := kvDB.AdminSplit(context.TODO(), "splitkey", "splitkey"); err != nil { + if err := kvDB.AdminSplit(ctx, "splitkey", "splitkey"); err != nil { t.Fatal(err) } diff --git a/pkg/storage/store.go b/pkg/storage/store.go index 7116a00cd252..541500dffe05 100644 --- a/pkg/storage/store.go +++ b/pkg/storage/store.go @@ -763,6 +763,8 @@ type StoreTestingKnobs struct { DisableTimeSeriesMaintenanceQueue bool // DisableRaftSnapshotQueue disables the raft snapshot queue. DisableRaftSnapshotQueue bool + // DisableConsistencyQueue disables the consistency checker. + DisableConsistencyQueue bool // DisableScanner disables the replica scanner. DisableScanner bool // DisablePeriodicGossips disables periodic gossiping. @@ -1004,6 +1006,9 @@ func NewStore(cfg StoreConfig, eng engine.Engine, nodeDesc *roachpb.NodeDescript if cfg.TestingKnobs.DisableRaftSnapshotQueue { s.setRaftSnapshotQueueActive(false) } + if cfg.TestingKnobs.DisableConsistencyQueue { + s.setConsistencyQueueActive(false) + } if cfg.TestingKnobs.DisableScanner { s.setScannerActive(false) } @@ -4396,6 +4401,9 @@ func (s *Store) setTimeSeriesMaintenanceQueueActive(active bool) { func (s *Store) setRaftSnapshotQueueActive(active bool) { s.raftSnapshotQueue.SetDisabled(!active) } +func (s *Store) setConsistencyQueueActive(active bool) { + s.consistencyQueue.SetDisabled(!active) +} func (s *Store) setScannerActive(active bool) { s.scanner.SetDisabled(!active) } diff --git a/pkg/storage/store_test.go b/pkg/storage/store_test.go index d1698ed16d04..ac3c6792ce9f 100644 --- a/pkg/storage/store_test.go +++ b/pkg/storage/store_test.go @@ -81,9 +81,15 @@ type testSenderFactory struct { } func (f *testSenderFactory) TransactionalSender( - typ client.TxnType, _ roachpb.TxnCoordMeta, + typ client.TxnType, coordMeta roachpb.TxnCoordMeta, ) client.TxnSender { - return &testSender{store: f.store} + return client.NewMockTransactionalSender( + func( + ctx context.Context, _ *roachpb.Transaction, ba roachpb.BatchRequest, + ) (*roachpb.BatchResponse, *roachpb.Error) { + return f.store.Send(ctx, ba) + }, + &coordMeta.Txn) } func (f *testSenderFactory) NonTransactionalSender() client.Sender { @@ -108,14 +114,6 @@ type testSender struct { store *Store } -func (db *testSender) GetMeta() 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) DisablePipelining() { 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.