-
Notifications
You must be signed in to change notification settings - Fork 3.9k
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
kvserver: improve intent cleanup for disconnected clients #64869
Conversation
47ca3e9
to
f0d8c8d
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.
Reviewed 1 of 1 files at r1, 6 of 6 files at r2.
Reviewable status:complete! 0 of 0 LGTMs obtained (waiting on @aliher1911 and @erikgrinaker)
pkg/kv/txn.go, line 36 at r2 (raw file):
// clients) then cleanup will be abandoned when the timeout expires. // // We generally want to clean up if possible, so we set it high at 1 minute. If
It seems difficult to properly set this timeout. We want to have some kind of release valve in cases where intent resolution is not keeping up to avoid unbounded goroutine growth. So at the limit, a timeout makes sense. However, we have a competing goal of spending as much time as we need to under normal operation to ensure that intents are cleaned up without leaks.
1 minute seems sufficient for most cases, but we've also seen cases where ranged intent resolution can lead to very slow intent resolution. This makes me wonder whether a hybrid scheme might be appropriate. I'm imagining some kind quota that permits a small number of operations to run without a timeout. If a rollback is able to reserve quota, it is able to run without a timeout. Otherwise, it must use a 1 minute timeout. This is an extension of the semi-sync pattern ("semi-semi-sync"?) we have elsewhere that would help place a soft limit on intent resolution concurrency.
Or maybe we should be setting this timeout dynamically based on some prediction of the cost of intent resolution. Even a basic heuristic might be sufficient, like 1 minute for transactions that have full fidelity in their intent tracking and 10 minutes for transactions that have condensed their intent tracking.
pkg/kv/kvserver/intent_resolver_integration_test.go, line 217 at r2 (raw file):
// Tests that intents and transaction records are cleaned up within a reasonable // timeframe in various scenarios. func TestIntentResolution(t *testing.T) {
Nice test!
Small nit: is the name too broad? This is a comprehensive test, and yet we already have a TestIntentResolution
and the name doesn't really say much about which aspects of the process are being tested. Maybe TestIntentResolutionReliablyCleansUp
or something that hints at the assertions the test is making?
pkg/kv/kvserver/intent_resolver_integration_test.go, line 274 at r2 (raw file):
} // assertIntents makes sure intents get cleaned up within a reasonable time.
nit: assertIntentCleanup
pkg/kv/kvserver/intent_resolver_integration_test.go, line 332 at r2 (raw file):
key := genKey(spec.singleRange) batch.Put(key, []byte("value")) if (i > 0 && i%10000 == 0) || i == spec.numKeys-1 {
nit: pull 10000
into a const batchSize = 10000
const right above this line.
pkg/kv/kvserver/intent_resolver_integration_test.go, line 351 at r2 (raw file):
rollbackCtx, cancel := context.WithCancel(ctx) cancel() if err := txn.Rollback(rollbackCtx); !errors.Is(err, context.Canceled) {
Do we want to test committing with a canceled context as well?
pkg/kv/kvserver/intent_resolver_integration_test.go, line 381 at r2 (raw file):
} type testStmtSpec struct {
What is Stmt
referring to here? Should this be testNonTxnSpec
?
pkg/kv/kvserver/replica_write.go, line 301 at r2 (raw file):
if ba.ContainsEndTxnRequest() { _ = r.store.stopper.RunAsyncTask( context.Background(),
Do we want to annotate this context?
pkg/kv/kvserver/replica_write.go, line 302 at r2 (raw file):
_ = r.store.stopper.RunAsyncTask( context.Background(), "async txn cleanup",
nit: consider pulling this into a const
since it's used twice.
pkg/kv/kvserver/replica_write.go, line 318 at r2 (raw file):
}) if err != nil { log.Warningf(ctx, "%v", err)
Want to add a bit more to this warning so that we'll know what it means in the logs more easily?
pkg/kv/kvserver/intentresolver/intent_resolver.go, line 773 at r1 (raw file):
ctx, cancel := ir.stopper.WithCancelOnQuiesce(ctx) defer cancel() ctx, cancel = context.WithTimeout(ctx, gcTxnRecordTimeout)
I'm surprised that this isn't running into issues with the TestContext
lint check.
EDIT: never mind, the next commit fixed this. Mind moving this to the first commit so that each commit passes tests on their own?
pkg/roachpb/batch.go, line 218 at r2 (raw file):
} // ContainsEndTxnRequest returns true iff the batch contains a EndTxnRequest.
The way this is usually done is if _, ok := ba.GetArg(roachpb.EndTxn); ok {
. This also has a convenient fast-path for EndTxn requests.
32035e9
to
8d0c6df
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.
Reviewable status:
complete! 0 of 0 LGTMs obtained (waiting on @aliher1911 and @nvanbenschoten)
pkg/kv/txn.go, line 36 at r2 (raw file):
Previously, nvanbenschoten (Nathan VanBenschoten) wrote…
It seems difficult to properly set this timeout. We want to have some kind of release valve in cases where intent resolution is not keeping up to avoid unbounded goroutine growth. So at the limit, a timeout makes sense. However, we have a competing goal of spending as much time as we need to under normal operation to ensure that intents are cleaned up without leaks.
1 minute seems sufficient for most cases, but we've also seen cases where ranged intent resolution can lead to very slow intent resolution. This makes me wonder whether a hybrid scheme might be appropriate. I'm imagining some kind quota that permits a small number of operations to run without a timeout. If a rollback is able to reserve quota, it is able to run without a timeout. Otherwise, it must use a 1 minute timeout. This is an extension of the semi-sync pattern ("semi-semi-sync"?) we have elsewhere that would help place a soft limit on intent resolution concurrency.
Or maybe we should be setting this timeout dynamically based on some prediction of the cost of intent resolution. Even a basic heuristic might be sufficient, like 1 minute for transactions that have full fidelity in their intent tracking and 10 minutes for transactions that have condensed their intent tracking.
Yeah, I struggled with this as well. It seems to me like what we'd really want for intent cleanup is a (persistent?) unbounded queue with a bounded worker pool: this is not latency-sensitive, but we want it to be reliable with low concurrency. Backpressure doesn't seem critical here, since the hard work of laying down the intents has already been done.
I'm kind of tempted to just use a background context for these async cleanups for now, and revisit our overall intent cleanup approach. We already use a background context for parallel commits, which arguably has a stronger need for backpressure.
pkg/kv/kvserver/intent_resolver_integration_test.go, line 351 at r2 (raw file):
Previously, nvanbenschoten (Nathan VanBenschoten) wrote…
Do we want to test committing with a canceled context as well?
I don't think we can do that here -- it would just error with context.Canceled
and leave the intents behind. We generally rely on the SQL session to trigger cleanup on context cancellation, but we're not using a SQL connection here. We therefore just test cancellation by calling Rollback()
, which is the path that SQL would use to clean up in this case.
pkg/kv/kvserver/intent_resolver_integration_test.go, line 381 at r2 (raw file):
Previously, nvanbenschoten (Nathan VanBenschoten) wrote…
What is
Stmt
referring to here? Should this betestNonTxnSpec
?
"Statement". It was meant to test the path used by naked statements, but it doesn't really map that well, so I renamed it to testNonTxn
.
pkg/kv/kvserver/replica_write.go, line 301 at r2 (raw file):
Previously, nvanbenschoten (Nathan VanBenschoten) wrote…
Do we want to annotate this context?
Yes, done.
pkg/kv/kvserver/intentresolver/intent_resolver.go, line 773 at r1 (raw file):
Previously, nvanbenschoten (Nathan VanBenschoten) wrote…
I'm surprised that this isn't running into issues with the
TestContext
lint check.EDIT: never mind, the next commit fixed this. Mind moving this to the first commit so that each commit passes tests on their own?
Ah, my bad, amended the wrong commit. Should really get used to using git absorb
for this.
Transaction records and intents are normally cleaned up asynchronously via `IntentResolver.CleanupTxnIntentsAsync()`, separately from the client's context. When the async task limit is exceeded, cleanup instead becomes synchronous and attached to the client context. However, the final `gcTxnRecord()` call to remove the transaction record is asynchronous even when intent cleanup is synchronous, to avoid holding onto the intent resolver task slot. This call will typically return to the caller before `gcTxnRecord()` completes, which may cause the caller to cancel the context (either via `defer cancel()` or a client disconnection) and in turn cancel the `gcTxnRecord()` call. This patch gives the async `gcTxnRecord()` call a separate background context that's independent of the client's context even in the synchronous case, with a 20 second timeout to avoid goroutine leaks. Release note (bug fix): Fixed a race condition during transaction cleanup that could leave old transaction records behind until MVCC garbage collection.
Whenever a client disconnects during an open transaction or in-flight DML statement, the server-side session will asynchronously clean up the transaction record and intents by rolling back the transaction. However, this had a very short timeout of 3 seconds, and even though the actual `IntentResolver.CleanupTxnIntentsAsync()` cleanup task runs asynchronously and independently of the client's context in the typical case, the short timeout could cause transaction cleanup to be cancelled if the `EndTxn` request took more than 3 seconds to get all the way through Raft execution or if the async task limit was exceeded such that the cleanup task kept using the client's context. This in turn could lead to intents building up over time. This patch increases the timeout when rolling back transactions for disconnected clients to 1 minute, and also tries to perform transaction cleanup when a client disconnects while waiting for an `EndTxn` command to go through Raft. Release note (bug fix): improved transaction cleanup for disconnected clients, to reduce intent buildup.
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! 0 of 0 LGTMs obtained (waiting on @aliher1911 and @nvanbenschoten)
pkg/kv/txn.go, line 36 at r2 (raw file):
Previously, erikgrinaker (Erik Grinaker) wrote…
Yeah, I struggled with this as well. It seems to me like what we'd really want for intent cleanup is a (persistent?) unbounded queue with a bounded worker pool: this is not latency-sensitive, but we want it to be reliable with low concurrency. Backpressure doesn't seem critical here, since the hard work of laying down the intents has already been done.
I'm kind of tempted to just use a background context for these async cleanups for now, and revisit our overall intent cleanup approach. We already use a background context for parallel commits, which arguably has a stronger need for backpressure.
Since we agreed offline to keep it simple for now, I suggest we just keep this 1 minute timeout for disconnected cleanups, and I've opened #65204 to use a queue for intent cleanup when we have time. In the typical case here there will be async task capacity, and so the cleanup will run with a background context anyway. Wdyt?
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.
Reviewed 5 of 5 files at r4.
Reviewable status:complete! 1 of 0 LGTMs obtained (waiting on @aliher1911 and @erikgrinaker)
pkg/kv/txn.go, line 36 at r2 (raw file):
Previously, erikgrinaker (Erik Grinaker) wrote…
Since we agreed offline to keep it simple for now, I suggest we just keep this 1 minute timeout for disconnected cleanups, and I've opened #65204 to use a queue for intent cleanup when we have time. In the typical case here there will be async task capacity, and so the cleanup will run with a background context anyway. Wdyt?
This all SGTM.
pkg/kv/kvserver/intent_resolver_integration_test.go, line 351 at r2 (raw file):
Previously, erikgrinaker (Erik Grinaker) wrote…
I don't think we can do that here -- it would just error with
context.Canceled
and leave the intents behind. We generally rely on the SQL session to trigger cleanup on context cancellation, but we're not using a SQL connection here. We therefore just test cancellation by callingRollback()
, which is the path that SQL would use to clean up in this case.
Ack. Thanks for the explanation.
bors r=nvanbenschoten |
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.
Reviewed 1 of 6 files at r2, 5 of 5 files at r4.
Reviewable status:complete! 1 of 0 LGTMs obtained (waiting on @erikgrinaker)
Build succeeded: |
kvserver: improve intent cleanup for disconnected clients
Whenever a client disconnects during an open transaction or in-flight
DML statement, the server-side session will asynchronously clean up the
transaction record and intents by rolling back the transaction.
However, this had a very short timeout of 3 seconds, and even though the
actual
IntentResolver.CleanupTxnIntentsAsync()
cleanup task runsasynchronously and independently of the client's context in the typical
case, the short timeout could cause transaction cleanup to be cancelled
if the
EndTxn
request took more than 3 seconds to get all the waythrough Raft execution or if the async task limit was exceeded such that
the cleanup task kept using the client's context. This in turn could
lead to intents building up over time.
This patch increases the timeout when rolling back transactions for
disconnected clients to 1 minute, and also tries to perform transaction
cleanup when a client disconnects while waiting for an
EndTxn
commandto go through Raft.
Resolves #64770, touches #60585.
Release note (bug fix): improved transaction cleanup for disconnected
clients, to reduce intent buildup.
kvserver: fix race condition during synchronous txn record cleanup
Transaction records and intents are normally cleaned up asynchronously
via
IntentResolver.CleanupTxnIntentsAsync()
, separately from theclient's context. When the async task limit is exceeded, cleanup instead
becomes synchronous and attached to the client context. However, the
final
gcTxnRecord()
call to remove the transaction record isasynchronous even when intent cleanup is synchronous, to avoid holding
onto the intent resolver task slot. This call will typically return to
the caller before
gcTxnRecord()
completes, which may cause the caller tocancel the context (either via
defer cancel()
or a clientdisconnection) and in turn cancel the
gcTxnRecord()
call.This patch gives the async
gcTxnRecord()
call a separate backgroundcontext that's independent of the client's context even in the
synchronous case, with a 20 second timeout to avoid goroutine leaks.
Resolves #64868, touches #60585.
Release note (bug fix): Fixed a race condition during transaction
cleanup that could leave old transaction records behind until MVCC
garbage collection.
/cc @cockroachdb/kv