-
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 txn rows written/read guardrails #69202
Conversation
75eb1c9
to
7ccc66c
Compare
It is missing tests, but otherwise RFAL, cc @michae2 |
Looking! 😄 |
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.
Thank you so much for doing this. It is looking great!
Reviewed 1 of 31 files at r6.
Reviewable status: complete! 0 of 0 LGTMs obtained (waiting on @yuzefovich)
pkg/util/log/eventpb/sql_audit_events.proto, line 129 at r6 (raw file):
string txn_id = 1 [(gogoproto.customname) = "TxnID", (gogoproto.jsontag) = ",omitempty"]; string session_id = 2 [(gogoproto.customname) = "SessionID", (gogoproto.jsontag) = ",omitempty"];
I think we need to add
(gogoproto.moretags) = "redact:\"nonsensitive\""
to these fields to prevent them from being redacted.
Hmm. These fields alone don't seem like enough information for a user to figure out which of their transactions violated the limit. I think we should include CommonSQLEventDetails
in the four Txn.*LimitReached
messages below. (An example of filling it is in logEventsWithOptions
in pkg/sql/event_log.go
.)
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.
Added some tests, but I think we need a bit more for the INTERNAL_PERF channel. Also, it's still missing the metrics increment whenever these new events occur (which the guardrail spec mention).
Reviewable status: complete! 0 of 0 LGTMs obtained (waiting on @michae2)
pkg/util/log/eventpb/sql_audit_events.proto, line 129 at r6 (raw file):
Previously, michae2 (Michael Erickson) wrote…
string txn_id = 1 [(gogoproto.customname) = "TxnID", (gogoproto.jsontag) = ",omitempty"]; string session_id = 2 [(gogoproto.customname) = "SessionID", (gogoproto.jsontag) = ",omitempty"];
I think we need to add
(gogoproto.moretags) = "redact:\"nonsensitive\""
to these fields to prevent them from being redacted.
Hmm. These fields alone don't seem like enough information for a user to figure out which of their transactions violated the limit. I think we should include
CommonSQLEventDetails
in the fourTxn.*LimitReached
messages below. (An example of filling it is inlogEventsWithOptions
inpkg/sql/event_log.go
.)
Good point, done.
Okay, I added the metrics, and I think it is RFAL. However, I realized that in many cases we won't log the internal queries when they reach the limits because we're using the default uninitialized session data in the callsites where we're initiating those internal queries. That seems ok to me, but curious to hear your thoughts. cc @vy-ton @michae2 |
When the limit is enforced with Separate note: For the |
No, the internal queries would be "invisible" for these guardrails (I mentioned only "logging" because the internal queries cannot error out - if either
Good point, I'll update accordingly. |
This sounds ok to me and would mirror how we're trying to make |
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 looking good. Just a few nits from me.
Reviewed 35 of 36 files at r11, all commit messages.
Reviewable status: complete! 0 of 0 LGTMs obtained (waiting on @adityamaru and @yuzefovich)
-- commits, line 120 at r11:
nit: Looks like a stray backtick here.
pkg/sql/conn_executor_exec.go, line 1099 at r11 (raw file):
} }
Would it be possible to factor this repeated logic into a function or closure that is called twice? That will make our lives a little easier if we ever have to modify this logic in the future.
pkg/sql/event_log_test.go, line 605 at r11 (raw file):
// variables to the values of the cluster settings just set. setup: ` SET CLUSTER SETTING sql.defaults.transaction_rows_written_log = 0;
nit: maybe = DEFAULT
would be more future-proof.
pkg/sql/exec_util.go, line 953 at r11 (raw file):
} MetaTxnRowsWrittenLog = metric.Metadata{ Name: "sql.guardrails.txn_rows_written_log.count",
nit: I think it will be confusing if the metric names differ from the variable names (txn_
vs transaction_
). It will be easier to remember if they are the same.
pkg/util/log/eventpb/sql_audit_events.proto, line 133 at r11 (raw file):
// ReadKind indicates that the "rows read" limit is reached if true and // the "rows written" limit otherwise. bool read_kind = 4 [(gogoproto.jsontag) = ",omitempty"];
nit: It would be nice if there were a jsontag to always omit, since this is redundant with the containing message types, and only really necessary when using it as an error. But I am having trouble finding documentation for gogoproto jsontags, so I'm not sure if this exists.
pkg/util/log/eventpb/sql_audit_events.proto, line 141 at r11 (raw file):
// even if there are more mutation statement within the transaction that haven't // been executed yet. message TxnRowsWrittenLimitReached {
nit: You might consider dropping Reached
(though this is fine too).
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 @adityamaru and @yuzefovich)
pkg/util/log/eventpb/sql_audit_events.proto, line 133 at r11 (raw file):
Previously, michae2 (Michael Erickson) wrote…
nit: It would be nice if there were a jsontag to always omit, since this is redundant with the containing message types, and only really necessary when using it as an error. But I am having trouble finding documentation for gogoproto jsontags, so I'm not sure if this exists.
Never mind, it looks like pkg/util/log/eventpb/gen.go
is generating the JSON-building code and only looks at a few specific jsontags.
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.
Yeah, it seems to me that if we want to support the auto commit in the general case when the erring guardrails are enabled, then we'd either have to introduce some coordinator in the SQL layer (through which all KV reads and writes go before reaching the KV layer) or to teach the KV layer about the guardrails. Neither of these options seem easy and clean and definitely not backportable.
We could, however, make the limitation a bit more loose. Namely, I'm thinking that if a statement contains only a single mutation, then at the point where we perform the auto-commit of the KV batch, we could check whether rowsWritten
has reached the transaction_rows_written_err
limit. It is a bit more difficult to check the "rows read", but for some mutations we know that they don't read any rows. I also think that the customer who is driving this is more interested in the rows written guardrail, so this might be an acceptable compromise. I'll try prototyping this idea to see how it looks.
Reviewable status: complete! 0 of 0 LGTMs obtained (and 2 stale) (waiting on @rytaft and @vy-ton)
Previously, michae2 (Michael Erickson) wrote…
Good catch, @yuzefovich. Hmm, does this mean that after an autocommit statement like
INSERT INTO foo VALUES (1);
there will be a dangling transaction left open in the session? Or is this autocommit option in execbuilder an optimization?
The autocommit option in the execbuilder is an optimization. It can only be enabled for implicit transactions when mutations satisfy some requirements (like no secondary indexes or something like that). This option means that we commit the KV txn as part of the last KV batch we have constructed to perform the mutation; without the auto commit the txn is committed by the connExecutor and will require a new call to the KV layer. I think that call might be an RPC to the remote node that is the leaseholder of the range on which the txn is anchored (I might be using an incorrect terminology here, or just can be wrong), so it's a really nice optimization to have.
pkg/sql/opt/exec/execbuilder/testdata/autocommit_nonmetamorphic, line 792 at r18 (raw file):
Previously, rytaft (Rebecca Taft) wrote…
is it worth adding another test case to confirm that this returns true when the guardrail is disabled? Maybe also a case to confirm that we can auto commit when the logging guardrail is enabled.
Done.
139fd0c
to
213c27e
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.
Actually, execbuilder.Builder.canAutoCommit
already ensures that there is at most one mutation in the query, so supporting the auto commit when the erring rows written guardrail is enabled seems quite easy. I've just added a second WIP commit that implements that logic (which I'll squash into the previous one if we decide to proceed with the idea). Let me know what you think about that WIP commit - I think it is pretty clean and seems like a nice win, so I like it.
I also added another commit (which is now first) which fixes the tracking of rows written/read.
Reviewable status: complete! 0 of 0 LGTMs obtained (and 2 stale) (waiting on @michae2, @rytaft, and @vy-ton)
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.
WIP commit looks fine to me. Good idea.
Reviewable status: complete! 0 of 0 LGTMs obtained (and 2 stale) (waiting on @michae2, @rytaft, @vy-ton, and @yuzefovich)
or CREATE STATISTICS statements. Note that enabling these guardrails comes at the cost of disabling the usage of the auto commit optimization for the mutation statements in implicit transactions.
(I'm sure you were going to do this, but just a reminder) with your WIP commit in place, this only applies to transaction_rows_read_err
, so best to say that specifically. Also I suggest instead of "the auto commit optimization" maybe "an optimization for auto-commit mutation statements".
pkg/sql/conn_executor_exec.go, line 1125 at r24 (raw file):
commonTxnRowsLimitDetails.Limit = errLimit err = pgerror.WithCandidateCode(&commonTxnRowsLimitDetails, pgcode.ProgramLimitExceeded) errCounter.Inc(1)
One thought I had tonight: we should move the counter increments up near the top, before we change shouldLog
or shouldErr
, so that the counts are not influenced by any of the "safety" logic turning off errors for internal queries, etc.
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! Looks like a good improvement.
Reviewed 32 of 39 files at r24, 10 of 10 files at r25, all commit messages.
Reviewable status: complete! 0 of 0 LGTMs obtained (and 2 stale) (waiting on @rytaft and @yuzefovich)
pkg/sql/tablewriter.go, line 198 at r25 (raw file):
// to ensure that only a single mutation operator is present in the plan. func (tb *tableWriterBase) finalize( ctx context.Context, rowsWritten, rowsWrittenLimit int64,
to avoid these extra params, do you think it makes sense to make them data members of tableWriterBase?
pkg/sql/tablewriter.go, line 202 at r25 (raw file):
// NB: unlike flushAndStartNewBatch, we don't bother with admission control // for response processing when finalizing. if tb.autoCommit == autoCommitEnabled && (rowsWrittenLimit == 0 || rowsWritten < rowsWrittenLimit) {
nit: should it be rowsWritten < rowsWrittenLimit
or rowsWritten <= rowsWrittenLimit
? (I forget what you used elsewhere, but we should also make those consistent)
9f38f40
to
2b5e671
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 2 stale) (waiting on @michae2 and @rytaft)
Previously, michae2 (Michael Erickson) wrote…
or CREATE STATISTICS statements. Note that enabling these guardrails comes at the cost of disabling the usage of the auto commit optimization for the mutation statements in implicit transactions.
(I'm sure you were going to do this, but just a reminder) with your WIP commit in place, this only applies to
transaction_rows_read_err
, so best to say that specifically. Also I suggest instead of "the auto commit optimization" maybe "an optimization for auto-commit mutation statements".
Done.
pkg/sql/conn_executor_exec.go, line 1125 at r24 (raw file):
Previously, michae2 (Michael Erickson) wrote…
One thought I had tonight: we should move the counter increments up near the top, before we change
shouldLog
orshouldErr
, so that the counts are not influenced by any of the "safety" logic turning off errors for internal queries, etc.
Hm, I feel like this might be more confusing - I think the counters should represent the actual state of how the system behaves over time, and if we logged an event instead of an error because it was an internal query, I'd expect to see the log counter be incremented but the err counter stay the same.
pkg/sql/tablewriter.go, line 198 at r25 (raw file):
Previously, rytaft (Rebecca Taft) wrote…
to avoid these extra params, do you think it makes sense to make them data members of tableWriterBase?
Refactored.
pkg/sql/tablewriter.go, line 202 at r25 (raw file):
Previously, rytaft (Rebecca Taft) wrote…
nit: should it be
rowsWritten < rowsWrittenLimit
orrowsWritten <= rowsWrittenLimit
? (I forget what you used elsewhere, but we should also make those consistent)
It should be rowsWritten < rowsWrittenLimit
because if we have rowsWritten == rowsWrittenLimit
, then we will emit an error in the connExecutor, and, thus, we don't want to perform the auto commit. (That's why I'm using word "reach" when talking about these limits rather than "exceed".)
Originally, I was debating whether == limit
should trigger an error or not, but in case it doesn't, we can use the default value of 0
to disable the limit, otherwise we'd have to use a negative int which seems not pretty.
Previously, we were creating a new `topLevelQueryStats` object for each sub- and post-query, and as a result we would accumulate the top level query stats only for the main query. I think this is undesirable and is now fixed. Release note: None Release justification: low-risk bug fix.
This commit introduces the guardrails on the number of rows written/read by a single txn. The limits are enforced after each statement of a txn has been fully executed (i.e. we don't proactively cancel work in the middle of the execution if the txn has just reached the limits). This is done in the connExecutor since it is a very convenient place to enforce the limits so that they apply only to the desired statements. Notably, things for which we don't want the application of the limits (things like BACKUP, IMPORT, CREATE STATISTICS, etc) don't go through the connExecutor and, thus, aren't affected. The accumulation of the number of rows read by a txn has already been in place, and this commit introduces the explicit collection of the number of rows written via the same mechanism - by propagating "rows written" metrics during the draining of the execution flow. Initially, we considered using "rows affected" values, but those have a different meaning from what we want. This metrics collection required teaching the planNodeToRowSource adapter to ask the mutation planNodes for the number of rows written. Note that in many cases, the internal executor doesn't have the sessionData properly set (i.e. the default values are used), so we'll never log anything then. This seems acceptable since the focus of these guardrails is on the externally initiated queries. One notable change is because of our "after the fact" enforcement of these limits, this commit disables the auto commit option for the mutation statements in the execbuilder for non-internal statements if `transaction_rows_read_err` guardrail is enabled. Release note (ops change): New cluster settings `sql.defaults.transaction_rows_written_log`, `sql.defaults.transaction_rows_written_err`, `sql.defaults.transaction_rows_read_log`, and `sql.defaults.transaction_rows_read_err` (as well as the corresponding session variables have been introduced. These settings determine the "size" of the transactions in written and read rows upon reaching of which the transactions are logged or rejected. The logging will go into SQL_PERF logging channel. Note that the internal queries (i.e. those issued by CockroachDB internally) cannot error out but can be logged instead into SQL_INTERNAL_PERF logging channel. The "written" limits apply to INSERT, INSERT INTO SELECT FROM, INSERT ON CONFLICT, UPSERT, UPDATE, and DELETE whereas the "read" limits apply to SELECT statement in addition to all of these. These limits will not apply to CREATE TABLE AS SELECT, IMPORT, TRUNCATE, DROP, ALTER TABLE, BACKUP, RESTORE, or CREATE STATISTICS statements. Note that enabling `transaction_rows_read_err` guardrail comes at the cost of disabling the usage of the auto commit optimization for the mutation statements in implicit transactions. Release justification: low-risk, high benefit change to existing functionality.
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 @michae2 and @rytaft)
pkg/sql/tablewriter.go, line 202 at r25 (raw file):
Previously, yuzefovich (Yahor Yuzefovich) wrote…
It should be
rowsWritten < rowsWrittenLimit
because if we haverowsWritten == rowsWrittenLimit
, then we will emit an error in the connExecutor, and, thus, we don't want to perform the auto commit. (That's why I'm using word "reach" when talking about these limits rather than "exceed".)Originally, I was debating whether
== limit
should trigger an error or not, but in case it doesn't, we can use the default value of0
to disable the limit, otherwise we'd have to use a negative int which seems not pretty.
Mistake: the last sentence should have said:
Originally, I was debating whether == limit
should trigger an error or not, but in case it does (the current implementation), we can use the default value of 0 to disable the limit, otherwise we'd have to use a negative int which seems not pretty.
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! 1 of 0 LGTMs obtained (and 1 stale) (waiting on @rytaft and @yuzefovich)
Previously, yuzefovich (Yahor Yuzefovich) wrote…
The autocommit option in the execbuilder is an optimization. It can only be enabled for implicit transactions when mutations satisfy some requirements (like no secondary indexes or something like that). This option means that we commit the KV txn as part of the last KV batch we have constructed to perform the mutation; without the auto commit the txn is committed by the connExecutor and will require a new call to the KV layer. I think that call might be an RPC to the remote node that is the leaseholder of the range on which the txn is anchored (I might be using an incorrect terminology here, or just can be wrong), so it's a really nice optimization to have.
Thanks!
pkg/sql/conn_executor_exec.go, line 1125 at r24 (raw file):
Previously, yuzefovich (Yahor Yuzefovich) wrote…
Hm, I feel like this might be more confusing - I think the counters should represent the actual state of how the system behaves over time, and if we logged an event instead of an error because it was an internal query, I'd expect to see the log counter be incremented but the err counter stay the same.
Fair enough, ok, let's leave it as-is.
pkg/sql/tablewriter.go, line 202 at r25 (raw file):
Previously, yuzefovich (Yahor Yuzefovich) wrote…
Mistake: the last sentence should have said:
Originally, I was debating whether
== limit
should trigger an error or not, but in case it does (the current implementation), we can use the default value of 0 to disable the limit, otherwise we'd have to use a negative int which seems not pretty.
Good point! Hmm, I wonder if I should change the row size guardrail to match? (Currently row size == max_row_size
does not cause logging or error.) The low-number-cases seem less important for row size than for number of rows per trx, so maybe it doesn't matter as much for that guardrail?
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! 1 of 0 LGTMs obtained (and 1 stale) (waiting on @michae2 and @rytaft)
pkg/sql/tablewriter.go, line 202 at r25 (raw file):
Previously, michae2 (Michael Erickson) wrote…
Good point! Hmm, I wonder if I should change the row size guardrail to match? (Currently
row size == max_row_size
does not cause logging or error.) The low-number-cases seem less important for row size than for number of rows per trx, so maybe it doesn't matter as much for that guardrail?
I think I prefer the current implementation of the row size guardrail slightly - to me in general row size == max_row_size
should be allowed, and I agree that given that we expect for that limit be pretty large, the question whether equality triggers an error or not is not that important.
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 39 of 39 files at r29, all commit messages.
Reviewable status: complete! 2 of 0 LGTMs obtained (waiting on @rytaft)
Thanks for the reviews! bors r+ |
Build succeeded: |
This should be backported on top of #67953. |
Ah, I forgot that we should add #67400 to the commit message. Oh well. |
This commit introduces the guardrails on the number of rows written/read
by a single txn. The limits are enforced after each statement of a txn
has been fully executed (i.e. we don't proactively cancel work in the
middle of the execution if the txn has just reached the limits). This is
done in the connExecutor since it is a very convenient place to enforce
the limits so that they apply only to the desired statements. Notably,
things for which we don't want the application of the limits (things
like BACKUP, IMPORT, CREATE STATISTICS, etc) don't go through the
connExecutor and, thus, aren't affected.
The accumulation of the number of rows read by a txn has already been in
place, and this commit introduces the explicit collection of the number
of rows written via the same mechanism - by propagating "rows written"
metrics during the draining of the execution flow. Initially, we
considered using "rows affected" values, but those have a different
meaning from what we want. This metrics collection required teaching the
planNodeToRowSource adapter to ask the mutation planNodes for the number
of rows written.
Note that in many cases, the internal executor doesn't have the
sessionData properly set (i.e. the default values are used), so we'll
never log anything then. This seems acceptable since the focus of these
guardrails is on the externally initiated queries.
One notable change is because of our "after the fact" enforcement of
these limits, this commit disables the auto commit option for the
mutation statements in the execbuilder for non-internal statements if
transaction_rows_read_err
guardrail is enabled.Release note (ops change): New cluster settings
sql.defaults.transaction_rows_written_log
,sql.defaults.transaction_rows_written_err
,sql.defaults.transaction_rows_read_log
, andsql.defaults.transaction_rows_read_err
(as well as the correspondingsession variables have been introduced. These settings determine the
"size" of the transactions in written and read rows upon reaching of
which the transactions are logged or rejected. The logging will go into
SQL_PERF logging channel. Note that the internal queries (i.e. those
issued by CockroachDB internally) cannot error out but can be logged
instead into SQL_INTERNAL_PERF logging channel. The "written" limits
apply to INSERT, INSERT INTO SELECT FROM, INSERT ON CONFLICT, UPSERT,
UPDATE, and DELETE whereas the "read" limits apply to SELECT statement
in addition to all of these. These limits will not apply to CREATE
TABLE AS SELECT, IMPORT, TRUNCATE, DROP, ALTER TABLE, BACKUP, RESTORE,
or CREATE STATISTICS statements. Note that enabling
transaction_rows_read_err
guardrail comes at the cost of disablingthe usage of the auto commit optimization for the mutation statements
in implicit transactions.
Release justification: low-risk, high benefit change to existing
functionality.