sql: fix internal executor usage in leaf txn#45966
sql: fix internal executor usage in leaf txn#45966craig[bot] merged 1 commit intocockroachdb:masterfrom
Conversation
|
@jordanlewis pretty please find me a volunteer to write a test for this. We need something that uses the internal executor on remote notes, through DistSQL. I tried simple stuff in the vein of what SQLSmith did in the linked issue, but couldn't repro immediately and I'd rather pass it to someone else. |
|
SQLSmith hit this again. The query that SQLSmith says it ran, I think, is: |
|
friendly ping |
611b4b6 to
56ad76c
Compare
|
Added a test that reproduces the panic deterministically on master. However, it seems like calls to |
56ad76c to
94aca84
Compare
|
Fixed the hang which was just a double mutex acquisition with the new code. I'm now running into: While running the test. I added a check to only configure stepping and step if the txn is not a leaf txn as well as one other check in This is now RFAL @andreimatei PTAL |
andreimatei
left a comment
There was a problem hiding this comment.
Reviewable status:
complete! 0 of 0 LGTMs obtained (waiting on @andreimatei and @asubiotto)
pkg/sql/conn_executor_exec.go, line 561 at r1 (raw file):
// here.) // We first ensure stepping mode is enabled if the txn is not a leaf txn.
Checking the txn type in the connEx seems wrong to me. I'd rather have a field on the connEx that explicitly disables stepping, set it in newConnExWithTxn regardless of leaf vs root txn, and hang a comment there saying that internal executors just don't get to have stepping and thus shouldn't be used for queries where the read set overlaps the write set.
But rather, I'm not actually sure why we don't support stepping on leaves. @knz do you remember?
pkg/sql/conn_executor_exec.go, line 575 at r1 (raw file):
// single/common function. That would be where the stepping mode // gets enabled once for all SQL statements executed "underneath". if ex.state.mu.txn.Type() != kv.LeafTxn {
nit: switch this to == RootTxn
pkg/sql/conn_executor_exec.go, line 620 at r1 (raw file):
txn := ex.state.mu.txn if !os.ImplicitTxn.Get() && (txn.Type() != kv.LeafTxn && txn.IsSerializablePushAndRefreshNotPossible()) {
Why was this necessary exactly?
I would hope that canAutoRetry is always false below when the connEx was built with newConnExecutorWithTxn, but after looking for a minute I can't say what would make it so.
knz
left a comment
There was a problem hiding this comment.
Reviewable status:
complete! 0 of 0 LGTMs obtained (waiting on @andreimatei and @asubiotto)
pkg/sql/conn_executor_exec.go, line 561 at r1 (raw file):
Previously, andreimatei (Andrei Matei) wrote…
Checking the txn type in the connEx seems wrong to me. I'd rather have a field on the
connExthat explicitly disables stepping, set it innewConnExWithTxnregardless of leaf vs root txn, and hang a comment there saying that internal executors just don't get to have stepping and thus shouldn't be used for queries where the read set overlaps the write set.
But rather, I'm not actually sure why we don't support stepping on leaves. @knz do you remember?
A couple of separate things
-
stepping in the leaf is non-sensical, because the stepping increases the sequence number and the sequence number must be monotonic across all leafs (and the root). Since we don't mutually synchronize all the leaves with each other, the monotonic behavior is impossible to implement.
-
internal executors should be able to do stepping, as stepping is required to execute single statements (e.g. a mutation with a cascade needs two steps)
andreimatei
left a comment
There was a problem hiding this comment.
Reviewable status:
complete! 0 of 0 LGTMs obtained (waiting on @andreimatei, @asubiotto, and @knz)
pkg/sql/conn_executor_exec.go, line 561 at r1 (raw file):
internal executors should be able to do stepping, as stepping is required to execute single statements (e.g. a mutation with a cascade needs two steps)
Well for whatever reason (I guess some class of crdb_internal functions or casts?) we seem to need internal executors in leaf txns, so I guess these guys don't actually need any stepping since they're not doing mutations.
stepping in the leaf is non-sensical, because the stepping increases the sequence number and the sequence number must be monotonic across all leafs (and the root). Since we don't mutually synchronize all the leaves with each other, the monotonic behavior is impossible to implement.
Well I was thinking we just make all the stepping-stuff no-ops on the argument that no mutations are going on concurrently. I was thinking I'd prefer that over testing for the txn type here as this patch does.
asubiotto
left a comment
There was a problem hiding this comment.
TFTR
Reviewable status:
complete! 0 of 0 LGTMs obtained (waiting on @andreimatei and @asubiotto)
pkg/sql/conn_executor_exec.go, line 561 at r1 (raw file):
Previously, andreimatei (Andrei Matei) wrote…
internal executors should be able to do stepping, as stepping is required to execute single statements (e.g. a mutation with a cascade needs two steps)
Well for whatever reason (I guess some class of crdb_internal functions or casts?) we seem to need internal executors in leaf txns, so I guess these guys don't actually need any stepping since they're not doing mutations.
stepping in the leaf is non-sensical, because the stepping increases the sequence number and the sequence number must be monotonic across all leafs (and the root). Since we don't mutually synchronize all the leaves with each other, the monotonic behavior is impossible to implement.
Well I was thinking we just make all the stepping-stuff no-ops on the argument that no mutations are going on concurrently. I was thinking I'd prefer that over testing for the txn type here as this patch does.
Do we really want to make stepping stuff no-ops? It seems weird for me that we would allow a caller to call ConfigureStepping for example and not panic or return an error (as a side note, we probably should never panic in this case) which could lead the caller to mistakenly believe the operation was successful. I added an explicit field on connEx
pkg/sql/conn_executor_exec.go, line 575 at r1 (raw file):
Previously, andreimatei (Andrei Matei) wrote…
nit: switch this to
== RootTxn
Switched to check the connexecutor boolean
pkg/sql/conn_executor_exec.go, line 620 at r1 (raw file):
Previously, andreimatei (Andrei Matei) wrote…
Why was this necessary exactly?
I would hope thatcanAutoRetryis always false below when the connEx was built withnewConnExecutorWithTxn, but after looking for a minute I can't say what would make it so.
Because IsSerializablePushAndRefreshNotPossible would panic if called on a leaf txn.
94aca84 to
9af3b93
Compare
andreimatei
left a comment
There was a problem hiding this comment.
Reviewable status:
complete! 0 of 0 LGTMs obtained (waiting on @andreimatei, @asubiotto, and @knz)
pkg/sql/conn_executor_exec.go, line 561 at r1 (raw file):
stepping in the leaf is non-sensical, because the stepping increases the sequence number and the sequence number must be monotonic across all leafs (and the root).
It doesn't... Does it? Stepping only records the current sequence number, it doesn't change it. So now I again think that the code would just work on leaves. There'd be nothing about seq nums to communicate back to root because the leaf is by definition r/o so the seq nums haven't advanced. The only information about stepping that'd get lost between leaf and root is the enablement of the stepping on the leaf, but I don't think that's a big deal.
But this patch seems pretty smelly to me now. I think it's quite unfortunate for the connEx to look at what type of transaction it's been given and change its behavior accordingly. Introducing the new transactionSteppingDisabled field did not make it any better - I was hoping we'd find a higher-level concept to latch the change on behavior on (like whether the session is read-only or read-write). I'm thinking the statements being run either need the stepping, in which case it's incorrect to not enable it, or don't in which case I'm hoping the txn code would just work fine and naturally be a no-op.
It also bothers me that further down we check this transactionSteppingDisabled for something that has nothing to do with stepping. I think we can get IsSerializablePushAndRefreshNotPossible to do the right thing one way or another again with no conditional logic in the connEx.
I'd like to make sure that we actually need to work on this. I guess the connEx was never meant to be used with a leaf (or more like the other way around - the leag was never meant to serve more then the specific needs of the TableReaders, not the much broader surface on the connEx). How exactly does the connEx end up being used in a leaf? Does it end up being used on a remote node, or is it happening on the gateway? Is it about casts that need to use it?
pkg/sql/conn_executor_exec.go, line 561 at r2 (raw file):
// here.) // We first ensure stepping mode is enabled if the txn is not a leaf txn.
the new comment is now stale; I'd remove it.
asubiotto
left a comment
There was a problem hiding this comment.
Reviewable status:
complete! 0 of 0 LGTMs obtained (waiting on @andreimatei and @asubiotto)
pkg/sql/conn_executor_exec.go, line 561 at r1 (raw file):
I'd like to make sure that we actually need to work on this. I guess the connEx was never meant to be used with a leaf (or more like the other way around - the leag was never meant to serve more then the specific needs of the TableReaders, not the much broader surface on the connEx). How exactly does the connEx end up being used in a leaf? Does it end up being used on a remote node, or is it happening on the gateway? Is it about casts that need to use it?
Based on the stack trace in #45924 (comment), this is executed on a remote node to perform a cast. The query in question is retrieving an oid (using queryOidWithJoin). I think disallowing internal executor usage in leaf txns is a non-starter so we have to find a way to make this work.
But this patch seems pretty smelly to me now. I think it's quite unfortunate for the connEx to look at what type of transaction it's been given and change its behavior accordingly. Introducing the new transactionSteppingDisabled field did not make it any better - I was hoping we'd find a higher-level concept to latch the change on behavior on (like whether the session is read-only or read-write). I'm thinking the statements being run either need the stepping, in which case it's incorrect to not enable it, or don't in which case I'm hoping the txn code would just work fine and naturally be a no-op.
I'm not sure if there is a higher-level concept, is there? An internal executor is created without any specific settings. I agree that it's not perfect, but I think we should move forward with this patch because it fixes the immediate panic and is self-contained (we need to backport to release-20.2) and work on improving this situation down the line.
It also bothers me that further down we check this transactionSteppingDisabled for something that has nothing to do with stepping. I think we can get IsSerializablePushAndRefreshNotPossible to do the right thing one way or another again with no conditional logic in the connEx.
I'm happy to change that but I need your input. How do I implement IsSerializablePushAndRefreshNotPossible for leaf txns? Just remove the check or return false unconditionally? If we remove this, we'll just have a single check for a leaf txn (I assume I might as well remove the boolean) which I think is acceptable.
andreimatei
left a comment
There was a problem hiding this comment.
Reviewable status:
complete! 0 of 0 LGTMs obtained (waiting on @andreimatei, @asubiotto, and @knz)
pkg/sql/conn_executor_exec.go, line 561 at r1 (raw file):
I'm not sure if there is a higher-level concept, is there? An internal executor is created without any specific settings.
I think the TxnCoordSender code for ConfigureStepping and Step will just work and naturally be a no-op when executing read-only statements (which are the only ones that can be executed in a leaf). So I believe we can just remove the respective assertions in those methods that we're not in a leaf. At least I don't see a reason to not do that.
For sanity, I would also try to make newConnExecutorWithTxn set the transaction_read_only session variable on the executor's session.
I'm happy to change that but I need your input. How do I implement IsSerializablePushAndRefreshNotPossible for leaf txns?
My hope here too is that we don't actually need any conditional logic anywhere. I think IsSerializablePushAndRefreshNotPossible works just fine in leaves; I think we can just remove the assertion that it's in a root. Rafa introduced that assertion, but I'm not entirely sure if he had a particular reason for it (although I don't blame him for adding it, I think it's sane to limit what can be called on leaves until it turns out there's a reason do allow it).
I would hope that the canAutoRetry variable below ends up always being false below in executors created with newConnExecutorWithTxn, regardless of whether we're in a root or a leaf. I looked briefly and couldn't convince myself that that's the case, but I think that's the sane thing to do - when the connEx is running in a higher level txn, it doesn't have information about how far back it can rewind, so I hope that the way in which we set it up makes that work. Maybe this line does it?
Line 122 in 1d38d0b
9af3b93 to
1e73ca1
Compare
asubiotto
left a comment
There was a problem hiding this comment.
Reviewable status:
complete! 0 of 0 LGTMs obtained (waiting on @andreimatei and @asubiotto)
pkg/sql/conn_executor_exec.go, line 561 at r1 (raw file):
For sanity, I would also try to make newConnExecutorWithTxn set the transaction_read_only session variable on the executor's session.
I don't understand this. What guarantees do we have that it's a read-only txn? We still want to support creating internal executors with a root txn that performs mutations.
I removed the assertions and the checks and my unit test runs fine. Let's see what CI says about it.
andreimatei
left a comment
There was a problem hiding this comment.
Reviewable status:
complete! 0 of 0 LGTMs obtained (waiting on @andreimatei, @asubiotto, and @knz)
pkg/sql/conn_executor_exec.go, line 561 at r1 (raw file):
I don't understand this. What guarantees do we have that it's a read-only txn? We still want to support creating internal executors with a root txn that performs mutations.
Sorry, I meant do that if the txn is a leaf (I think looking at the leaf vs root in the ctor is fine).
1e73ca1 to
7209aca
Compare
asubiotto
left a comment
There was a problem hiding this comment.
Alright, I think this is ready for a final look.
Reviewable status:
complete! 0 of 0 LGTMs obtained (waiting on @andreimatei and @asubiotto)
pkg/sql/conn_executor_exec.go, line 561 at r1 (raw file):
Previously, andreimatei (Andrei Matei) wrote…
I don't understand this. What guarantees do we have that it's a read-only txn? We still want to support creating internal executors with a root txn that performs mutations.
Sorry, I meant do that if the txn is a leaf (I think looking at the leaf vs root in the ctor is fine).
👍 Done.
pkg/sql/conn_executor_exec.go, line 561 at r2 (raw file):
Previously, andreimatei (Andrei Matei) wrote…
the new comment is now stale; I'd remove it.
Done.
andreimatei
left a comment
There was a problem hiding this comment.
LGTM but let's give @knz a chance to look this over.
The commit message is no longer complete; say something more broadly about internal executors in leaves. Feel free to take authorship too.
Reviewable status:
complete! 0 of 0 LGTMs obtained (waiting on @andreimatei and @asubiotto)
pkg/sql/conn_executor_internal_test.go, line 380 at r3 (raw file):
} func TestInternalExecutorInLeafTxnDoesNotPanic(t *testing.T) {
if this doesn't need to be in the sql package, move it to internal_test.go; that's where most of the tests are.
7209aca to
9a3d56d
Compare
|
Reworked commit message and moved test. |
knz
left a comment
There was a problem hiding this comment.
modulo two polish items see below
Reviewed 1 of 5 files at r1, 1 of 3 files at r2, 4 of 4 files at r3, 2 of 2 files at r4.
Reviewable status:complete! 1 of 0 LGTMs obtained (waiting on @andreimatei)
pkg/sql/conn_executor.go, line 698 at r4 (raw file):
) *connExecutor { ex := s.newConnExecutor(ctx, sd, sdDefaults, stmtBuf, clientComm, memMetrics, srvMetrics, appStats) if txn.Type() == kv.LeafTxn {
Can you add an explanatory comment here that motivates this piece of code, for the benefit of the future maintainer.
pkg/sql/txn_state.go, line 217 at r4 (raw file):
} else { if priority != roachpb.UnspecifiedUserPriority { panic(fmt.Sprintf("unexpected priority when using an existing txn: %s", priority))
panic(errors.AssertionFailedf(...)) please
Internal executors use a conn executor under the hood, which sets a priority and configures stepping on the transaction it is provided with. Both operations would previously cause panics when used with a leaf transaction. This commit reworks setting transaction priority on leaf transactions to avoid crashing when the new priority is the same as the old one, and allows stepping to be configured on a leaf transaction since doing so is a noop. Fixes cockroachdb#45924 Release note: None Release justification: crash fix
9a3d56d to
bd255e5
Compare
asubiotto
left a comment
There was a problem hiding this comment.
Reviewable status:
complete! 0 of 0 LGTMs obtained (and 1 stale) (waiting on @andreimatei and @knz)
pkg/sql/conn_executor.go, line 698 at r4 (raw file):
Previously, knz (kena) wrote…
Can you add an explanatory comment here that motivates this piece of code, for the benefit of the future maintainer.
Done.
pkg/sql/txn_state.go, line 217 at r4 (raw file):
Previously, knz (kena) wrote…
panic(errors.AssertionFailedf(...))please
Done.
|
TFTR bors r=knz,andreimatei |
|
Build succeeded: |
The internal executor was setting a priority on a leaf txn, which is not
allowed. It was supposed to be a no-op since the priority was the same
as the existing one, but it crashed nonetheless. This patch makes the
no-op even more no-op.
Fixes #45924
Release note: None
Release justification: crash fix