-
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: fix portals after exhausting rows #48842
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.
Looks good, I think. Are there no other places where we could execute and exhaust a portal?
I'm concerned that the failures in teamcity could be related, though they look quite unrelated. Can you reproduce them locally?
pkg/sql/conn_executor_exec.go
Outdated
// change would forget to add the call). | ||
// | ||
// TODO(andrei): really the code should be re-architectued to ensure | ||
// TODO(andrei): really the code should be re-architectured to ensure |
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.
nit: re-architected* :)
pkg/sql/conn_executor.go
Outdated
case stateOpen: | ||
// We're about the execute the statement in an open state which | ||
// could trigger the dispatch to the execution engine. However, it | ||
// is possible that we're trying to execute already exhausted |
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.
nit: execute an already exhausted*
pkg/sql/conn_executor.go
Outdated
// We're about the execute the statement in an open state which | ||
// could trigger the dispatch to the execution engine. However, it | ||
// is possible that we're trying to execute already exhausted | ||
// portal - in such scenario we should return no rows, but the |
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.
nit: in such a scenario*
pkg/sql/conn_executor.go
Outdated
ev, payload, err = ex.execStmt(stmtCtx, curStmt, stmtRes, pinfo) | ||
switch ex.machine.CurState().(type) { | ||
case stateOpen: | ||
// We're about the execute the statement in an open state which |
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.
nit: we're about to execute
pkg/sql/conn_executor.go
Outdated
// nothing to return on an exhausted portal. | ||
if !portal.exhausted { | ||
ev, payload, err = ex.execStmt(stmtCtx, curStmt, stmtRes, pinfo) | ||
// The query execution has been complete (because we only |
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.
nit: has completed
I can reproduce failures locally, so it must be this change to blame, but I'm at a loss at what's wrong, I was hoping you guys would see the reason :) |
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! I have no insight re the logic test failures
Reviewable status: complete! 0 of 0 LGTMs obtained (waiting on @andreimatei, @mjibson, and @yuzefovich)
pkg/sql/conn_executor.go, line 1424 at r1 (raw file):
ev, payload, err = ex.execStmt(stmtCtx, curStmt, stmtRes, pinfo) // The query execution has been complete (because we only // supports execution of portals to completion) which means
nit: s/supports/support
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 @andreimatei, @mjibson, and @yuzefovich)
pkg/sql/conn_executor.go, line 1409 at r1 (raw file):
} stmtCtx := withStatement(ctx, ex.curStmt) switch ex.machine.CurState().(type) {
Have you putting the setting off the exhausted flag inside limitedCommandResult
? That's the guy that know whether a portal is exhausted or not.
The checking of whether a portal is exhausted I'd try to put in execStmtInOpenState()
so you don't have to inspect the state at this level. Maybe add an Exhausted()
method on the RestrictedCommandResult()
interface.
pkg/sql/conn_executor.go, line 1427 at r1 (raw file):
// that all rows from this portal have been returned - the // portal is now exhausted. portal.exhausted = true
Do we need some logic to reset this field when doing auto-retries? See prepStmtNamespace.resetTo()
. Please add a test showing that an auto-retry works in re-executing a portal.
I think before this patch portals were immutable so it was easy to share instances between the set of portals captured at the "rewind position" and the current set of portals. But if they're not immutable, more care I think is needed.
The failures are from some jobs tests: The current test only tests a |
ed388c4
to
da50275
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.
I fixed up the first commit, and now the CI is green. The issue was not resetting portal's newly added exhausted
flag during auto-retries. The PR is missing a test for auto-retries with re-executing the portal, so it is still in a WIP mode, but ready for another look.
I think fundamentally the issue here is that the main state machine is not aware of existence of a partial "side" state machine implemented by limitedCommandResult
which was a known limitation during the portals' implementation. We probably have two choices here:
- do a hacky solution that I'm proposing here by adding some additional tracking info and teaching the conn executor's state machine about an "exhausted" state of a portal - this is done in the first commit. The benefit is that it's a relatively minor change, but we're making the whole situation with portals handling slightly worse.
- perform a fundamental rewrite of how portals are handled. This will probably require rewriting the conn executor's state machine so that it supported what
limitedCommandResult
gives us. I'm not sure how big of a lift that would be, but I think it was considered during initial portals implementation and probably deemed too big of an investment.
Reviewable status: complete! 0 of 0 LGTMs obtained (waiting on @andreimatei and @mjibson)
pkg/sql/conn_executor.go, line 1409 at r1 (raw file):
Have you putting the setting off the exhausted flag inside
limitedCommandResult
? That's the guy that know whether a portal is exhausted or not.
I have thought about it, but I don't think it is possible update the exhausted
flag within the command result because that thing doesn't know when there will be no more rows to add using AddRow
. I think it is known only once dispatchToExecutionEngine
method returns.
We could add another method to RestrictedCommandResult
interface, something like SetExhausted
which would notify the command result that the execution engine has returned all of the rows, that would occur in execWithDistSQLEngine
, after PlanAndRun
call if there is no error. What do you think about that?
The checking of whether a portal is exhausted I'd try to put in
execStmtInOpenState()
so you don't have to inspect the state at this level. Maybe add anExhausted()
method on theRestrictedCommandResult()
interface.
Hm, I prototyped both things in the second commit, but now the regression test fails :/ Maybe you could see what I'm doing wrong?
pkg/sql/conn_executor.go, line 1427 at r1 (raw file):
Previously, andreimatei (Andrei Matei) wrote…
Do we need some logic to reset this field when doing auto-retries? See
prepStmtNamespace.resetTo()
. Please add a test showing that an auto-retry works in re-executing a portal.
I think before this patch portals were immutable so it was easy to share instances between the set of portals captured at the "rewind position" and the current set of portals. But if they're not immutable, more care I think is needed.
Yes, you were right - missing logic for auto-retries handling was the cause of failures we saw on CI earlier. Could you give me a pointer to how I would write such test? I see conn_executor_internal_test.go
which mocks out a connExecutor
, but I'm not sure how I would trigger an auto-retry.
pkg/sql/pgwire/testdata/pgtest/portals, line 482 at r2 (raw file):
Previously, mjibson (Matt Jibson) wrote…
Add a statement type that returns a ROWS type, probably an UPDATE. Execute that twice and then SELECT from it to verify that the UPDATE was only happened once. So it'd probably have to be like
UPDATE t SET s = s + 1
or something that's not idempotent.
Thanks, added.
However, there is one thing I can't figure out (in the first commit) that maybe you could spot a problem with. Namely, if the statement is of "RowsAffected" type (an update without RETURNING), then I'm hitting an error when executing the portal for the second time running against postgres.
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 @andreimatei and @mjibson)
pkg/sql/pgwire/testdata/pgtest/portals, line 482 at r2 (raw file):
Previously, yuzefovich wrote…
Thanks, added.
However, there is one thing I can't figure out (in the first commit) that maybe you could spot a problem with. Namely, if the statement is of "RowsAffected" type (an update without RETURNING), then I'm hitting an error when executing the portal for the second time running against postgres.
Ok, so we've updated the test so that an UPDATE without RETURNING in a portal can only be executed once and returns an error on the second try, but an UPDATE with RETURNING can be attempted to be executed multiple times yet only the first one actually runs.
However, in the former case the "fix" in the first commit exhibits a different behavior:
datadriven.go:118: testdata/pgtest/portals:523: waiting for *pgproto3.ErrorResponse, got &pgproto3.ReadyForQuery{TxStatus:0x54}
meaning that we don't return an error, and, instead, do nothing:
until
ReadyForQuery
----
{"Type":"CommandComplete","CommandTag":"UPDATE 0"}
{"Type":"ReadyForQuery","TxStatus":"T"}
The UPDATE is actually only executed once. It seems like if we want to also return an error in such scenario, more thought is needed.
{"Type":"CommandComplete","CommandTag":"DROP ROLE"} | ||
{"Type":"ReadyForQuery","TxStatus":"I"} | ||
|
||
# Execute a statement of "RowsAffected" statement type. We will try to execute |
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.
Add a test that's a copy of this but does an ALTER/CREATE/DROP or something to assert we do whatever postgres does in that case too.
pkg/sql/prepared_stmt.go
Outdated
@@ -121,6 +121,16 @@ type preparedStatementsAccessor interface { | |||
// References need to be registered with incRef() and de-registered with | |||
// decRef(). | |||
type PreparedPortal struct { | |||
*immutablePreparedPortal |
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.
What's the reason we need to separate out the types here? Why can't we add exhausted to the existing PreparedPortal type?
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 @andreimatei and @mjibson)
pkg/sql/conn_executor.go, line 1409 at r1 (raw file):
Previously, yuzefovich wrote…
Have you putting the setting off the exhausted flag inside
limitedCommandResult
? That's the guy that know whether a portal is exhausted or not.I have thought about it, but I don't think it is possible update the
exhausted
flag within the command result because that thing doesn't know when there will be no more rows to add usingAddRow
. I think it is known only oncedispatchToExecutionEngine
method returns.We could add another method to
RestrictedCommandResult
interface, something likeSetExhausted
which would notify the command result that the execution engine has returned all of the rows, that would occur inexecWithDistSQLEngine
, afterPlanAndRun
call if there is no error. What do you think about that?The checking of whether a portal is exhausted I'd try to put in
execStmtInOpenState()
so you don't have to inspect the state at this level. Maybe add anExhausted()
method on theRestrictedCommandResult()
interface.Hm, I prototyped both things in the second commit, but now the regression test fails :/ Maybe you could see what I'm doing wrong?
I moved that second commit to a different branch, it is here yuzefovich@1c7b540.
pkg/sql/prepared_stmt.go, line 124 at r3 (raw file):
Previously, mjibson (Matt Jibson) wrote…
What's the reason we need to separate out the types here? Why can't we add exhausted to the existing PreparedPortal type?
Added a comment. Basically I think we want to have multiple PreparedPortal
s share the same "immutable core" struct, but we also want to have separate exhausted
variable so that we could rewind the portal correctly.
pkg/sql/pgwire/testdata/pgtest/portals, line 376 at r3 (raw file):
Previously, mjibson (Matt Jibson) wrote…
Remove all the
Sync
afterExecute
, this will remove some of the doubleReadyForQuery
things you have around.
I don't think we can remove Sync
s. Here is a quote from Postgres docs:
At completion of each series of extended-query messages, the frontend should issue a Sync message. This parameterless message causes the backend to close the current transaction if it's not inside a BEGIN/COMMIT transaction block ("close" meaning to commit if no error, or roll back if error). Then a ReadyForQuery response is issued.
Also, there are Sync
s after Execute
s in this file already.
pkg/sql/pgwire/testdata/pgtest/portals, line 489 at r3 (raw file):
Previously, mjibson (Matt Jibson) wrote…
Add a test that's a copy of this but does an ALTER/CREATE/DROP or something to assert we do whatever postgres does in that case too.
Done.
However, similar to an UPDATE without RETURNING, Postgres returns an error whereas we now silently do nothing. Still I think that the first commit in this PR is beneficial since it fixes a worse problem (that a portal can be executed multiple times mistakenly) at the cost of deviation in the behavior of handling of portals on statements that don't return row sets (such portals can only be executed to completion, and it appears that PG returns an error on consequent attempts whereas we'll be silently doing nothing). I moved such cases into portals_crbugs
.
pkg/sql/pgwire/testdata/pgtest/portals, line 464 at r4 (raw file):
---- {"Type":"DataRow","Values":[{"text":"30"}]} {"Type":"CommandComplete","CommandTag":"UPDATE 3"}
There is a difference between us and Postgres on the command tag here:
testdata/pgtest/portals:460: ReadyForQuery
expected:
{"Type":"DataRow","Values":[{"text":"30"}]}
{"Type":"CommandComplete","CommandTag":"UPDATE 3"}
{"Type":"ReadyForQuery","TxStatus":"T"}
found:
{"Type":"DataRow","Values":[{"text":"30"}]}
{"Type":"CommandComplete","CommandTag":"UPDATE 1"}
{"Type":"ReadyForQuery","TxStatus":"T"}
Is this concerning?
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 @andreimatei, @mjibson, and @yuzefovich)
pkg/sql/pgwire/testdata/pgtest/portals, line 376 at r3 (raw file):
Previously, yuzefovich wrote…
I don't think we can remove
Sync
s. Here is a quote from Postgres docs:
At completion of each series of extended-query messages, the frontend should issue a Sync message. This parameterless message causes the backend to close the current transaction if it's not inside a BEGIN/COMMIT transaction block ("close" meaning to commit if no error, or roll back if error). Then a ReadyForQuery response is issued.
Also, there areSync
s afterExecute
s in this file already.
I meant remove Sync
after Query
, not Execute
.
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 @andreimatei and @mjibson)
pkg/sql/pgwire/testdata/pgtest/portals, line 376 at r3 (raw file):
Previously, mjibson (Matt Jibson) wrote…
I meant remove
Sync
afterQuery
, notExecute
.
Oh, that makes sense. I think I removed all such Sync
s.
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 @andreimatei, @mjibson, and @yuzefovich)
pkg/sql/conn_executor.go, line 1427 at r1 (raw file):
Previously, yuzefovich wrote…
Yes, you were right - missing logic for auto-retries handling was the cause of failures we saw on CI earlier. Could you give me a pointer to how I would write such test? I see
conn_executor_internal_test.go
which mocks out aconnExecutor
, but I'm not sure how I would trigger an auto-retry.
But now if we're resetting the exhausted
field when we rewind, in portalNamespace.resetTo()
, I think you can set it here unconditionally, no? And I think that's the correct thing to do - you don't want to allow this portal to be executed again after an error. Which I think it couldn't because, after an error, you have to either start a new txn or rollback to a savepoint (both of which muck with the portals), but still I think you want protection at this layer too.
pkg/sql/conn_executor.go, line 1443 at r4 (raw file):
// to prevent such behavior, we check whether the portal has been // exhausted and execute the statement only if it hasn't. If it has // been exhausted, then we simply do nothing because there is
Is this how Postgres behaves - there's no message sent back when trying to execute an exhausted portal? No CommandComplete
message?
pkg/sql/prepared_stmt.go, line 124 at r3 (raw file):
Previously, yuzefovich wrote…
Added a comment. Basically I think we want to have multiple
PreparedPortal
s share the same "immutable core" struct, but we also want to have separateexhausted
variable so that we could rewind the portal correctly.
The current structure is confusing because you've left incRef
as a method on PreparedPortal
but it refers just to the immutablePreparedPortal
. In prepStmtNamespace.resetTo()
we now increment on a p
and then proceed to make a copy of that p
.
Since we're adding this extra state, I would try to get rid of any sharing in prepStmtNamespace
; it's becoming too complicated to try to share some, but not all of the portal. So, instead of map[string]*PreparedPortal
, I'd have map[string]PreparedPortal
. You can't copy the memAcc
but I think that having an account per portal is pretty silly and inefficient anyway. I'd put prepStmtNamespace
in charge of accounting for all the portal - let it add and delete bytes from a single account as it registers/unregister portals.
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 @andreimatei and @mjibson)
pkg/sql/conn_executor.go, line 1427 at r1 (raw file):
Previously, andreimatei (Andrei Matei) wrote…
But now if we're resetting the
exhausted
field when we rewind, inportalNamespace.resetTo()
, I think you can set it here unconditionally, no? And I think that's the correct thing to do - you don't want to allow this portal to be executed again after an error. Which I think it couldn't because, after an error, you have to either start a new txn or rollback to a savepoint (both of which muck with the portals), but still I think you want protection at this layer too.
I think you're right, done. This condition came from me not really understanding how connExecutor
works - I can't say that I fully understand it, but I have a better grasp of it now.
pkg/sql/conn_executor.go, line 1443 at r4 (raw file):
Previously, andreimatei (Andrei Matei) wrote…
Is this how Postgres behaves - there's no message sent back when trying to execute an exhausted portal? No
CommandComplete
message?
Postgres does send CommandComplete
with SELECT 0
command tag (and we have such newly-added test case in pgtest/portals:386-396
). Is the comment confusing? By "we do nothing" I meant that we need to not dispatch the query to the execution engine, but the remaining state transitions of connExecutor
will still occur, only as if the execution engine returned 0 rows. Updated the comment.
What's different is that we are not returning an error in cases when Postgres would on attempts on executing exhausted portals for certain statement types. See my comment on testdata/portals
file.
pkg/sql/prepared_stmt.go, line 124 at r3 (raw file):
Previously, andreimatei (Andrei Matei) wrote…
The current structure is confusing because you've left
incRef
as a method onPreparedPortal
but it refers just to theimmutablePreparedPortal
. InprepStmtNamespace.resetTo()
we now increment on ap
and then proceed to make a copy of thatp
.
Since we're adding this extra state, I would try to get rid of any sharing inprepStmtNamespace
; it's becoming too complicated to try to share some, but not all of the portal. So, instead ofmap[string]*PreparedPortal
, I'd havemap[string]PreparedPortal
. You can't copy thememAcc
but I think that having an account per portal is pretty silly and inefficient anyway. I'd putprepStmtNamespace
in charge of accounting for all the portal - let it add and delete bytes from a single account as it registers/unregister portals.
Good point, done.
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 @andreimatei, @mjibson, and @yuzefovich)
pkg/sql/conn_executor.go, line 1443 at r4 (raw file):
Previously, yuzefovich wrote…
Postgres does send
CommandComplete
withSELECT 0
command tag (and we have such newly-added test case inpgtest/portals:386-396
). Is the comment confusing? By "we do nothing" I meant that we need to not dispatch the query to the execution engine, but the remaining state transitions ofconnExecutor
will still occur, only as if the execution engine returned 0 rows. Updated the comment.What's different is that we are not returning an error in cases when Postgres would on attempts on executing exhausted portals for certain statement types. See my comment on
testdata/portals
file.
comment is better now
You're sure that postgres allows you to Execute
a portal multiple times after the portal is suspended, right?
pkg/sql/conn_executor_prepare.go, line 401 at r5 (raw file):
if portal, ok := ex.extraTxnState.prepStmtsNamespace.portals[portalName]; !ok { panic(fmt.Sprintf("portal %s doesn't exist", portalName)) } else {
nit: drop the else
and unindent the expected case
pkg/sql/pgwire/testdata/pgtest/portals, line 346 at r5 (raw file):
{"Type":"ReadyForQuery","TxStatus":"I"} # Regression for #48448
instead of click-bait, please put some words about what we're testing
pkg/sql/pgwire/testdata/pgtest/portals, line 396 at r5 (raw file):
{"Type":"CommandComplete","CommandTag":"SELECT 0"} {"Type":"ReadyForQuery","TxStatus":"T"}
I'd try another Execute
here. I wanna make sure that pg returns SELECT 0
multiple times.
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 @andreimatei, @mjibson, and @yuzefovich)
pkg/sql/pgwire/testdata/pgtest/portals, line 464 at r4 (raw file):
Previously, yuzefovich wrote…
There is a difference between us and Postgres on the command tag here:
testdata/pgtest/portals:460: ReadyForQuery expected: {"Type":"DataRow","Values":[{"text":"30"}]} {"Type":"CommandComplete","CommandTag":"UPDATE 3"} {"Type":"ReadyForQuery","TxStatus":"T"} found: {"Type":"DataRow","Values":[{"text":"30"}]} {"Type":"CommandComplete","CommandTag":"UPDATE 1"} {"Type":"ReadyForQuery","TxStatus":"T"}
Is this concerning?
If this question is for me, I think this is the time to figure out the exact details of the pg implementation. Obviously, the protocol is only defined by the Postgres code :0
I don't understand what the diff here says, but I think the one below seems bad
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.
Alright, I think this PR is now in a pretty good shape - it fixes the bug and doesn't introduce any deviations from the postgres (at the cost of some hacky code). RFAL.
Reviewable status: complete! 0 of 0 LGTMs obtained (waiting on @andreimatei and @mjibson)
pkg/sql/conn_executor.go, line 1443 at r4 (raw file):
Previously, andreimatei (Andrei Matei) wrote…
comment is better now
You're sure that postgres allows you to
Execute
a portal multiple times after the portal is suspended, right?
Yes, that's the case for statements of tree.Rows
statement type.
I also updated the code to return errors similar to what Postgres does, and AFAICT this PR doesn't introduce any new deviations from PG.
pkg/sql/pgwire/testdata/pgtest/portals, line 464 at r4 (raw file):
Previously, andreimatei (Andrei Matei) wrote…
If this question is for me, I think this is the time to figure out the exact details of the pg implementation. Obviously, the protocol is only defined by the Postgres code :0
I don't understand what the diff here says, but I think the one below seems bad
I looked into the postgres code, and - in a sense - they store the number of "affected rows" in such cases so that when executing exhausted portals the corresponding number is used in the command tag. I updated the code to have such behavior in CRDB as well.
pkg/sql/pgwire/testdata/pgtest/portals, line 346 at r5 (raw file):
Previously, andreimatei (Andrei Matei) wrote…
instead of click-bait, please put some words about what we're testing
Done.
pkg/sql/pgwire/testdata/pgtest/portals, line 396 at r5 (raw file):
Previously, andreimatei (Andrei Matei) wrote…
I'd try another
Execute
here. I wanna make sure that pg returnsSELECT 0
multiple times.
Done.
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've talked with Matt, and we decided that the new additions that attempt to simulate the "rows affected" behavior in the command tag are incorrect and very brittle, so that code has been deleted. The current PR fixes the really bad problem (of re-executing exhausted portals) but introduces some deviations which are less severe. We found such trade-off acceptable.
Reviewable status: complete! 0 of 0 LGTMs obtained (waiting on @andreimatei and @mjibson)
pkg/sql/conn_executor.go, line 1477 at r6 (raw file):
Previously, andreimatei (Andrei Matei) wrote…
please move the exhausted portal code out of this function, to an
execExhaustedPortal()
or so
The code here has been simplified, but I also extracted a separate function to not flood this file with long comments.
pkg/sql/conn_executor.go, line 1488 at r6 (raw file):
Previously, andreimatei (Andrei Matei) wrote…
s/return/returns
Done.
pkg/sql/conn_executor.go, line 1489 at r6 (raw file):
Previously, mjibson (Matt Jibson) wrote…
What about statements like
VALUES (1)
?
It's an interesting point. I was a little bit surprised, but this PR behaves exactly like Postgres because a query like VALUES (0)
is actually parsed as a tree.Select
object in which Select
parameter is a tree.ValuesClause
. Added it as a test.
I'm assuming that PG's CMD_SELECT
is equivalent to ours `Stmt.AST.StatementTag() == "SELECT" (which seems like a reasonable assumption given https://github.com/postgres/postgres/blob/1aac32df89eb19949050f6f27c268122833ad036/src/include/nodes/nodes.h#L67
Update: this code has been removed.
pkg/sql/conn_executor.go, line 1510 at r6 (raw file):
Previously, mjibson (Matt Jibson) wrote…
This comment is slightly misleading. It should describe how, while we do support portal suspension, it happens in another state machine and thus once execStmt has returned, the portal has executed to completion.
Good point, updated.
pkg/sql/pgwire/command_result.go, line 332 at r6 (raw file):
Previously, mjibson (Matt Jibson) wrote…
I don't think this is correct. See https://github.com/postgres/postgres/blob/2f48ede080f42b97b594fb14102c82ca1001b80c/src/backend/tcop/pquery.c#L227. There it looks at two things to determine the ONE_SELECT portal strategy: it must have a commandType of CMD_SELECT and not be have a modifying CTE (https://github.com/postgres/postgres/blob/2f48ede080f42b97b594fb14102c82ca1001b80c/src/backend/parser/parse_cte.c#L147). Here we are only checking for the tag, not the commandType.
I agree, the current code deviates from Postgres.
However, understanding whether a query has a modifying CTE (or a subquery) seems quite hard at the point when newCommandResult
is called - it is done before makeOptimizerPlan
, and I feel like a minor deviation like this should not block this PR. Namely, the deviation is minor because this suspensionStrategy
currently only influences "rows affected" parameter which is need to populate correctly "command tag".
Update: this code has been removed.
pkg/sql/pgwire/command_result.go, line 366 at r6 (raw file):
Previously, mjibson (Matt Jibson) wrote…
These comments should document how, or if, they relate to which enum values in postgres.
This code has been removed.
pkg/sql/pgwire/command_result.go, line 478 at r6 (raw file):
Previously, andreimatei (Andrei Matei) wrote…
This comment doesn't help very much.
What would be great here is talking about which portals' results return the number of rows returned from the beginning, and which portals return the number of rows part of the currentExecPortal
. Like, am I right to understand thatselect * from foo
executed with limit 100 will return a tag saying 100 rows with every resume, butupdate foo returning bar
executed with limit 100 will return tags 100, then 200, then 300?
This code has been removed.
pkg/sql/conn_executor_prepare.go
Outdated
@@ -397,6 +394,17 @@ func (ex *connExecutor) addPortal( | |||
return nil | |||
} | |||
|
|||
// exhaustPortal marks a portal with the provided name as "exhausted" and | |||
// returns an error if there is no portal with such name. |
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.
returns an error -> panics
Very +1 with allowing slightly differing behavior so that we can fix the bad reexec bug. |
Previously, we would erroneously restart the execution from the very beginning of empty, unclosed portals after they have been fully exhausted when we should be returning no rows or an error in such scenarios. This is now fixed by tracking whether a portal is exhausted or not and intercepting the calls to `execStmt` when the conn executor state machine is in an open state. Note that the current solution has known deviations from Postgres: - when attempting to execute portals of statements that don't return row sets, on the second and consequent attempt PG returns an error while we are silently doing nothing (meaning we do not run the statement at all and return 0 rows) - we incorrectly populate "command tag" field of pgwire messages of some rows-returning statements after the portal suspension (for example, a suspended UPDATE RETURNING in PG will return the total count of "rows affected" while we will return the count since the last suspension). These deviations are deemed acceptable since this commit fixes a much worse problem - re-executing an exhausted portal (which could be a mutation meaning, previously, we could have executed a mutation multiple times). The reasons for why this commit does not address these deviations are: - Postgres has a concept of "portal strategy" (see https://github.com/postgres/postgres/blob/2f9661311b83dc481fc19f6e3bda015392010a40/src/include/utils/portal.h#L89). - Postgres has a concept of "command" type (these are things like SELECTs, UPDATEs, INSERTs, etc, see https://github.com/postgres/postgres/blob/1aac32df89eb19949050f6f27c268122833ad036/src/include/nodes/nodes.h#L672). CRDB doesn't have these concepts, and without them any attempt to simulate Postgres results in a very error-prone and brittle code. Release note (bug fix): Previously, CockroachDB would erroneously restart the execution of empty, unclosed portals after they have been fully exhausted, and this has been fixed.
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'll wait for @andreimatei to take another quick look. Let me know if you think it's not necessary.
Reviewable status: complete! 0 of 0 LGTMs obtained (waiting on @andreimatei and @mjibson)
pkg/sql/conn_executor_prepare.go, line 398 at r7 (raw file):
Previously, mjibson (Matt Jibson) wrote…
returns an error -> panics
Oops, indeed. I have Goland collapsing such statements, and usually those are "return error".
Hey @andreimatei, are you intending to take a look? That's ok if not - I think this PR has received a fair amount of scrutiny, and I'd like to merge it tomorrow. |
Go ahead
…On Wed, Aug 5, 2020, 6:53 PM yuzefovich ***@***.***> wrote:
Hey @andreimatei <https://github.com/andreimatei>, are you intending to
take a look? That's ok if not - I think this PR has received a fair amount
of scrutiny, and I'd like to merge it tomorrow.
—
You are receiving this because you were mentioned.
Reply to this email directly, view it on GitHub
<#48842 (comment)>,
or unsubscribe
<https://github.com/notifications/unsubscribe-auth/AAC4C4N7CMYL2REWFJBFRPTR7HPF3ANCNFSM4NAGUTJQ>
.
|
Thanks everyone for the input! bors r+ |
This PR was included in a batch that was canceled, it will be automatically retried |
Build succeeded: |
PR cockroachdb#48842 added logic to exhaust portals after executing them. This had issues when the portal being executed closes itself, which happens when using DEALLOCATE in a prepared statement. Now we check if the portal still exists before exhausting it. There is no release note as this fixes a bug that only exists in unreleased versions. Release note: None
52940: sql: allow DEALLOCATE ALL with a prepared statement r=asubiotto a=rafiss PR #48842 added logic to exhaust portals after executing them. This had issues when the portal being executed closes itself, which happens when using DEALLOCATE in a prepared statement. Now we check if the portal still exists before exhausting it. There is no release note as this fixes a bug that only exists in unreleased versions. fixes #52915 fixes #52220 fixes #52880 fixes #52506 Release note: None 52958: RFCs: update geospatial RFC r=otan a=otan Update geospatial RFC to be in line with updated offerings and commands. Release note: None 52961: builtins: implement ST_MakeBox2D r=sumeerbhola a=otan Resolves #52900. Release note (sql change): Implement ST_MakeBox2D. 52963: builtins: implement the PostGIS_GetBBox builtin r=sumeerbhola a=otan Release note (sql change): Implement the PostGIS_GetBBox builtin. 52965: tree: implement casts between box2d and geometry r=rytaft,sumeerbhola a=otan Release note (sql change): Implement the ability to cast between box2d and geometry types. Co-authored-by: Rafi Shamim <rafi@cockroachlabs.com> Co-authored-by: Oliver Tan <otan@cockroachlabs.com>
PR cockroachdb#48842 added logic to exhaust portals after executing them. This had issues when the portal being executed closes itself, which happens when using DEALLOCATE in a prepared statement. Now we check if the portal still exists before exhausting it. There is no release note as this fixes a bug that only exists in unreleased versions. Release note: None
PR cockroachdb#48842 added logic to exhaust portals after executing them. This had issues when the portal being executed closes itself, which happens when using DEALLOCATE in a prepared statement. Now we check if the portal still exists before exhausting it. There is no release note as this fixes a bug that only exists in unreleased versions. Release note: None
PR cockroachdb#48842 added logic to exhaust portals after executing them. This had issues when the portal being executed closes itself, which happens when using DEALLOCATE in a prepared statement. Now we check if the portal still exists before exhausting it. There is no release note as this fixes a bug that only exists in unreleased versions. Release note: None
PR cockroachdb#48842 added logic to exhaust portals after executing them. This had issues when the portal being executed closes itself, which happens when using DEALLOCATE in a prepared statement. Now we check if the portal still exists before exhausting it. There is no release note as this fixes a bug that only exists in unreleased versions. Release note: None
Previously, we would erroneously restart the execution from the very
beginning of empty, unclosed portals after they have been fully
exhausted when we should be returning no rows or an error in such
scenarios. This is now fixed by tracking whether a portal is exhausted
or not and intercepting the calls to
execStmt
when the conn executorstate machine is in an open state.
Note that the current solution has known deviations from Postgres:
sets, on the second and consequent attempt PG returns an error while we
are silently doing nothing (meaning we do not run the statement at all
and return 0 rows)
rows-returning statements after the portal suspension (for example,
a suspended UPDATE RETURNING in PG will return the total count of "rows
affected" while we will return the count since the last suspension).
These deviations are deemed acceptable since this commit fixes a much
worse problem - re-executing an exhausted portal (which could be
a mutation meaning, previously, we could have executed a mutation
multiple times).
The reasons for why this commit does not address these deviations are:
(see https://github.com/postgres/postgres/blob/2f9661311b83dc481fc19f6e3bda015392010a40/src/include/utils/portal.h#L89).
SELECTs, UPDATEs, INSERTs, etc,
see https://github.com/postgres/postgres/blob/1aac32df89eb19949050f6f27c268122833ad036/src/include/nodes/nodes.h#L672).
CRDB doesn't have these concepts, and without them any attempt to
simulate Postgres results in a very error-prone and brittle code.
Fixes: #48448.
Release note (bug fix): Previously, CockroachDB would erroneously
restart the execution of empty, unclosed portals after they have been
fully exhausted, and this has been fixed.