kv: resolve conflicting intents immediately for latency-sensitive requests#64500
kv: resolve conflicting intents immediately for latency-sensitive requests#64500nvb wants to merge 1 commit intocockroachdb:masterfrom
Conversation
erikgrinaker
left a comment
There was a problem hiding this comment.
Nice optimization! There might be further gains to be had by tweaking the batching parameters as well, opened #64556 to benchmark and tune this.
Reviewed 4 of 4 files at r1.
Reviewable status:complete! 0 of 0 LGTMs obtained (waiting on @aliher1911 and @nvanbenschoten)
pkg/kv/kvserver/concurrency/lock_table_waiter.go, line 642 at r1 (raw file):
// See pushLockTxn for an explanation of these options. We only set the // SendImmediately flag to true if we are resolving fewer than or exactly // IntentResolverBatchSize intents. Otherwise, this a bulk intent resolution
...this is a bulk...
pkg/kv/kvserver/concurrency/lock_table_waiter.go, line 647 at r1 (raw file):
opts := intentresolver.ResolveOptions{ Poison: true, SendImmediately: len(deferredResolution) <= intentresolver.IntentResolverBatchSize,
A single deferredResolution item can be a range span covering thousands of intents. Presumably we do pagination in the intent resolver to avoid these requests from holding onto latches for too long without interruption, and we're bypassing that mechanism here.
It also seems a bit iffy to have to enforce intent resolver invariants outside of the intent resolver.
pkg/kv/kvserver/intentresolver/intent_resolver.go, line 796 at r1 (raw file):
// that is willing to trade off some throughput for lower latency. // // In addition to disabling batching, the option will also disable pagination.
As mentioned above, I don't think we'll want to disable pagination here. How about we add a Flush() method to RequestBatcher which immediately sends off the pending batch, but then paginates as usual? We could also consider flushing again for each of the resumespans and/or subsequent batches, to lower the latency of resolving large ranges.
pkg/kv/kvserver/intentresolver/intent_resolver.go, line 861 at r1 (raw file):
batcher = ir.irRangeBatcher default: panic("unexpected")
Might want to add in %T here, although it shouldn't really happen.
pkg/kv/kvserver/intentresolver/intent_resolver.go, line 889 at r1 (raw file):
var rangeReqs []roachpb.ResolveIntentRangeRequest for i := range intents { intent := &intents[i]
Is this a performance optimization over for i, intent := range?
pkg/kv/kvserver/intentresolver/intent_resolver.go, line 914 at r1 (raw file):
} for i := range rangeReqs { reqs[i+len(pointReqs)] = &rangeReqs[i]
What's the benefit in reordering ranged intents at the end? Getting the presumably faster point intents out of the way first?
…uests Related to cockroachdb#60585. This commit adds a new `SendImmediately` option to the `IntentResolver` `ResolveOptions`, which instructs the `IntentResolver` to send the provided intent resolution requests immediately, instead of adding them to a batch and waiting up to 10ms (defaultIntentResolutionBatchWait) for that batch to fill up with other intent resolution requests. This can be used to avoid any batching-induced latency and should be used only by foreground traffic that is willing to trade off some throughput for lower latency. The commit then sets this flag for intent resolution requests initiated by the `lockTableWaiter`. Unlike most calls into the `IntentResolver`, which are performed by background tasks that are more than happy to trade increased latency for increased throughput, the `lockTableWaiter` calls into the `IntentResolver` on behalf of a foreground operation. It wants intent resolution to complete as soon as possible, so it is willing to trade reduced throughput for reduced latency. I tested this out by abandoning 100 different intents by different transactions in a table by disabling async intent resolution and then scanning over the table: ``` ./cockroach sql --insecure -e 'create table a (i int); create table b (i int);' for i in {1..100}; do ./cockroach sql --insecure -e "begin; insert into a values (${i}); insert into b values (${i}); commit;" done ./cockroach sql --insecure -e 'select count(*) from b' ``` Without this change, the scan took 8.1s. With this change, the scan took 7.6s. This 6% speed up of about 500ms checks out. In either case, the waiter needs to wait for the (configurable) 50ms `kv.lock_table.coordinator_liveness_push_delay` to expire before it pushes each txn. However, without this change, it then waits another 5ms (`defaultIntentResolutionBatchIdle`) before issuing each of its resolve intent requests. With this change, these resolve intent requests are issued immediately. The speedup is more pronounced with a high-priority scan, which does not wait for the 50ms `kv.lock_table.coordinator_liveness_push_delay`. If we run the scan with high-priority, then without this change, it takes 3.3s and with it, it takes 2.8s, a 15% speedup. Release note (performance improvement): SQL statements that must clean up intents from previous statements now do so moderately more efficiently.
17237d4 to
de71992
Compare
aliher1911
left a comment
There was a problem hiding this comment.
Reviewable status:
complete! 0 of 0 LGTMs obtained (waiting on @erikgrinaker and @nvanbenschoten)
pkg/kv/kvserver/concurrency/lock_table_waiter.go, line 647 at r1 (raw file):
Previously, erikgrinaker (Erik Grinaker) wrote…
A single
deferredResolutionitem can be a range span covering thousands of intents. Presumably we do pagination in the intent resolver to avoid these requests from holding onto latches for too long without interruption, and we're bypassing that mechanism here.It also seems a bit iffy to have to enforce intent resolver invariants outside of the intent resolver.
Maybe we can use SendImmediately as a hint and let resolver decide if it would do it immediately or not? That would keep the policy inside the callee and requested behaviour in caller.
…uests Fixes cockroachdb#50390. Related to cockroachdb#60585. Related to cockroachdb#103126. Closes cockroachdb#64500, which was an earlier attempt to solve this issue using a similar approach. This commit addresses the comments on that PR, which focused on the pagination of intent resolution when bypassing the request batcher. We still don't try to solve this issue, and instead limit the cases where intent resolution bypasses the request batcher to those where pagination is not necessary. This commit adds a new `sendImmediately` option to the `IntentResolver` `ResolveOptions`, which instructs the `IntentResolver` to send the provided intent resolution requests immediately, instead of adding them to a batch and waiting up to 10ms (defaultIntentResolutionBatchWait) for that batch to fill up with other intent resolution requests. This can be used to avoid any batching-induced latency and should be used only by foreground traffic that is willing to trade off some throughput for lower latency. The commit then sets this flag for single-key intent resolution requests initiated by the `lockTableWaiter`. Unlike most calls into the `IntentResolver`, which are performed by background tasks that are more than happy to trade increased latency for increased throughput, the `lockTableWaiter` calls into the `IntentResolver` on behalf of a foreground operation. It wants intent resolution to complete as soon as possible, so it is willing to trade reduced throughput for reduced latency. I tested this out by writing 10,000 different intents in a normal-priority transaction and then scanning over the table using a high-priority transaction. The test was performed on a 3-node roachprod cluster to demonstrate the effect with realistic network and disk latencies. ``` -- session 1 CREATE TABLE keys (k BIGINT NOT NULL PRIMARY KEY); BEGIN; INSERT INTO keys SELECT generate_series(1, 10000); -- session 2 BEGIN PRIORITY HIGH; SELECT count(*) FROM keys; ``` Without this change, the scan took 70.078s. With this change, the scan took 15.958s. This 78% speed up checks out. Each encountered intent is resolved serially (see cockroachdb#103126), so the per-intent latency drops from 7ms to 1.6ms. This improvement by about 5ms agrees with the `defaultIntentResolutionBatchIdle`, which delays each resolution request that passes through a RequestBatcher. With this change, these resolve intent requests are issued immediately and this delay is not experienced. While this is a significant improvement and will be important for Read Committed transactions after cockroachdb#102014, this is still not quite enough to resolve cockroachdb#103126. For that, we need to batch the resolve intent requests themselves using a form of deferred intent resolution like we added in cockroachdb#49218 (for finalized transactions). A similar improvement is seen for scans that encounter many abandoned intents from many different transactions. This scenario bypasses the deferred intent resolution path added in cockroachdb#49218, because the intents are each written by different transactions. The speedup for this scenario was presented in cockroachdb#64500. Release note (performance improvement): SQL statements that must clean up intents from many different previously abandoned transactions now do so moderately more efficiently.
|
Replacing with #103265, which addresses the commentary here. |
…uests Fixes cockroachdb#50390. Related to cockroachdb#60585. Related to cockroachdb#103126. Closes cockroachdb#64500, which was an earlier attempt to solve this issue using a similar approach. This commit addresses the comments on that PR, which focused on the pagination of intent resolution when bypassing the request batcher. We still don't try to solve this issue, and instead limit the cases where intent resolution bypasses the request batcher to those where pagination is not necessary. This commit adds a new `sendImmediately` option to the `IntentResolver` `ResolveOptions`, which instructs the `IntentResolver` to send the provided intent resolution requests immediately, instead of adding them to a batch and waiting up to 10ms (defaultIntentResolutionBatchWait) for that batch to fill up with other intent resolution requests. This can be used to avoid any batching-induced latency and should be used only by foreground traffic that is willing to trade off some throughput for lower latency. The commit then sets this flag for single-key intent resolution requests initiated by the `lockTableWaiter`. Unlike most calls into the `IntentResolver`, which are performed by background tasks that are more than happy to trade increased latency for increased throughput, the `lockTableWaiter` calls into the `IntentResolver` on behalf of a foreground operation. It wants intent resolution to complete as soon as possible, so it is willing to trade reduced throughput for reduced latency. I tested this out by writing 10,000 different intents in a normal-priority transaction and then scanning over the table using a high-priority transaction. The test was performed on a 3-node roachprod cluster to demonstrate the effect with realistic network and disk latencies. ``` -- session 1 CREATE TABLE keys (k BIGINT NOT NULL PRIMARY KEY); BEGIN; INSERT INTO keys SELECT generate_series(1, 10000); -- session 2 BEGIN PRIORITY HIGH; SELECT count(*) FROM keys; ``` Without this change, the scan took 70.078s. With this change, the scan took 15.958s. This 78% speed up checks out. Each encountered intent is resolved serially (see cockroachdb#103126), so the per-intent latency drops from 7ms to 1.6ms. This improvement by about 5ms agrees with the `defaultIntentResolutionBatchIdle`, which delays each resolution request that passes through a RequestBatcher. With this change, these resolve intent requests are issued immediately and this delay is not experienced. While this is a significant improvement and will be important for Read Committed transactions after cockroachdb#102014, this is still not quite enough to resolve cockroachdb#103126. For that, we need to batch the resolve intent requests themselves using a form of deferred intent resolution like we added in cockroachdb#49218 (for finalized transactions). A similar improvement is seen for scans that encounter many abandoned intents from many different transactions. This scenario bypasses the deferred intent resolution path added in cockroachdb#49218, because the intents are each written by different transactions. The speedup for this scenario was presented in cockroachdb#64500. Release note (performance improvement): SQL statements that must clean up intents from many different previously abandoned transactions now do so moderately more efficiently.
103265: kv: resolve conflicting intents immediately for latency-sensitive requests r=nvanbenschoten a=nvanbenschoten Fixes #50390. Related to #60585. Related to #103126. Closes #64500, which was an earlier attempt to solve this issue using a similar approach. This commit addresses the comments on that PR, which focused on the pagination of intent resolution when bypassing the request batcher. We still don't try to solve this issue, and instead limit the cases where intent resolution bypasses the request batcher to those where pagination is not necessary. This commit adds a new `sendImmediately` option to the `IntentResolver` `ResolveOptions`, which instructs the `IntentResolver` to send the provided intent resolution requests immediately, instead of adding them to a batch and waiting up to 10ms (`defaultIntentResolutionBatchWait`) for that batch to fill up with other intent resolution requests. This can be used to avoid any batching-induced latency and should be used only by foreground traffic that is willing to trade off some throughput for lower latency. The commit then sets this flag for single-key intent resolution requests initiated by the `lockTableWaiter`. Unlike most calls into the `IntentResolver`, which are performed by background tasks that are more than happy to trade increased latency for increased throughput, the `lockTableWaiter` calls into the `IntentResolver` on behalf of a foreground operation. It wants intent resolution to complete as soon as possible, so it is willing to trade reduced throughput for reduced latency. I tested this out by writing 10,000 different intents in a normal-priority transaction and then scanning over the table using a high-priority transaction. The test was performed on a 3-node roachprod cluster to demonstrate the effect with realistic network and disk latencies. ```sql -- session 1 CREATE TABLE keys (k BIGINT NOT NULL PRIMARY KEY); BEGIN; INSERT INTO keys SELECT generate_series(1, 10000); -- session 2 BEGIN PRIORITY HIGH; SELECT count(*) FROM keys; ``` Without this change, the scan took **70.078s**. With this change, the scan took **15.958s**. This **78%** speed-up checks out. Each encountered intent is resolved serially (see #103126), so the **per-intent latency** drops from **7ms** to **1.6ms.** This improvement by about 5ms agrees with the `defaultIntentResolutionBatchIdle`, which delays each resolution request that passes through a RequestBatcher. With this change, these resolve intent requests are issued immediately and this delay is not experienced. While this is a significant improvement and will be important for Read Committed transactions after #102014, this is still not quite enough to resolve #103126. For that, we need to batch the resolve intent requests themselves using a form of deferred intent resolution like we added in #49218 (for finalized transactions). A similar improvement is seen for scans that encounter many abandoned intents from many different transactions. This scenario bypasses the deferred intent resolution path added in #49218, because the intents are each written by different transactions. The speedup for this scenario was presented in #64500. Release note (performance improvement): SQL statements that must clean up intents from many different previously abandoned transactions now do so moderately more efficiently. 103362: sql: validate primary / secondary region localities at end of txn r=fqazi a=fqazi Previously, if a database was restored with skip_localities, there was no way to modify this database to set the primary region since validation would kick in too early during the statement. This meant fixing the regions in a restored database was impossible if the primary region was no longer valid. To address this, this patch, delays locality validation till the end of the transaction. Fixes: #103290 Release note (bug fix): SET PRIMARY REGION and SET SECONDARY REGION did not validate transactionally, which could prevent cleaning up removed regions after a restore. 103373: concurrency: small refactors in preparation for reservation removal r=nvanbenschoten a=arulajmani See individual commits for details. Informs: #103361 103538: go.mod: bump Pebble to 6f2788660198, rework shared storage wrapper r=RaduBerinde a=RaduBerinde 6f278866 shared: improve interface for more efficient reading 9eb2c407 db: log events to testing.T in unit tests f32e7dc6 db: add reserved Pebblev4 sstable format 5a6b91b8 objstorage: improve test and add read ahead test 2bc4319e objstorage: remove genericFileReadable 8143ffb9 objstorage: fix readaheadState initialization 06d08888 db: add Compact.Duration metric e7213de0 db: add Uptime metric e9005aed db: don't delete files during ingest application 222b43ec internal/arenaskl: fix Skiplist doc typo Release note: None Epic: none Co-authored-by: Nathan VanBenschoten <nvanbenschoten@gmail.com> Co-authored-by: Faizan Qazi <faizan@cockroachlabs.com> Co-authored-by: Arul Ajmani <arulajmani@gmail.com> Co-authored-by: Radu Berinde <radu@cockroachlabs.com>
Related to #60585.
Fixes #50390.
This commit adds a new
SendImmediatelyoption to theIntentResolverResolveOptions, which instructs theIntentResolverto send the provided intent resolution requests immediately, instead of adding them to a batch and waiting up to 10ms (defaultIntentResolutionBatchWait) for that batch to fill up with other intent resolution requests. This can be used to avoid any batching-induced latency and should be used only by foreground traffic that is willing to trade off some throughput for lower latency.The commit then sets this flag for intent resolution requests initiated by the
lockTableWaiter. Unlike most calls into theIntentResolver, which are performed by background tasks that are more than happy to trade increased latency for increased throughput, thelockTableWaitercalls into theIntentResolveron behalf of a foreground operation. It wants intent resolution to complete as soon as possible, so it is willing to trade reduced throughput for reduced latency.I tested this out by abandoning 100 different intents by different transactions in a table by disabling async intent resolution and then scanning over the table:
Without this change, the scan took 8.1s. With this change, the scan took 7.6s. This 6% speedup of about 500ms checks out. In either case, the waiter needs to wait for the (configurable) 50ms
kv.lock_table.coordinator_liveness_push_delayto expire before it pushes each txn. However, without this change, it then waits another 5ms (defaultIntentResolutionBatchIdle) before issuing each of its resolve intent requests. With this change, these resolve intent requests are issued immediately. The speedup is more pronounced with a high-priority scan, which does not wait for the 50mskv.lock_table.coordinator_liveness_push_delay. If we run the scan with high priority, then without this change, it takes 3.3s and with it, it takes 2.8s, a 15% speedup.Release note (performance improvement): SQL statements that must clean up intents from previous statements now do so moderately more efficiently.
/cc @cockroachdb/kv