-
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
sql: add sql.mutations.mutation_batch_byte_size setting #67537
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.
Nice, thanks!
In terms of testing, I think we usually use tracing to confirm the KV level operations. I'd probably look at execbuilder/testdata/show_trace_nonmetamorphic
and execbuilder/testdata/autocommit_nonmetamorphic
for inspiration.
Reviewed 11 of 11 files at r1.
Reviewable status: complete! 0 of 0 LGTMs obtained (waiting on @dt and @nvanbenschoten)
pkg/sql/tablewriter.go, line 115 at r1 (raw file):
// a different value in tests. maxBatchSize int // maxBatchSize determines the maximum number of key and value bytes in the KV
nit: s/maxBatchSize/maxBatchByteSize/
.
630a853
to
5f08c41
Compare
It seems like there are still some cases where the eval'ed |
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.
Thanks, the trace tests indeed look like they do what I want. Updated with a test case
Reviewable status: complete! 0 of 0 LGTMs obtained (waiting on @nvanbenschoten and @yuzefovich)
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.
There is also the insert fast path (sql/insert_fast_path.go) which doesn't do any batching. It is used when we insert from a VALUES with less than MaxBatchSize rows:
We'd need to change it to also do batching internally. (Or maybe check the values and only allow the fast path if they are not big).
Reviewable status: complete! 0 of 0 LGTMs obtained (waiting on @nvanbenschoten and @yuzefovich)
IMO a large I'm not saying that we shouldn't also batch a big VALUES clause, but I think even as is, this is better than nothing, and since this is fixing a bug that can blocking some RESTOREs, I was trying to minimize the behavior change to keep it backport-safe if I can. |
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 6 of 15 files at r2, 1 of 1 files at r4.
Reviewable status: complete! 0 of 0 LGTMs obtained (waiting on @dt, @nvanbenschoten, and @yuzefovich)
pkg/sql/opt/exec/execbuilder/testdata/show_trace_nonmetamorphic, line 400 at r4 (raw file):
query TT SELECT operation, message FROM [SHOW KV TRACE FOR SESSION] WHERE message LIKE '%r$rangeid: sending batch%'
Hm, I'm a bit confused - I would think that new blobs
table would have the next range id and we needed to update $rangeid
logic test variable, do you know why that's not the case?
I donno, it looks like they're both in range 40.
|
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.
Oh well, nvm then. but might be worth for Nathan to take a quick look at pkg/kv
change.
Reviewable status: complete! 1 of 0 LGTMs obtained (waiting on @nvanbenschoten and @yuzefovich)
@nvanbenschoten I think in offline convo you indicated you were okay with the idea of adding some tracking to the kv client |
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 6 of 15 files at r2, 1 of 1 files at r4.
Reviewable status: complete! 2 of 0 LGTMs obtained (waiting on @dt and @yuzefovich)
pkg/kv/batch.go, line 56 at r4 (raw file):
AdmissionHeader roachpb.AdmissionHeader reqs []roachpb.RequestUnion mutationReqBytes int
Let's name this approxMutationReqBytes
and give it a comment similar to what you have below.
pkg/kv/batch.go, line 495 at r4 (raw file):
b.appendReqs(roachpb.NewConditionalPut(k, v, expValue, allowNotExist)) } b.mutationReqBytes += len(k) + len(expValue) + len(v.RawBytes)
I don't think we need to include expValue
in this estimate. We won't be writing the expValue
.
pkg/sql/tablewriter.go, line 132 at r4 (raw file):
} var maxBatchBytes = settings.RegisterByteSizeSetting(
Should this be pushed into the mutations
package? It's strange to me that the setting lives here but the metamorphic const lives elsewhere.
Also, if we were going to revive the cluster setting for MaxBatchSize
, where should that live. Maybe a question for @yuzefovich.
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! 2 of 0 LGTMs obtained (waiting on @dt, @nvanbenschoten, and @yuzefovich)
pkg/sql/tablewriter.go, line 132 at r4 (raw file):
Previously, nvanbenschoten (Nathan VanBenschoten) wrote…
Should this be pushed into the
mutations
package? It's strange to me that the setting lives here but the metamorphic const lives elsewhere.Also, if we were going to revive the cluster setting for
MaxBatchSize
, where should that live. Maybe a question for @yuzefovich.
Personally, I don't have a strong preference about the placement of these things, either seems reasonable to me.
The original sql.testing.mutations.max_batch_size
setting lived in mutations
package. Back then we also performed the "resolution" of the value in FlowBase.Setup
which was updating a global value, and I actually prefer David's approach in tableWriterBase.init
.
@dt how far back do we think this is back portable (if at all)? Asking because the customer that prompted this fix is looking to understand what we are doing to fix the issue. |
I was vaguely thinking I'd like to backport to 20.2, but I'll defer to SQL-Queries TLs on backport call and if so, how far back |
Previously we always constructed 10k row insert batches, regardless of the size of those rows. With large rows, this could easily exceed the kv size limit of 64MB. This changes batch construction to track the size of added keys and values, and send the batch either when it has 10k entries of when the size of added keys and values exceeds the setting, which defaults to 4MB. Release note (bug fix): INSERT and UPDATE statements which operate on larger rows are split into batches using the sql.mutations.mutation_batch_byte_size setting
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 2 stale) (waiting on @nvanbenschoten and @yuzefovich)
pkg/kv/batch.go, line 495 at r4 (raw file):
Previously, nvanbenschoten (Nathan VanBenschoten) wrote…
I don't think we need to include
expValue
in this estimate. We won't be writing theexpValue
.
Ah, right, I guess with prop-eval'ed KV we don't put the actual cput in the replicated batch, just the result
pkg/sql/tablewriter.go, line 132 at r4 (raw file):
Previously, yuzefovich (Yahor Yuzefovich) wrote…
Personally, I don't have a strong preference about the placement of these things, either seems reasonable to me.
The original
sql.testing.mutations.max_batch_size
setting lived inmutations
package. Back then we also performed the "resolution" of the value inFlowBase.Setup
which was updating a global value, and I actually prefer David's approach intableWriterBase.init
.
So leave it as-is?
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 r5.
Reviewable status: complete! 0 of 0 LGTMs obtained (and 2 stale) (waiting on @dt, @nvanbenschoten, and @yuzefovich)
pkg/sql/tablewriter.go, line 132 at r4 (raw file):
Previously, dt (David Taylor) wrote…
So leave it as-is?
Yeah, sgtm.
TFTRs! bors r+ |
Build succeeded: |
@yuzefovich How do you feel about 21.1 and 20.2 backports here? This came up because |
I think it is justifiable for a backport to both branches. One thought there is whether we want to backport it in such a manner that by default the behavior doesn't change? I mean we would essentially set the default value of the new cluster setting to zero, and we'd be ignoring the setting unless it was changed to something greater than zero. This way we'd have a safe workaround for users who run into "command is too large" problems when restoring. I guess I'm a bit worried about this change having some performance implications, and merging it to master seems ok to me given that we have plenty of time to - hopefully - catch perf regressions if they occur, but 2 weeks baking period might not be enough. E.g. if we want to insert 10k rows 5KB each, previously we would have a single batch with 50MB worth of mutations, and now we will have 13 batches - will 13 batches be slower? One the other hand, though, I think we do recommend "pagination" of mutations on the app side, so maybe this example is an edge case and can be reasonably ignored. |
one wrinkle with requiring a cluster setting to opt into pagination to avoid hitting the limit and failing is that settings themselves are also restored during a RESTORE, so RESTORE pretty much always wants its |
I see. Ok, then I'll run some quick benchmarks to see whether my concern is valid. |
Hm, in my quick sanity check on the gceworker multiple smaller batches are faster for INSERT than a large single batch, so my concerns don't seem to be valid. I'll open up the backports and will merge after the 2 weeks baking period. |
@yuzefovich hm, yes, this was driven by a customer who has indeed asked for followup fixes and is on 20.2, but we could see if there is room to push back since it isn't clean and just point to the progress in 21.1. If we do want to fix on 20.2, maybe better to just add the new setting / size limit in a more or less fresh change, rather than try to backport this one on top of the #57483 changes? |
I agree, I'm currently doing some plumbing, and the backport isn't as clean, but the amount of additional changes is limited. My initial thought of having to cherry-pick things from #57483 was incorrect. |
Previously we always constructed 10k row insert batches, regardless of the
size of those rows. With large rows, this could easily exceed the kv size
limit of 64MB. This changes batch construction to track the size of added
keys and values, and send the batch either when it has 10k entries of when
the size of added keys and values exceeds the setting, which defaults to 4MB.
Fixes #67542.
Release note (bug fix): INSERT and UPDATE statements which operate on larger rows are split into batches using the sql.mutations.mutation_batch_byte_size setting