Skip to content

kv,storage: persist gateway node id in transaction intents#73500

Merged
craig[bot] merged 2 commits intocockroachdb:masterfrom
AlexTalks:contention_event_node_id
Dec 17, 2021
Merged

kv,storage: persist gateway node id in transaction intents#73500
craig[bot] merged 2 commits intocockroachdb:masterfrom
AlexTalks:contention_event_node_id

Conversation

@AlexTalks
Copy link
Copy Markdown
Contributor

@AlexTalks AlexTalks commented Dec 6, 2021

This change augments the TxnMeta protobuf structure to include the
gateway node ID (responsible for initiating the transaction) when
serializing the intent. By doing so, this commit enables the Contention
Event Store proposed in #71965, utilizing option 2.

Release note: None

@cockroach-teamcity
Copy link
Copy Markdown
Member

This change is Reviewable

@AlexTalks AlexTalks added the do-not-merge bors won't merge a PR with this label. label Dec 6, 2021
@AlexTalks AlexTalks force-pushed the contention_event_node_id branch 3 times, most recently from 02d50fc to 4766820 Compare December 9, 2021 23:02
@AlexTalks AlexTalks marked this pull request as ready for review December 9, 2021 23:03
@AlexTalks AlexTalks requested review from a team as code owners December 9, 2021 23:03
@AlexTalks AlexTalks force-pushed the contention_event_node_id branch from 4766820 to e61aad6 Compare December 10, 2021 00:33
@AlexTalks AlexTalks removed the do-not-merge bors won't merge a PR with this label. label Dec 10, 2021
Copy link
Copy Markdown
Contributor

@Azhng Azhng left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks for working on this and this is awesome! I just have some minor style nits and rest looks all good from SQL Obs 👍

Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (waiting on @AlexTalks, @lidorcarmel, and @nvanbenschoten)


pkg/kv/txn.go, line 128 at r1 (raw file):

	now := db.clock.NowAsClockTimestamp()
	kvTxn := roachpb.MakeTransaction("unnamed", nil, roachpb.NormalUserPriority, now.ToTimestamp(),
		db.clock.MaxOffset().Nanoseconds(), int32(db.ctx.NodeID.SQLInstanceID()))

nit: let's keep the original formatting here?

	kvTxn := roachpb.MakeTransaction(
		"unnamed",
		nil, // baseKey
		roachpb.NormalUserPriority,
		now.ToTimestamp(),
		db.clock.MaxOffset().Nanoseconds(),
                int32(db.ctx.NodeID.SQLInstanceID()),
	)

pkg/kv/kvclient/kvcoord/dist_sender_test.go, line 2171 at r1 (raw file):

	ds := NewDistSender(cfg)

	txn := roachpb.MakeTransaction("foo", nil, 1.0, clock.Now(), 0, 1)

nit: let's add the inline /* sqlNodeID */ comment here


pkg/kv/kvserver/replica_test.go, line 4552 at r1 (raw file):

	keyB := roachpb.Key("b")
	// Start a transaction early to get a low timestamp.
	txn := roachpb.MakeTransaction("test", keyA, roachpb.NormalUserPriority, tc.Clock().Now(), 0 /* maxOffsetNs */, 0 /* sqlNodeID */)

small nit: wrap the long line here ?

Copy link
Copy Markdown
Contributor

@nvb nvb left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Nice! I haven't made my way through all of the files that changed yet, but left a few comments on the important ones.

Reviewed 13 of 51 files at r1, all commit messages.
Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (waiting on @AlexTalks, @Azhng, and @lidorcarmel)


pkg/kv/txn.go, line 120 at r1 (raw file):

//
// See also db.NewTxn().
func NewTxn(ctx context.Context, db *DB, gatewayNodeID roachpb.NodeID) *Txn {

We had discussed not needing this gatewayNodeID param anymore in any of these constructors if/when we make it possible to determine whether SQLInstanceID is a reference to a KV node or to a SQL tenant node. Do you mind opening an issue to track that refactor?


pkg/kv/txn.go, line 128 at r1 (raw file):

Previously, Azhng (Archer Zhang) wrote…

nit: let's keep the original formatting here?

	kvTxn := roachpb.MakeTransaction(
		"unnamed",
		nil, // baseKey
		roachpb.NormalUserPriority,
		now.ToTimestamp(),
		db.clock.MaxOffset().Nanoseconds(),
                int32(db.ctx.NodeID.SQLInstanceID()),
	)

+1


pkg/roachpb/data.go, line 916 at r1 (raw file):

// write conflicts in a way that avoids starvation of long-running
// transactions (see Replica.PushTxn).
// TODO(sarkesian): Write comment about sqlNodeID

Do you mind addressing this TODO in this PR?


pkg/sql/logictest/logic.go, line 281 at r1 (raw file):

//    Introduces a sleep period. Example: sleep 2s
//
//  - user <username> <nodeidx=N>

nit: [nodeidx=N]


pkg/storage/enginepb/mvcc3.proto, line 126 at r1 (raw file):

  reserved 8;

  // The node ID of the SQL node where this transaction originated.

What do you think about calling this either CoordinatorNodeID or GatewayNodeID? I'm not sure that SQLNodeID implies the right thing if this is a transaction that is being started by a KV node.

This actually raises a few interesting questions. What exactly is this representing? And what should it represent?

Currently, the transaction coordinator (TxnCoordSender) is always the same as the SQL gateway that uses the transaction. But you could imagine an architecture where this is not the case. For instance, in a reimagined architecture, it could be possible for the SQL executor to use a remote TxnCoordSender. So if these were decoupled, which component would this be a reference to? I think the answer is the "gateway" and not the "coordinator", because that's what houses the transaction's SQL metadata.

However, you could also imagine an architecture where the KV layer might want to know about the coordinator node ID of a given transaction, because it might want to talk to it directly. For instance, we may want a transaction that aborts some other transaction to reach out to its coordinator and inform it directly, instead of waiting an arbitrary amount of time for the aborted transaction to find out that it's aborted. So this might be an opportune time to begin storing the coordinator ID in intents.

What do you think?


pkg/storage/enginepb/mvcc3.proto, line 130 at r1 (raw file):

  // representing either a SQLInstanceID of a SQL pod or a gateway NodeID,
  // and is utilized for SQL Observability.
  // TODO(sarkesian): Refactor/include gogoproto.casttype when #73309 completes.

Do you mind mentioning which type we plan to use? Is it GenericNodeID?

Copy link
Copy Markdown
Contributor Author

@AlexTalks AlexTalks left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (waiting on @AlexTalks, @lidorcarmel, and @nvanbenschoten)


pkg/kv/txn.go, line 120 at r1 (raw file):

Previously, nvanbenschoten (Nathan VanBenschoten) wrote…

We had discussed not needing this gatewayNodeID param anymore in any of these constructors if/when we make it possible to determine whether SQLInstanceID is a reference to a KV node or to a SQL tenant node. Do you mind opening an issue to track that refactor?

OK - created #73871.

Copy link
Copy Markdown
Contributor Author

@AlexTalks AlexTalks left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (waiting on @AlexTalks, @Azhng, @lidorcarmel, and @nvanbenschoten)


pkg/kv/kvserver/replica_test.go, line 4552 at r1 (raw file):

Previously, Azhng (Archer Zhang) wrote…

small nit: wrap the long line here ?

Done.


pkg/roachpb/data.go, line 916 at r1 (raw file):

Previously, nvanbenschoten (Nathan VanBenschoten) wrote…

Do you mind addressing this TODO in this PR?

Ha! Thanks for catching, I forgot to come back to this.


pkg/storage/enginepb/mvcc3.proto, line 126 at r1 (raw file):

Previously, nvanbenschoten (Nathan VanBenschoten) wrote…

What do you think about calling this either CoordinatorNodeID or GatewayNodeID? I'm not sure that SQLNodeID implies the right thing if this is a transaction that is being started by a KV node.

This actually raises a few interesting questions. What exactly is this representing? And what should it represent?

Currently, the transaction coordinator (TxnCoordSender) is always the same as the SQL gateway that uses the transaction. But you could imagine an architecture where this is not the case. For instance, in a reimagined architecture, it could be possible for the SQL executor to use a remote TxnCoordSender. So if these were decoupled, which component would this be a reference to? I think the answer is the "gateway" and not the "coordinator", because that's what houses the transaction's SQL metadata.

However, you could also imagine an architecture where the KV layer might want to know about the coordinator node ID of a given transaction, because it might want to talk to it directly. For instance, we may want a transaction that aborts some other transaction to reach out to its coordinator and inform it directly, instead of waiting an arbitrary amount of time for the aborted transaction to find out that it's aborted. So this might be an opportune time to begin storing the coordinator ID in intents.

What do you think?

After speaking directly we agreed to use CoordinatorNodeID as the field name... it should be noted that if we do have a scenario in which a SQL executor uses a remote TxnCoordSender, it would be necessary (for the purposes of SQL Observability) to have this field represent the node of the SQL executor, and if we wanted to keep track of the KV coordinator node, we'd need a separate field. It sounds like this scenario has been discussed and decided against previously, however, so using the more generic name CoordinatorNodeID should be fine.


pkg/storage/enginepb/mvcc3.proto, line 130 at r1 (raw file):

Previously, nvanbenschoten (Nathan VanBenschoten) wrote…

Do you mind mentioning which type we plan to use? Is it GenericNodeID?

Also followed up on this directly - we agreed this would be a GenericNodeID.

@AlexTalks AlexTalks force-pushed the contention_event_node_id branch from 5abe810 to 7a741d8 Compare December 15, 2021 22:43
Copy link
Copy Markdown
Contributor

@nvb nvb left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

:lgtm_strong:

Reviewed 27 of 51 files at r1, 1 of 3 files at r2, 18 of 20 files at r4, 2 of 2 files at r5, all commit messages.
Reviewable status: :shipit: complete! 1 of 0 LGTMs obtained (waiting on @AlexTalks, @Azhng, and @lidorcarmel)


pkg/sql/logictest/testdata/logic_test/contention_event, line 80 at r5 (raw file):

true

# check that the sql instance id / gateway node id is properly stored with the contention event

nit: this comment needs a small update to use the new name.

@AlexTalks AlexTalks force-pushed the contention_event_node_id branch from 7a741d8 to c3d1075 Compare December 16, 2021 22:01
@AlexTalks
Copy link
Copy Markdown
Contributor Author

bors r+

@craig
Copy link
Copy Markdown
Contributor

craig bot commented Dec 16, 2021

Build failed (retrying...):

@ajwerner
Copy link
Copy Markdown
Contributor

Seems like maybe this and some other change skew: https://teamcity.cockroachdb.com/viewLog.html?buildId=3932844&buildTypeId=Cockroach_UnitTests
roachvet: pkg/kv/kvserver/client_replica_test.go:547:64: not enough arguments in call to roachpb.MakeTransaction

bors r-

sorry @AlexTalks

@craig
Copy link
Copy Markdown
Contributor

craig bot commented Dec 16, 2021

Canceled.

@AlexTalks
Copy link
Copy Markdown
Contributor Author

AlexTalks commented Dec 16, 2021

Ack! Thanks for catching, will fix & repush.

This change augments the `TxnMeta` protobuf structure to include the
gateway node ID (responsible for initiating the transaction) when
serializing the intent.  By doing so, this commit enables the Contention
Event Store proposed in cockroachdb#71965, utilizing option 2.

Release note: None
While we have support for switching users, which each hold a distinct db
client handle, there was previously no way to set a specific node for
the SQL client to connect to other than the default used for the entire
test.  With this change, we are now able to override the node used for
the SQL client connection for a particular user with the following
statement in a test:
```
user testuser nodeidx=3
```
With this directive, subsequent statements will be made via a SQL
connection to `NodeID` 4 (as `NodeID`s are 1-indexed), using user
`testuser`.  When a subsequent switch to the user is made that does not
specify a nodeidx override, such as `user root`, subsequent statements
will revert to using the default node for the test (typically `NodeID`
1).

This enables support for testing Contention Events, where the node
initiating the contending transaction needs to be validated, among other
use cases.

Note that the connections are per-user, so opening multiple connections
to different nodes for a single user is not currently supported.

Release note: None
@AlexTalks AlexTalks force-pushed the contention_event_node_id branch from c3d1075 to 1420276 Compare December 17, 2021 00:37
@AlexTalks
Copy link
Copy Markdown
Contributor Author

bors r+

@craig craig bot merged commit 089affc into cockroachdb:master Dec 17, 2021
@craig
Copy link
Copy Markdown
Contributor

craig bot commented Dec 17, 2021

Build succeeded:

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

None yet

Projects

None yet

Development

Successfully merging this pull request may close these issues.

sql, kv: KV to provide an interface for SQL to provide identifiers to be stored in Txn Intent

5 participants