-
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 bulkio.column_backfill.update_chunk_size_threshold_bytes #83544
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -275,6 +275,7 @@ func (cb *ColumnBackfiller) RunColumnBackfillChunk( | |
tableDesc catalog.TableDescriptor, | ||
sp roachpb.Span, | ||
chunkSize rowinfra.RowLimit, | ||
updateChunkSizeThresholdBytes rowinfra.BytesLimit, | ||
alsoCommit bool, | ||
traceKV bool, | ||
) (roachpb.Key, error) { | ||
|
@@ -390,6 +391,20 @@ func (cb *ColumnBackfiller) RunColumnBackfillChunk( | |
); err != nil { | ||
return roachpb.Key{}, err | ||
} | ||
|
||
// Exit early to flush if the batch byte size exceeds a predefined | ||
// threshold. This can happen when table rows are more on the "fat" side, | ||
// typically with large BYTES or JSONB columns. | ||
// | ||
// This helps prevent exceedingly large raft commands which will | ||
// for instance cause schema changes to be unable to either proceed or to | ||
// roll back. | ||
// | ||
// The threshold is ignored when zero. | ||
// | ||
if updateChunkSizeThresholdBytes > 0 && b.ApproximateMutationBytes() >= int(updateChunkSizeThresholdBytes) { | ||
break | ||
} | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I found that a threshold was more convenient than an absolute max size, both because it was easier to implement and because then I wouldn't have to worry about batches with zero puts, in the case where one put is just that big. What do we do then? not update the row? It didn't make sense, so I used a threshold. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. sounds good to me There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. okay, so, in addition to the old 64 MiB, you put this inside the for loop to say that if the to-be-put batch exceeds the threshold then break and flush, right? There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Almost, it's in addition to the number-of-rows limit (default 200). The 64 MiB limit we've changed in the customer support issues doesn't come into play here, that's a hard limit on the size of raft commands, which if exceeded causes the schema change to be blocked. This new setting I'm adding helps ensure we never reach that hard limit. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. yeah, my mistake, I meant the 200-row limit. Thanks for your explanation! |
||
} | ||
// Write the new row values. | ||
writeBatch := txn.Run | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -2196,6 +2196,66 @@ SELECT * FROM multipleinstmt ORDER BY id ASC; | |
2 b b false NULL true NULL | ||
3 c c false NULL true NULL | ||
|
||
subtest column_backfiller_update_batching | ||
|
||
let $use_decl_sc | ||
SHOW use_declarative_schema_changer | ||
|
||
statement ok | ||
SET use_declarative_schema_changer = 'off'; | ||
|
||
statement ok | ||
BEGIN; | ||
CREATE TABLE tb AS SELECT 123::INT AS k FROM generate_series(1, 10); | ||
SET tracing = on,kv; | ||
ALTER TABLE tb ADD COLUMN v STRING NOT NULL DEFAULT ('abc'::STRING); | ||
SET tracing = off; | ||
|
||
# Check that the column backfiller batches all its Puts into one batch. | ||
query I | ||
SELECT count(*) FROM [SHOW KV TRACE FOR SESSION] WHERE message LIKE '%sending batch%' AND message LIKE '% Put to %'; | ||
---- | ||
1 | ||
|
||
query I | ||
SELECT count(*) FROM tb WHERE v = 'abc'; | ||
---- | ||
10 | ||
|
||
statement ok | ||
ROLLBACK; | ||
|
||
# Bring the threshold way down to force column backfiller batches to have no more 1 Put each. | ||
statement ok | ||
SET CLUSTER SETTING bulkio.column_backfill.update_chunk_size_threshold_bytes = 1; | ||
|
||
statement ok | ||
BEGIN; | ||
CREATE TABLE tb AS SELECT 123::INT AS k FROM generate_series(1, 10); | ||
SET tracing = on,kv; | ||
ALTER TABLE tb ADD COLUMN v STRING NOT NULL DEFAULT ('abc'::STRING); | ||
SET tracing = off; | ||
|
||
query I | ||
SELECT count(*) FROM [SHOW KV TRACE FOR SESSION] WHERE message LIKE '%sending batch%' AND message LIKE '% Put to %'; | ||
---- | ||
10 | ||
|
||
query I | ||
SELECT count(*) FROM tb WHERE v = 'abc'; | ||
---- | ||
10 | ||
|
||
statement ok | ||
ROLLBACK; | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. This subtest only exercises in-txn column backfills, I couldn't figure out how to retrieve the traces in the other case. Those traces are emitted, according to the debugger at least, but I couldn't collect them. Anyway I figured that this would be enough. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Yeah, I like the tests you added as well. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Can we also have a "correctness" test? Namely, if set this threshold to be small and add a column, we expect it to use >1 batch and succeed by being able to observe the newly added column with the correct default value. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I'm not sure it's that valuable, we already arbitrarily break down batches in 200-row increments, but it also doesn't cost anything, so I'll add a SELECT. |
||
|
||
# Undo subtest side effects. | ||
statement ok | ||
RESET CLUSTER SETTING bulkio.column_backfill.update_chunk_size_threshold_bytes; | ||
|
||
statement ok | ||
SET use_declarative_schema_changer = $use_decl_sc; | ||
|
||
subtest storage_params | ||
|
||
statement ok | ||
|
@@ -2306,8 +2366,8 @@ FROM ( | |
LEFT JOIN pg_catalog.pg_depend r ON l.table_id = r.objid; | ||
---- | ||
table_id name state refobjid | ||
205 test_serial_b_seq PUBLIC 204 | ||
204 test_serial PUBLIC NULL | ||
207 test_serial_b_seq PUBLIC 206 | ||
206 test_serial PUBLIC NULL | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. These and the others all get bumped because my subtest non-transactionally increments the descriptor ID counter twice. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. why don't we put your subtest at the end of the file? There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. In a sane world I would, however, doing so often causes annoying merge conflicts when backporting. |
||
|
||
statement ok | ||
DROP TABLE test_serial; | ||
|
@@ -2341,8 +2401,8 @@ FROM ( | |
LEFT JOIN pg_catalog.pg_depend r ON l.table_id = r.objid; | ||
---- | ||
table_id name state refobjid | ||
207 test_serial_b_seq PUBLIC 206 | ||
206 test_serial PUBLIC NULL | ||
209 test_serial_b_seq PUBLIC 208 | ||
208 test_serial PUBLIC NULL | ||
|
||
statement ok | ||
ALTER TABLE test_serial DROP COLUMN b; | ||
|
@@ -2357,7 +2417,7 @@ FROM ( | |
LEFT JOIN pg_catalog.pg_depend r ON l.table_id = r.objid; | ||
---- | ||
table_id name state refobjid | ||
206 test_serial PUBLIC NULL | ||
208 test_serial PUBLIC NULL | ||
|
||
statement ok | ||
DROP TABLE test_serial; | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -103,7 +103,11 @@ func (cb *columnBackfiller) CurrentBufferFill() float32 { | |
|
||
// runChunk implements the chunkBackfiller interface. | ||
func (cb *columnBackfiller) runChunk( | ||
ctx context.Context, sp roachpb.Span, chunkSize rowinfra.RowLimit, _ hlc.Timestamp, | ||
ctx context.Context, | ||
sp roachpb.Span, | ||
chunkSize rowinfra.RowLimit, | ||
updateChunkSizeThresholdBytes rowinfra.BytesLimit, | ||
_ hlc.Timestamp, | ||
) (roachpb.Key, error) { | ||
var key roachpb.Key | ||
var commitWaitFn func(context.Context) error | ||
|
@@ -124,16 +128,16 @@ func (cb *columnBackfiller) runChunk( | |
// waiting for consistency when backfilling a column on GLOBAL tables. | ||
commitWaitFn = txn.DeferCommitWait(ctx) | ||
|
||
// TODO(knz): do KV tracing in DistSQL processors. | ||
var err error | ||
key, err = cb.RunColumnBackfillChunk( | ||
ctx, | ||
txn, | ||
cb.desc, | ||
sp, | ||
chunkSize, | ||
true, /*alsoCommit*/ | ||
false, /*traceKV*/ | ||
updateChunkSizeThresholdBytes, | ||
true, /*alsoCommit*/ | ||
cb.flowCtx.TraceKV, | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I figure I'd leave this change in, as the debugger assures me that this flag is set when |
||
) | ||
return err | ||
}) | ||
|
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.
I took this default value because that's also the default max size when scanning.