-
Notifications
You must be signed in to change notification settings - Fork 3.8k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
kv: add WaitPolicy option to BatchRequest #52388
kv: add WaitPolicy option to BatchRequest #52388
Conversation
d18fdc8
to
afb9d10
Compare
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Pls tell me again why it'd be harder to set the policy at the request level instead of the batch level? I think allowing it to also be set on the batch would be ideal in which case it'd override the individual requests (or it could even error if an individual request also sets it).
How does the Error
policy behave within high-pri txns? I think we want to make sure that it's pushes only succeed on abandoned txns, not also on lower-pri live txns, but that's not what the code does (is it?). Or better yet, how about splitting Error
into ErrorOnEqualOrHigherPri
and ErrorOnAny
?
Have you put any though into a policy for select... skip locked
?
Pls make sure that the no-wait policy is reflected in ba.String()
.
Reviewable status: complete! 0 of 0 LGTMs obtained (waiting on @andreimatei and @nvanbenschoten)
pkg/kv/kvserver/concurrency/lock_table_waiter.go, line 173 at r1 (raw file):
err = w.pushLockTxn(ctx, req, state) } else { err = newWriteIntentErr(state)
what about non-replicated locks? Do we want WriteIntentError
for those?
pkg/kv/kvserver/concurrency/lock_table_waiter.go, line 442 at r1 (raw file):
switch req.WaitPolicy { case lock.WaitPolicy_Block: // The wait policy signifies that the request wants to wait until the
nit: s/The wait policy/This wait policy
same below
pkg/kv/kvserver/concurrency/lock_table_waiter.go, line 450 at r1 (raw file):
switch ws.guardAccess { case spanset.SpanReadOnly: pushType = roachpb.PUSH_TIMESTAMP
Not related to this patch, but please remind me - if a PUSH_TIMESTAMP
succeeds, will that unblock other readers at lower timestamps (besides the one that did the successful push)?
pkg/kv/kvserver/concurrency/lock_table_waiter.go, line 451 at r1 (raw file):
case spanset.SpanReadOnly: pushType = roachpb.PUSH_TIMESTAMP log.VEventf(ctx, 3, "pushing timestamp of txn %s above %s", ws.txn.ID.Short(), h.Timestamp)
3
seems high here and below. Let's use the more standard 2.
pkg/kv/kvserver/concurrency/lock/locking.proto, line 176 at r1 (raw file):
// Error indicates that if a request encounters a conflicting locks held by // another active transaction, it should raise an error instead of blocking.
spell out what error
Also spell out that this doesn't apply to abandoned locks.
afb9d10
to
a89adf9
Compare
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Pls tell me again why it'd be harder to set the policy at the request level instead of the batch level? I think allowing it to also be set on the batch would be ideal in which case it'd override the individual requests (or it could even error if an individual request also sets it).
Because we aggregate and deduplicate all of a batch request's spans before entering the concurrency manager. So the concurrency manager doesn't know which spans correspond to which (it could be multiple) individual requests. I don't disagree that this could be a nice thing to support, but since we don't need it, I don't think we should go out of our way to support it.
How does the Error policy behave within high-pri txns? I think we want to make sure that it's pushes only succeed on abandoned txns, not also on lower-pri live txns, but that's not what the code does (is it?). Or better yet, how about splitting Error into ErrorOnEqualOrHigherPri and ErrorOnAny?
That's a good question. Its pushes do only succeed on abandoned txns, see:
cockroach/pkg/kv/kvserver/batcheval/cmd_push_txn.go
Lines 230 to 236 in d013d12
case pushType == roachpb.PUSH_TOUCH: | |
// If just attempting to cleanup old or already-committed txns, | |
// pusher always fails. | |
pusherWins = false | |
case CanPushWithPriority(&args.PusherTxn, &reply.PusheeTxn): | |
reason = "pusher has priority" | |
pusherWins = true |
And also see TestPushTxnHeartbeatTimeout
, which tests the push behavior.
I added this case to TestTxnWaitPolicies
.
Have you put any though into a policy for select... skip locked ?
Not a lot. Only that I'm not as eager to implement it because we haven't seen any interest in it from users and also because it violates serializability, whereas nowait
does not.
Pls make sure that the no-wait policy is reflected in ba.String()
Good point. Done.
Reviewable status: complete! 0 of 0 LGTMs obtained (waiting on @andreimatei)
pkg/kv/kvserver/concurrency/lock_table_waiter.go, line 173 at r1 (raw file):
Previously, andreimatei (Andrei Matei) wrote…
what about non-replicated locks? Do we want
WriteIntentError
for those?
Yes, that's ok. WriteIntentError is a bit of a misnomer at this point now that we have different forms of locks, but it's exactly what we want.
pkg/kv/kvserver/concurrency/lock_table_waiter.go, line 442 at r1 (raw file):
Previously, andreimatei (Andrei Matei) wrote…
nit: s/The wait policy/This wait policy
same below
Done.
pkg/kv/kvserver/concurrency/lock_table_waiter.go, line 450 at r1 (raw file):
Previously, andreimatei (Andrei Matei) wrote…
Not related to this patch, but please remind me - if a
PUSH_TIMESTAMP
succeeds, will that unblock other readers at lower timestamps (besides the one that did the successful push)?
Let on in this function we use the IntentResolver to move the lock's timestamp forward. Doing so triggers a state transition in the lock-table which unblocks the other readers. This is discussed in this method's comment.
pkg/kv/kvserver/concurrency/lock_table_waiter.go, line 451 at r1 (raw file):
Previously, andreimatei (Andrei Matei) wrote…
3
seems high here and below. Let's use the more standard 2.
Done.
pkg/kv/kvserver/concurrency/lock/locking.proto, line 176 at r1 (raw file):
spell out what error
I actually omitted this deliberately, as the error type is not a concern of this package. This package is merely defining the locking concepts that the KV API uses. The comment on Header.wait_policy
spells out what error is returned by the KV API when this wait policy is used.
Also spell out that this doesn't apply to abandoned locks.
That's what the "active transaction" was meant to indicate, but I agree that it's better to be explicit. Done.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Reviewable status: complete! 1 of 0 LGTMs obtained (waiting on @nvanbenschoten)
pkg/kv/kvserver/concurrency/lock_table_waiter.go, line 173 at r1 (raw file):
Previously, nvanbenschoten (Nathan VanBenschoten) wrote…
Yes, that's ok. WriteIntentError is a bit of a misnomer at this point now that we have different forms of locks, but it's exactly what we want.
ack
pkg/kv/kvserver/concurrency/lock_table_waiter.go, line 450 at r1 (raw file):
Previously, nvanbenschoten (Nathan VanBenschoten) wrote…
Let on in this function we use the IntentResolver to move the lock's timestamp forward. Doing so triggers a state transition in the lock-table which unblocks the other readers. This is discussed in this method's comment.
ack
pkg/kv/kvserver/concurrency/lock_table_waiter.go, line 477 at r2 (raw file):
// until completion (except when DontRetryPushTxnFailures is set). // Transform the error into a WriteIntentError. if _, ok := err.GetDetail().(*roachpb.TransactionPushError); ok && pushType == roachpb.PUSH_TOUCH {
consider looking at the req.WaitPolicy
again instead of looking at the pushType
. I think I'd find that clearer and you wouldn't need the comment above.
a89adf9
to
e5f43b4
Compare
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
TFTR!
Reviewable status: complete! 1 of 0 LGTMs obtained (waiting on @andreimatei)
pkg/kv/kvserver/concurrency/lock_table_waiter.go, line 477 at r2 (raw file):
Previously, andreimatei (Andrei Matei) wrote…
consider looking at the
req.WaitPolicy
again instead of looking at thepushType
. I think I'd find that clearer and you wouldn't need the comment above.
That's actually how I had it, and I switched to this because I was thinking that other wait policies might use a PUSH_TOUCH in the future. But I think you're right that mapping a TransactionPushError to a WriteIntentError is a property of the wait policy, not the push type. I'll switch back.
Informs cockroachdb#40476. Informs cockroachdb#51624. This commit introduces the concept of request "wait policies". A WaitPolicy specifies the behavior of a request when it encounters conflicting locks held by other active transactions. The default behavior is to block until the conflicting lock is released, but other policies can make sense in special situations. Within this new formalization, the commit creates two initial wait policy variants: ``` // Block indicates that if a request encounters a conflicting locks held by // another active transaction, it should wait for the conflicting lock to be // released before proceeding. Block = 0; // Error indicates that if a request encounters a conflicting locks held by // another active transaction, it should raise an error instead of blocking. Error = 1; ``` `Block` is equivalent to the current behavior, so there's no further action needed for that variant. However, the `Error` policy is new, and this commit teaches the `lockTableWaiter` about it. It ensures that when a request with the `Error` wait policy encounters a conflicting lock, it uses a PUSH_TOUCH PushRequest to determine whether the lock is abandoned or whether its holder is still active. If the holder is abandoned, the request cleans up the lock and proceeds. If the holder is still active, a WriteIntentError is returned to the client. This will unblock both of the referenced issues.
e5f43b4
to
1682c40
Compare
bors r+ |
Build succeeded: |
Fixes half of cockroachdb#40476. In cockroachdb#52388, we added support for a new `Error` wait policy to be attached to BatchRequests. This wait policy dictates that if a request encounters a conflicting locks held by another active transaction, it should raise an error instead of blocking. These semantics closely match those of FOR {UPDATE,SHARE} NOWAIT in PostgreSQL. With NOWAIT, the statement reports an error, rather than waiting, if a selected row cannot be locked immediately. This means that we are able to use the new kv-level wait policy to implement NOWAIT in SQL. Doing so is fairly mechanical, as a lot of the plumbing necessary was already put into place in cockroachdb#44429. The only difference I can see in the semantics is that in PostgreSQL, rows that are being INSERTed by one transaction are not even considered for locking by other transactions, so NOWAIT will not throw an error when encountering these rows. Instead, it will silently ignore them. This has less to do with NOWAIT itself, and more to do with a difference in the concurrency control implementation between Cockroach and PostgreSQL. Since this isn't specific to NOWAIT, I don't think it's a blocker for this change. Release note (sql change): SELECT ... FOR {UPDATE,SHARE} NOWAIT is now supported. The option can be used to throw and error instead of blocking on contended row-level lock acquisition.
Fixes half of cockroachdb#40476. In cockroachdb#52388, we added support for a new `Error` wait policy to be attached to BatchRequests. This wait policy dictates that if a request encounters a conflicting locks held by another active transaction, it should raise an error instead of blocking. These semantics closely match those of FOR {UPDATE,SHARE} NOWAIT in PostgreSQL. With NOWAIT, the statement reports an error, rather than waiting, if a selected row cannot be locked immediately. This means that we are able to use the new kv-level wait policy to implement NOWAIT in SQL. Doing so is fairly mechanical, as a lot of the plumbing necessary was already put into place in cockroachdb#44429. The only difference I can see in the semantics is that in PostgreSQL, rows that are being INSERTed by one transaction are not even considered for locking by other transactions, so NOWAIT will not throw an error when encountering these rows. Instead, it will silently ignore them. This has less to do with NOWAIT itself, and more to do with a difference in the concurrency control implementation between Cockroach and PostgreSQL. Since this isn't specific to NOWAIT, I don't think it's a blocker for this change. Release note (sql change): SELECT ... FOR {UPDATE,SHARE} NOWAIT is now supported. The option can be used to throw and error instead of blocking on contended row-level lock acquisition.
Fixes half of cockroachdb#40476. In cockroachdb#52388, we added support for a new `Error` wait policy to be attached to BatchRequests. This wait policy dictates that if a request encounters a conflicting locks held by another active transaction, it should raise an error instead of blocking. These semantics closely match those of FOR {UPDATE,SHARE} NOWAIT in PostgreSQL. With NOWAIT, the statement reports an error, rather than waiting, if a selected row cannot be locked immediately. This means that we are able to use the new kv-level wait policy to implement NOWAIT in SQL. Doing so is fairly mechanical, as a lot of the plumbing necessary was already put into place in cockroachdb#44429. The only difference I can see in the semantics is that in PostgreSQL, rows that are being INSERTed by one transaction are not even considered for locking by other transactions, so NOWAIT will not throw an error when encountering these rows. Instead, it will silently ignore them. This has less to do with NOWAIT itself, and more to do with a difference in the concurrency control implementation between Cockroach and PostgreSQL. Since this isn't specific to NOWAIT, I don't think it's a blocker for this change. Release note (sql change): SELECT ... FOR {UPDATE,SHARE} NOWAIT is now supported. The option can be used to throw and error instead of blocking on contended row-level lock acquisition.
52522: sql: support FOR {UPDATE,SHARE} NOWAIT r=nvanbenschoten a=nvanbenschoten Fixes half of #40476. In #52388, we added support for a new `Error` wait policy to be attached to BatchRequests. This wait policy dictates that if a request encounters a conflicting locks held by another active transaction, it should raise an error instead of blocking. These semantics closely match those of FOR {UPDATE,SHARE} NOWAIT in PostgreSQL. With NOWAIT, the statement reports an error, rather than waiting, if a selected row cannot be locked immediately. This means that we are able to use the new kv-level wait policy to implement NOWAIT in SQL. Doing so is fairly mechanical, as a lot of the plumbing necessary was already put into place in #44429. The only difference I can see in the semantics is that in PostgreSQL, rows that are being INSERTed by one transaction are not even considered for locking by other transactions, so NOWAIT will not throw an error when encountering these rows. Instead, it will silently ignore them. This has less to do with NOWAIT itself, and more to do with a difference in the concurrency control implementation between Cockroach and PostgreSQL. Since this isn't specific to NOWAIT, I don't think it's a blocker for this change. Release note (sql change): SELECT ... FOR {UPDATE,SHARE} NOWAIT is now supported. The option can be used to throw and error instead of blocking on contended row-level lock acquisition. 52628: importccl: support IMPORT INTO for DELIMITED and PGCOPY r=pbardea a=Anzoteh96 Previously, IMPORT INTO is supported only for CSV and AVRO. This PR extends the support of IMPORT INTO for DELIMITED and PGCOPY, in the hope that this will be useful when supporting default columns for imports from these files. This involves populating the targeted columns from import processor when creating new input reader and row converter for DELIMITED and PGCOPY, when the targeted columns are specified in `IMPORT INTO`. Fixes #52405 Release note (general): IMPORT INTO is now supported for DELIMITED and PGCOPY file formats. 52769: sql: avoid flake in scan_test r=asubiotto a=jordanlewis The new sqlliveness package doesn't get along well with the kv batch size "atomic", so turn down its check frequency in the test. Possibly we should make the "atomic" actually atomic, but I'm not sure about the runtime performance change. Fixes #52683 Release note: None Co-authored-by: Nathan VanBenschoten <[email protected]> Co-authored-by: anzoteh96 <[email protected]> Co-authored-by: Jordan Lewis <[email protected]>
Informs #40476.
Informs #51624.
This commit introduces the concept of request "wait policies". A
WaitPolicy specifies the behavior of a request when it encounters
conflicting locks held by other active transactions. The default
behavior is to block until the conflicting lock is released, but other
policies can make sense in special situations.
Within this new formalization, the commit creates two initial wait
policy variants:
Block
is equivalent to the current behavior, so there's no furtheraction needed for that variant.
However, the
Error
policy is new, and this commit teaches thelockTableWaiter
about it. It ensures that when a request with theError
wait policy encounters a conflicting lock, it uses a PUSH_TOUCHPushRequest to determine whether the lock is abandoned or whether its
holder is still active. If the holder is abandoned, the request cleans
up the lock and proceeds. If the holder is still active, a
WriteIntentError is returned to the client.
This will unblock both of the referenced issues.