sql: introduce new internal executor interfaces#82477
sql: introduce new internal executor interfaces#82477craig[bot] merged 7 commits intocockroachdb:masterfrom
Conversation
knz
left a comment
There was a problem hiding this comment.
Could you explain to us what you recommend happens in the following "common" use case:
err := db.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error {
... := internalExecutor.QueryRowEx(ctx, ..., txn, query)
})FYI, What this pattern does is to run the query but make it robust to transaction retry errors: the code inside db.Txn() will re-execute the closure (containing the QueryRowEx call) every time a retry error is encountered, automatically.
Now, what would happen with the proposed APIs? It seems to me that it would be "expensive" to re-initialize an internalexecutor from scratch every time db.Txn invokes the closure.
Would there be a way to "refresh" the kv.Txn object inside the executor, the same way that sql.connExecutor does internally for auto-retries (like in prepareTxnForRetryWithRewind())?
Reviewable status:
complete! 0 of 0 LGTMs obtained (waiting on @ajwerner, @miretskiy, and @rafiss)
|
If I understand correctly, nothing really has been made worse by this interface change. I'd like to unpack the extent to which it has.
The
I don't think I buy this. The internalExecutor we have today is shockingly thin. We already build a new connExecutor for each invocation. That means this change is not worse than any existing code which uses an explicit Lines 129 to 140 in cc6aa33 |
|
ok, that's a good explanation thanks |
|
Thanks @ajwerner for the explanation! Yes the understanding is correct.
As Andrew mentioned, the initialization of an internal executor itself is to call So if it's acceptable to re-init the internal executor for each retry in the |
22de15b to
a305630
Compare
445e55e to
a9b858c
Compare
I disagree that it makes the story worse; I view this as an incremental improvement. This is evident by the fact that we already see the roundtrip tests improving. If your concern is just about tech debt, then with clear commenting and documentation, I really don't think we are setting ourselves for the disaster that you fear.
I feel that we need these new interfaces merged for us to have common ground when discussing how to make further improvements. I feel pretty strongly that the way forward to this goal is to merge this PR and to tackle the other issues you raise in future PRs. I worry that trying to tackle each issue in this PR might have been making it harder to iterate on addressing the concerns you are raising.
We still plan to work through tech debt in this area during the stability period. I hope it's clear that I'm not trying to dismiss your feedback. I'm trying to say that the best way to address your valuable feedback is to merge this large PR and continue working on improvements. |
|
Does this PR fix #70888? |
Sadly no. This PR doesn't deprecate the existing |
|
Okay thanks for the update! No worries, Rome wasn't built in a day. Could we get an Informs: in your commit message to that issue just so things are looped together? Thanks! |
Yeah, sure! I'll add that. |
|
Made changes per comments, except changing |
|
Made changes including changing |
068f90b to
3df480e
Compare
|
Friendly reping-ing @ajwerner for another look. Thanks! |
ajwerner
left a comment
There was a problem hiding this comment.
This is more or less where I want it. I think it'd be good if we delegate the functionality we need in the descs layer to a method in the sql layer on the InternalExecutorFactory, but otherwise, I'm content with the shape this has taken. Thanks for all the iteration.
pkg/sql/catalog/descs/factory.go
Outdated
| // InternalExecutorFactoryWithTxn is used to create an internal executor | ||
| // with associated extra txn state information. | ||
| type InternalExecutorFactoryWithTxn func( |
There was a problem hiding this comment.
it feels to me like this should just be a method on the InternalExecutorFactory and that this library should describe the interface method. That way, we can avoid the closure in the server package.
There was a problem hiding this comment.
Yeah I thought about that but it can be a bit difficult -- we will have to import descs package to sqlutil for the collection parameter, but that will again bring dependency loop.
There was a problem hiding this comment.
I don't think that's true, sqlutil doesn't need to include this method in its interface, just the sql.InternalExecutorFactory has to implement the method. As in, we define an interface in descs which is distinct from the interface in sqlutil.
There was a problem hiding this comment.
Ah, i see what you meant. Yeah, that makes sense, will do!
…are passed This commit adds a boolean field `fromOuterTxn` to the conn executor's extraTxnState. It's set true when the conn executor is run with a not-nil txn passed from the internal executor, and hence the collection and the job records, which are passed from the caller of the internal executor to the conn executor, should not be released when the conn executor close. Instead, we leave the caller to release them. This commit also changed the descriptor collection and schema changer state stored in conn executor's `ExtraTxnState` to pointer. We also deprecated `collectionFactory.MakeCollection()` with `collectionFactory.NewCollection()`. Release note: None
This commit 1. renamed the original `sqlutil.SessionBoundInternalExecutorFactory` to a more general name `sqlutil.InternalExecutorFactory`, and 2. change this factory from a function to an interface, that include a `NewInternalExecutor()` method with the same logic as the original function. Release note: none
This commit modified how a child conn executor is initialized under internal executor. We modified the logic of initializing a conn executor under internal executor. If there's a descriptor collection, txn state, job collection and schema change job passed to the internal executor, we let the child conn executor inherit them, instead of creating a new set for itself. Release note: None
Currently, the internal executor always create its own descriptor collections, txn state, job collection and etc. for its conn executor, even though it's run underneath a "parent" query. These recreation can unneccesarily reduce the query efficiency in some use cases, such as when an internal executor is used under a planner context. In this case, the internal executor is expected to inherit these info from the planner, rather than creating its own. To make this rule more explicit, this commit adds a series of query functions under `sql.planner`. Each of these functions wrap both the init of an internal executor and the query execution. In this way, the internal executor always stores the info inherited from the parent planner, and will pass it to its child conn executor. fixes cockroachdb#69495 Release note: None
…Executor() This commit introduces two functions that allow users to run sql statements with an internal executor. We intend to limit the usage of a real internal executor only inside these functions, instead of free-floating or hanging off certain structs. In other words, we restrict the init of an internal executor. The motivation is that if an internal executor is used to run multiple sql statements in a txn manner, these executions are expected to use the same set of info (such as descriptor collections) among their conn executor. While this rule can be easily forgot by the user of internal executors. Hence we provide an interface that wraps the initialization of internal executors with the query executions, so that the users won't need to be worried. Informs: once all existing usages of the internal executors are replaced with the new interfaces proposed here, cockroachdb#70888 should be solved. Release note: None
This commit provide an example to refactor the current use cases with the new internal executor interfaces. In this example, originally, the internal executor was used with a nil txn. We now replace it with ieFactory.RunWithoutTxn(). Release Note: None Release note (<category, see below>): <what> <show> <why>
….WithTxn() This commit is to provides example to refactor the usages of internal executor with the new interfaces. Idealy, if a planner is involved, use the query functions for `sql.planner`. Otherwise, if the query is to run with a not-nil txn, we should use collectionFactory.WithTxn(). Release note: None
3df480e to
afb5db4
Compare
|
Thanks all for reviewing! |
|
Build failed (retrying...): |
|
Build succeeded: |
|
|
||
| // RunWithoutTxn is to create an internal executor without binding to a txn, | ||
| // and run the passed function with this internal executor. | ||
| func (ief *InternalExecutorFactory) RunWithoutTxn( |
There was a problem hiding this comment.
Why was this method necessary / how is it different from NewInternalExecutor() ? Can't the caller run exactly the two lines inside this function?
The comment talks about "binding to a txn" without other explanations. I don't think either fewer or more words would be needed since most callers don't care about that / don't know what "binding to a txn" means.
There was a problem hiding this comment.
I agree that this function is far from ideal... We made this function with the hope of avoiding use cases where an internal executor is created without binding to any txn-related metadata, but is used to run queries with a not nil-txn. In other words, we wanted to make the usages "with" and "without" an outer txn more distinct from each other, and let callers think twice about which one they should use.
I think we can add a comment saying that it's disallowed to use this function to run DDLs or multiple statement in a transactional manner.
There was a problem hiding this comment.
You can have a NewInternalExecutorWithoutTxn if you insist on the importance of having the "without part" in your face, but I don't see why the caller needs to structure its logic into a closure if they're not getting anything in return.
There was a problem hiding this comment.
I think if we only have NewInternalExecutorWithoutTxn, it can still happen to the caller to use it to run statements with a not-nil txn, which is wrong. To wrap it in this function is to make it more explicit that you shouldn't do this (though it's true that we can't truly disallow it here)
I think the ideal case is to remove the txn field in internal executor's query functions (e.g. ie.QueryRowEx()). The txn should be bound to the internal executor, rather than each statement execution. With that, I think it's totally fine for us to remove this function and just do ie := NewInternalExecutorWithoutTxn ()
This PR aims to provide a set of safer interfaces for the internal executor, making it less easy to abuse.
Currently, each conn executor underneath the internal executor (we call it “child executor”) has its own set of information, such as descriptor collection, job collection, schema change jobs, etc, even when it’s run with a not-nil outer
kv.Txn, or there're multiple SQL executions under the samekv.Txn.This is not intuitive, since it violates a rather deep principle that a
descs.Collectionand a SQL txn have a 1:1 relationship. The code doesn’t enforce that, but it ought to. The more places that make it possible to decouple this, the more anxious we get.Ideally, internal executor with a not-nil txn is either planner or
collectionFactoryoriented, so that the txn is always tightly coupled with the descriptor collection. We thus propose a set of new interfaces to ensure this coupling.Currently, the usage of an internal executor query function (e.g.
InternalExecutor.ExecEx()) falls into the following 3 categories:For usage 1, the descriptor collections, txn state, job collections, and session data from the parent planner are expected to be passed to the internal executor's child conn executor.
For usage 2 and 3, if multiple SQL statements are run under the same txn, these executions should share the descs.Collection, txn state machine, job collections and session data for their conn executors.
To suit these 3 use cases, we proposed 3 interfaces for each of the query function:
(In the following we use
InternalExecutor.ExecExas the example)func (p *planner) ExecExUpdated(), where the internal executor is always initialized withdescs.Collection,TxnStateand etc. from thesql.planner.ieFactory.WithoutTxn(), where the query is always run with a nil kv.Txn.CollectionFactory.TxnWithExecutor(). In this function, the internal executor is generated and passed to the call back function to run the query.We also tried refactoring some of the existing use cases to give an example of the new interface.
(Note that the ultimate goal of this improvement is to deprecate all the "free-hanging"
InternalExecutorobjects (such assql.ExecutorConfig.InternalExecutor) and replace them with anInternalExecutorFactoryfield.InternalExecutorFactoryis to initialize a REAL internal executor, but it cannot be used directly to run SQL statement queries.Instead, we wrap the initialization of an internal executor inside each query function, i.e. init it only when you really need to run a query. In other words, the creation of an internal executor becomes closer to the query running.)
fixes #69495
fixes #78998
Release Note: None