-
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
intentresolver: batch intent resolution across transactions by range #34803
intentresolver: batch intent resolution across transactions by range #34803
Conversation
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.
This is really nice! You can add a "Fixes #30780" to the third commit.
The benchmarks look great. I'm interested in the effect of this change with --batch=1 --secondary-index
. I'd also be interested in the worst-case scenario, where the batching isn't doing anything except preventing concurrent transactions from proceeding.
How should that work? Add a semaphore for total concurrency? Per range?
Adding a semaphore that bounds the total number of in-flight requests sounds like a good idea to me. I don't think we need to be all that fancy here.
- Question for my own edification: will we ever try to resolve an intent on a
Key which will failkeys.Addr
?
keys.Addr
really should never fail if we're dealing with properly built keys. The errors cases are all about malformed keys.
- Cancellation won't cancel requests queued in the requestbatcher but not sent.
Since we're not relying on this right now, I wouldn't add any extra complication.
- Should we hold off on this performance change until after the stabilization
period and release?
Let's discuss this tomorrow. I think this kind of change would be ok to merge in the next week, but I'd like to verify that.
Reviewed 1 of 1 files at r1, 5 of 5 files at r2, 5 of 5 files at r3.
Reviewable status: complete! 0 of 0 LGTMs obtained (waiting on @ajwerner)
pkg/kv/range_cache.go, line 243 at r2 (raw file):
} func (rdc *RangeDescriptorCache) LookupRangeDescriptor(
You'll need a comment on this, although the linter probably already told you.
pkg/kv/kvbase/range_cache.go, line 30 at r3 (raw file):
LookupRangeDescriptor(ctx context.Context, key roachpb.RKey) ( *roachpb.RangeDescriptor, error)
I think our style guide recommends:
LookupRangeDescriptor(
ctx context.Context, key roachpb.RKey,
) (*roachpb.RangeDescriptor, error)
🤷♂️
pkg/storage/intentresolver/intent_resolver.go, line 178 at r3 (raw file):
Name: "intent_resolver_ir_batcher", MaxMsgsPerBatch: intentResolverBatchSize, // TODO(ajwerner): Justify these settings.
👍
We should make sure to discuss the effect that this can have on concurrent actors. Extra points if you use the phrase "contention footprint" somewhere in there.
pkg/storage/intentresolver/intent_resolver.go, line 703 at r3 (raw file):
Key: txnKey, }) _, err := ir.gcBatcher.Send(ctx, rangeID, &gcArgs)
Now that we've threaded a RangeDescriptorCache dependency in here, do you think we should use it for this? If not, let's leave a comment as to why.
pkg/storage/intentresolver/intent_resolver.go, line 806 at r3 (raw file):
rKey, err := keys.Addr(req.Key) if err != nil { errors.Wrapf(err, "failed to resolve addr for key %q", req.Key)
Are you meaning to return these errors.Wrapf
calls?
pkg/storage/intentresolver/intent_resolver.go, line 808 at r3 (raw file):
errors.Wrapf(err, "failed to resolve addr for key %q", req.Key) } rDesc, err := ir.rdc.LookupRangeDescriptor(ctx, rKey)
Leave a note that this is best-effort and is just for batching.
pkg/storage/intentresolver/intent_resolver.go, line 814 at r3 (raw file):
// Are there different classes of error here that require different // treatment? errors.Wrapf(err, "failed to look up range descriptor for key %q", req.Key)
I would just log a warning, probably with a log.EveryN
limiter. We can then just use rangeID 0, which will fall back to a single large batch request that needs to get split up.
pkg/storage/intentresolver/intent_resolver.go, line 842 at r3 (raw file):
if resp.Err != nil { return resp.Err }
nit: consider adding something like:
// Ignore the response.
_ = resp.Response
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.
The benchmarks look great. I'm interested in the effect of this change with --batch=1 --secondary-index
For the batch 1 with secondary index here's some preliminary results with --splits 256 and --concurrency 512. I suspect that the results are sensitive to both of those values. I'm also sure that the 32 core machines can push more concurrency than this. I'm going to try playing around with some of the settings.
8 core batch size 1 with secondary index:
name old ops/s new ops/s delta
KV0 2.14k ± 3% 2.25k ± 2% +5.18% (p=0.008 n=5+5)
32 core, batch size 1 with secondary index
name old ops/s new ops/s delta
KV0 3.66k ± 2% 3.89k ± 1% +6.34% (p=0.008 n=5+5)
Reviewable status: complete! 0 of 0 LGTMs obtained (waiting on @ajwerner)
This PR fixes a bug discovered during the implementation of cockroachdb#34803 whereby a nil batch may be send due to a stale timer that corresponds to the sole batch being sent due to size and no logic to cancel its timer. This condition occurs when there is one outstanding batch for a single range and it gets filled according to size constraints. Hopefully this case is rare because in cases where a batch is filled based on size limits hopefully there is traffic to another range. This is a pretty egregious bug. While fixing it I encountered another gotcha with regards to the timeutil.Timer.Stop method for which I added additional comments. Release note: None
This PR fixes a bug discovered during the implementation of cockroachdb#34803 whereby a nil batch may be send due to a stale timer that corresponds to the sole batch being sent due to size and no logic to cancel its timer. This condition occurs when there is one outstanding batch for a single range and it gets filled according to size constraints. Hopefully this case is rare because in cases where a batch is filled based on size limits hopefully there is traffic to another range. This is a pretty egregious bug. While fixing it I encountered another gotcha with regards to the timeutil.Timer.Stop method for which I added additional comments. Release note: None
34837: client/requestbatcher: fix panic when only batch is sent due to size r=ajwerner a=ajwerner This PR fixes a bug discovered during the implementation of #34803 whereby a nil batch may be send due to a stale timer that corresponds to the sole batch being sent due to size and no logic to cancel its timer. This condition occurs when there is one outstanding batch for a single range and it gets filled according to size constraints. Hopefully this case is rare because in cases where a batch is filled based on size limits hopefully there is traffic to another range. This is a pretty egregious bug. While fixing it I encountered another gotcha with regards to the timeutil.Timer.Stop method for which I added additional comments. Release note: None Co-authored-by: Andrew Werner <[email protected]>
5a849e5
to
99edbc9
Compare
Release note: None
99edbc9
to
f0f9db6
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 11 of 11 files at r4, 2 of 2 files at r5, 6 of 6 files at r6, 6 of 6 files at r7.
Reviewable status: complete! 1 of 0 LGTMs obtained (waiting on @ajwerner)
pkg/internal/client/requestbatcher/batcher.go, line 67 at r5 (raw file):
machanism
pkg/internal/client/requestbatcher/batcher.go, line 130 at r5 (raw file):
//Note that values // smaller than 10 will result in a limit of 10.
Why? Where is this enforced?
pkg/storage/gc_queue_test.go, line 677 at r7 (raw file):
var resolved syncmap.Map tsc.TestingKnobs.IntentResolverKnobs.MaxIntentResolutionBatchSize = 1
Let's explain why we need this.
pkg/storage/intentresolver/intent_resolver.go, line 90 at r7 (raw file):
for other contending writes
other contending reads or writes
This commit is an attempt to limit the total request concurrency that clients may produce when using a RequestBatcher. The mechanism added is designed to be simple and effective without introducing more complex infrastructure like queueing or batch delaying in order to adhere to stricter bounds. Release note: None
The existing LookupRangeDescriptor interface exposes details about eviction keys which are appropriate for internal users but confusing and unnecssary for read-only use of the object. This change also exposes a kvbase interface for the RangeDescriptorCache in anticipation of use in lower level packages. Release note: None
This PR modifies the IntentResolver to batch intent resolution across different transactions on a (best effort) per range basis. It acheives this batching by plumbing RangeDescriptorCache into the IntentResolver so that an intent-range mapping can be determined and then using the RequestBatcher introduced in concurrently without requiring additional goroutines this change extends the interface to allow the client to provide the response channel. **Performance Wins** The change yields significant throughput wins in targetted benchmarks. We expect that write-heavy workloads which generate a large number of intents scattered over ranges to benefit most from this change. The following benchmarks were run using kv0 with a batch size of 10 and secondary indices enabled (see command). On both 4- and 32-core nodes a ~30% throughput increase is observed. ``` ./workload run kv '{pgurl:1-3}' --init --splits=256 --duration 90s --batch 10 --secondary-index --read-percent=0 --concurrency=512 ``` 8-Core ``` name old ops/s new ops/s delta Cockroach 510 ± 3% 651 ± 7% +27.58% (p=0.008 n=5+5) ``` 32-core ``` name old ops/s new ops/s delta KV0 1.01k ± 3% 1.32k ± 1% +30.78% (p=0.008 n=5+5) ``` No change in performance for TPCC was observed: 8-Core (400 warehouses) ``` _elapsed_______tpmC____efc__avg(ms)__p50(ms)__p90(ms)__p95(ms)__p99(ms)_pMax(ms) 300.0s 6073.2 94.5% 180.0 167.8 268.4 318.8 402.7 2013.3 name old ops/s new ops/s delta tpmc 6.09k ± 1% 6.08k ± 0% ~ (p=0.886 n=4+4) ``` 32-Core (1500 warehouses) ``` _elapsed_______tpmC____efc__avg(ms)__p50(ms)__p90(ms)__p95(ms)__p99(ms)_pMax(ms) 300.0s 19303.0 93.8% 342.5 335.5 453.0 520.1 704.6 3623.9 name old ops/s new ops/s delta tpmc 19.3k ± 0% 19.3k ± 0% ~ (p=1.000 n=4+4) ``` Release note (performance improvement): Increase write throughput for workloads which write large numbers of intents by coalescing intent resolution requests across transactions.
f0f9db6
to
5b02cd1
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 (and 1 stale) (waiting on @nvanbenschoten)
pkg/internal/client/requestbatcher/batcher.go, line 130 at r5 (raw file):
Previously, nvanbenschoten (Nathan VanBenschoten) wrote…
//Note that values // smaller than 10 will result in a limit of 10.
Why? Where is this enforced?
Err, I had planned on doing this but didn't. Instead I added the default and made 1 work for the recovery threshold.
pkg/kv/kvbase/range_cache.go, line 30 at r3 (raw file):
Previously, nvanbenschoten (Nathan VanBenschoten) wrote…
LookupRangeDescriptor(ctx context.Context, key roachpb.RKey) ( *roachpb.RangeDescriptor, error)
I think our style guide recommends:
LookupRangeDescriptor( ctx context.Context, key roachpb.RKey, ) (*roachpb.RangeDescriptor, error)
🤷♂️
Done.
pkg/storage/intentresolver/intent_resolver.go, line 806 at r3 (raw file):
Previously, nvanbenschoten (Nathan VanBenschoten) wrote…
Are you meaning to return these
errors.Wrapf
calls?
I made the lookup best effort
bors r+ |
34803: intentresolver: batch intent resolution across transactions by range r=ajwerner a=ajwerner This PR modifies the IntentResolver to batch intent resolution across different transactions on a (best effort) per range basis. It acheives this batching by plumbing RangeDescriptorCache into the IntentResolver so that an intent-range mapping can be determined and then using the RequestBatcher introduced in concurrently without requiring additional goroutines this change extends the interface to allow the client to provide the response channel. **Performance Wins** The change yields significant throughput wins in targeted benchmarks. We expect that write-heavy workloads which generate a large number of intents scattered over ranges to benefit most from this change. The following benchmarks were run using kv0 with a batch size of 10 and secondary indices enabled (see command). On both 4- and 32-core nodes a ~30% throughput increase is observed. ``` ./workload run kv '{pgurl:1-3}' --init --splits=256 --duration 90s --batch 10 --secondary-index --read-percent=0 --concurrency=512 ``` 8-Core ``` name old ops/s new ops/s delta KV0 510 ± 3% 651 ± 7% +27.58% (p=0.008 n=5+5) ``` 32-core ``` name old ops/s new ops/s delta KV0 1.01k ± 3% 1.32k ± 1% +30.78% (p=0.008 n=5+5) ``` No change in performance for TPCC was observed: 8-Core (400 warehouses) ``` _elapsed_______tpmC____efc__avg(ms)__p50(ms)__p90(ms)__p95(ms)__p99(ms)_pMax(ms) 300.0s 6073.2 94.5% 180.0 167.8 268.4 318.8 402.7 2013.3 name old ops/s new ops/s delta tpmc 6.09k ± 1% 6.08k ± 0% ~ (p=0.886 n=4+4) ``` 32-Core (1500 warehouses) ``` _elapsed_______tpmC____efc__avg(ms)__p50(ms)__p90(ms)__p95(ms)__p99(ms)_pMax(ms) 300.0s 19303.0 93.8% 342.5 335.5 453.0 520.1 704.6 3623.9 name old ops/s new ops/s delta tpmc 19.3k ± 0% 19.3k ± 0% ~ (p=1.000 n=4+4) ``` Highly Contended Workload: The worst case workload for this change ought to be one that has contention on transactions but not too much contention that the increase in the contention footprint caused by this change gets caught in the noise. To evaluate this workload we run the kv0 benchmark with a secondary index and a short cycle length. With a very short cycle length the change shows no change to a modest win. Surprisingly with a longer cycle length such that the expected contention is lower it shows a real win in throughput and tail latency with 32 core, cycle length 16, 256 splits, concurrency 64 ``` name old ops/s new ops/s delta KV0 472 ± 3% 483 ± 1% +2.35% (p=0.016 n=5+5) ``` 32 core, cycle length 16, splits 512, concurrency 256 ``` name old ops/s new ops/s delta KV0 487 ± 2% 491 ± 2% ~ (p=0.841 n=5+5) ``` 8 core, cycle length 128, splits 256, concurrency 256 ``` name old ops/s new ops/s delta KV0 201 ± 7% 356 ± 7% +77.08% (p=0.016 n=5+4) ``` Fixes #30780 Release note (performance improvement): Increase write throughput for workloads which write large numbers of intents by coalescing intent resolution requests across transactions. Co-authored-by: Andrew Werner <[email protected]>
Build succeeded |
This PR adds a configuration option for a timeout for batch request sends in the request batcher and adopts this option in the intent resolver. Prior to the introduction of batching in cockroachdb#34803 intent resolution was performed with a maximum timeout of 30s. This PR uses that same constant and additionally applies it to batches of requests to gc transaction records. Informs cockroachdb#36806. Release note: None
This PR adds a configuration option for a timeout for batch request sends in the request batcher and adopts this option in the intent resolver. Prior to the introduction of batching in cockroachdb#34803 intent resolution was performed with a maximum timeout of 30s. This PR uses that same constant and additionally applies it to batches of requests to gc transaction records. Informs cockroachdb#36806. Release note: None
36845: requestbatcher,intentresolver: add timeout in requestbatcher and adopt r=ajwerner a=ajwerner This PR adds a configuration option for a timeout for batch request sends in the request batcher and adopts this option in the intent resolver. Prior to the introduction of batching in #34803 intent resolution was performed with a maximum timeout of 30s. This PR uses that same constant and additionally applies it to batches of requests to gc transaction records. Informs #36806. Release note: None Co-authored-by: Andrew Werner <[email protected]>
Fixes cockroachdb#46752. Resolves the recent perf regression on TPC-C. This commit follows in the footsteps of cockroachdb#34803 and introduces batching for ranged intent resolution, where previously only point intent resolution was batched. As we found in cockroachdb#46752, this is more important than it has been in the past, because implicit SELECT FOR UPDATE acquires a ranged lock on each row that it updates instead of a single-key lock. The change addresses this by adding a third request batcher to IntentResolver. ResolveIntent requests and ResolveIntentRange requests are batched separately, which is necessary for the use of MaxSpanRequestKeys to work properly (in fact, to be accepted by DistSender at all). To accommodate the ranged nature of ResolveIntentRange request, the notion of pagination is introduced into RequestBatcher. A MaxKeysPerBatchReq option is added to the configuration of a RequestBatcher, which corresponds to the MaxSpanRequestKeys value set on each BatchRequest header. The RequestBatcher is then taught about request pagination and how to work with partial responses. See the previous commit for clarification about the semantics at play here. Release justification: important fix to avoid a performance regression when implicit SELECT FOR UPDATE is enabled.
Fixes cockroachdb#46752. Resolves the recent perf regression on TPC-C. This commit follows in the footsteps of cockroachdb#34803 and introduces batching for ranged intent resolution, where previously only point intent resolution was batched. As we found in cockroachdb#46752, this is more important than it has been in the past, because implicit SELECT FOR UPDATE acquires a ranged lock on each row that it updates instead of a single-key lock. The change addresses this by adding a third request batcher to IntentResolver. ResolveIntent requests and ResolveIntentRange requests are batched separately, which is necessary for the use of MaxSpanRequestKeys to work properly (in fact, to be accepted by DistSender at all). To accommodate the ranged nature of ResolveIntentRange request, the notion of pagination is introduced into RequestBatcher. A MaxKeysPerBatchReq option is added to the configuration of a RequestBatcher, which corresponds to the MaxSpanRequestKeys value set on each BatchRequest header. The RequestBatcher is then taught about request pagination and how to work with partial responses. See the previous commit for clarification about the semantics at play here. Release justification: important fix to avoid a performance regression when implicit SELECT FOR UPDATE is enabled.
46952: kv: batch ranged intent resolution r=nvanbenschoten a=nvanbenschoten Fixes #46752. Resolves the recent perf regression on TPC-C. This commit follows in the footsteps of #34803 and introduces batching for ranged intent resolution, where previously only point intent resolution was batched. As we found in #46752, this is more important than it has been in the past, because implicit SELECT FOR UPDATE acquires a ranged lock on each row that it updates instead of a single-key lock. The change addresses this by adding a third request batcher to IntentResolver. ResolveIntent requests and ResolveIntentRange requests are batched separately, which is necessary for the use of MaxSpanRequestKeys to work properly (in fact, to be accepted by DistSender at all). To accommodate the ranged nature of ResolveIntentRange request, the notion of pagination is introduced into RequestBatcher. A MaxKeysPerBatchReq option is added to the configuration of a RequestBatcher, which corresponds to the MaxSpanRequestKeys value set on each BatchRequest header. The RequestBatcher is then taught about request pagination and how to work with partial responses. The semantics at play here are clarified in an earlier commit in the PR. Release justification: important fix to avoid a performance regression when implicit SELECT FOR UPDATE is enabled. All commits except the last are testing-only. The last commit is subtle but small and well-tested. @andreimatei: I assigned you because I think you know the most about `MaxSpanRequestKeys`. I'm mostly interested to get your input on the "rationalize Header.MaxSpanRequestKeys" commit (testing + comments only). Co-authored-by: Nathan VanBenschoten <[email protected]>
Fixes cockroachdb#46752. Resolves the recent perf regression on TPC-C. This commit follows in the footsteps of cockroachdb#34803 and introduces batching for ranged intent resolution, where previously only point intent resolution was batched. As we found in cockroachdb#46752, this is more important than it has been in the past, because implicit SELECT FOR UPDATE acquires a ranged lock on each row that it updates instead of a single-key lock. The change addresses this by adding a third request batcher to IntentResolver. ResolveIntent requests and ResolveIntentRange requests are batched separately, which is necessary for the use of MaxSpanRequestKeys to work properly (in fact, to be accepted by DistSender at all). To accommodate the ranged nature of ResolveIntentRange request, the notion of pagination is introduced into RequestBatcher. A MaxKeysPerBatchReq option is added to the configuration of a RequestBatcher, which corresponds to the MaxSpanRequestKeys value set on each BatchRequest header. The RequestBatcher is then taught about request pagination and how to work with partial responses. See the previous commit for clarification about the semantics at play here. Release justification: important fix to avoid a performance regression when implicit SELECT FOR UPDATE is enabled.
This PR modifies the IntentResolver to batch intent resolution across different
transactions on a (best effort) per range basis. It acheives this batching by
plumbing RangeDescriptorCache into the IntentResolver so that an intent-range
mapping can be determined and then using the RequestBatcher introduced in
concurrently without requiring additional goroutines this change extends the
interface to allow the client to provide the response channel.
Performance Wins
The change yields significant throughput wins in targeted benchmarks. We expect
that write-heavy workloads which generate a large number of intents scattered
over ranges to benefit most from this change. The following benchmarks were run
using kv0 with a batch size of 10 and secondary indices enabled (see command).
On both 4- and 32-core nodes a ~30% throughput increase is observed.
8-Core
32-core
No change in performance for TPCC was observed:
8-Core (400 warehouses)
32-Core (1500 warehouses)
Highly Contended Workload:
The worst case workload for this change ought to be one that has contention on transactions but not too much contention that the increase in the contention footprint caused by this change gets caught in the noise. To evaluate this workload we run the kv0 benchmark with a secondary index and a short cycle length. With a very short cycle length the change shows no change to a modest win. Surprisingly with a longer cycle length such that the expected contention is lower it shows a real win in throughput and tail latency with
32 core, cycle length 16, 256 splits, concurrency 64
32 core, cycle length 16, splits 512, concurrency 256
8 core, cycle length 128, splits 256, concurrency 256
Fixes #30780
Release note (performance improvement): Increase write throughput for workloads
which write large numbers of intents by coalescing intent resolution requests
across transactions.