Skip to content
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

distsql: expressions evaluation uses the wrong Txn sometimes #41992

Open
andreimatei opened this issue Oct 29, 2019 · 13 comments
Open

distsql: expressions evaluation uses the wrong Txn sometimes #41992

andreimatei opened this issue Oct 29, 2019 · 13 comments
Labels
C-bug Code not up to spec/doc, specs & docs deemed correct. Solution expected to change code/behavior. T-sql-queries SQL Queries Team

Comments

@andreimatei
Copy link
Contributor

andreimatei commented Oct 29, 2019

Gateway flows will sometimes use the Root txn erroneously for evaluating transactions, when the should be using a Leaf.

The processes of creating the processors/operators in a flow makes a copy of the EvalCtx from the FlowCtx by using NewEvalCtx. This generally happens when processors/operators are instantiated. The EvalCtx contains a *client.Txn. On the gateway, that transaction is initially the Root transaction. However, if there's any remote flows or if the gateway flow has any concurrency, we'll later switch the transaction that the gateway flow uses to a Leaf. We're switching FlowCtx.Txn and FlowCtx.EvalCtx.Txn. Unfortunately,
Unfortunately, by the time we switch that txn, all the processors have already captured the old one for the purposes of expression evaluation. This is bad because the gateway flow can have concurrency in it, and RootTxns don't support concurrency (#25329). Worse, RootTxn cannot be used in conjunction with remote leaves (this is another type of concurrency, really) because the Root might refresh at an inopportune time causing write skew.

The txn is used during expression evaluation by some built-in function - many pg_ ones use it to query system tables using the internalExecutor.

I think the solution here is to pass the transaction explicitly to the builtins that need it (one way or another). In fact I think it'd be a good idea to take the txn out of the EvalCtx completely and leave the EvalCtx for immutable session attributes.
This is related to #15670 which complains that the context.Context that captured through the EvalCtx and used by expression evaluation is similarly the wrong one.
It is also related to #41222, which complains that we might be missing to collect the metadata of transactions used for expression evaluation. This all speaks to the fact that expression evaluation was kinda bolted onto DistSQL processors without sufficient smoothing.

Jira issue: CRDB-5401

@andreimatei andreimatei added the C-bug Code not up to spec/doc, specs & docs deemed correct. Solution expected to change code/behavior. label Oct 29, 2019
@knz
Copy link
Contributor

knz commented Oct 29, 2019

In fact I think it'd be a good idea to take the txn out of the EvalCtx completely and leave the EvalCtx for immutable session attributes.

Very good idea.

@yuzefovich
Copy link
Member

I started thinking about this issue, and here are some thoughts. I think we want to change flowinfra.Flow interface so that ConcurrentExecution could be called at any time. The set up of the flows would look as:

f := newFlow()
isConcurrent := f.ConcurrentExecution(flowSpec, opt)
// Figure out txn based on whether the flow is local only and 'isConcurrent'
...
f.Setup(..., txn)

ConcurrentExecution will take in (*execinfrapb.FlowSpec, flowinfra.FuseOpt) and it will perform a lightweight "setup" of the flow to figure out whether there is any concurrency. For vectorized flow (at the moment) this check is very easy (since the only source of concurrency is ParallelUnorderedSynchronizer). For row-based flows we will need to extract logic of fuse() function from setupProcessors.

This will allow us to figure out what txn the flow should be using before Flow.Setup is called. The I'm not sure whether we want to

  1. have the flow itself store *client.Txn and have a reference to the flow from the processors/operators that need access to txn
    or
  2. instantiate processors/operators so that they hold on to *client.Txn themselves

(I'm thinking that we probably should do option 2 with the flow also having a reference to txn, but without processors/operators having a reference to the flow.)

This will allow us to remove Flow.SetTxn method from the interface.

The main complication of fuse() function refactor I see is how we could figure out whether a processor implements execinfra.RowSource interface without actually instantiating the processor - we would probably need to have a map from ProcessorSpec to a boolean, but this can be error-prone. We could also do a "heavyweight" setup in ConcurrentExecution (similar to how we do SupportsVectorized check) - we would do the full setup (with a nil txn) to see whether all processors have been fused. It seems less error-prone but also less performant.

Thoughts?

@knz
Copy link
Contributor

knz commented Nov 30, 2019

I think the solution here is to pass the transaction explicitly to the builtins that need it (one way or another). In fact I think it'd be a good idea to take the txn out of the EvalCtx completely and leave the EvalCtx for immutable session attributes.

I think you're on to something here but this needs to be thought about a little more. It would become performance problem to multiply the number of arguments (and overall number of bytes on the stack) threaded at every level of the Eval() call stack. It's also data duplication which is not warranted if we ever intend to perform multiple evaluations in parallel.

Here's a more thorough analysis: the EvalContext struct today carries data from multiple levels:

Scope Examples
cluster-global ClusterID, NodeID
gateway-global Locality,
session-scoped SessionData, SessionAccessor, SessionBoundInternalExecutor, Sequence, DB
SQL txn-scoped TxnState, TxnReadOnly, TxnImplicit, TxnTimestamp
KV txn-scoped Planner, Txn
statement-scoped StmtTimestamp, Placeholders, Annotations, SkipNormalize
execution node TestingKnobs, Mon, CollationEnv, ReCache
specific scalar evaluation Context
Scope On gateway, should be initialized when? On gateway, currently initialized
cluster-global Once per node Once per statement (BAD)
gateway-global Once when gateway node initializes Once per statement (BAD)
session-scoped Once when session is opened Once per statement (BAD)
SQL txn-scoped On gateway by SQL executor upon txn change Once per statement (BAD)
KV txn-scoped On gateway by SQL executor upon KV retry events Once per statement (BAD)
statement-scoped Once per statement Once per statement (ok)
Scope Should be copied in every distsql flow spec? Currently copied in distsql flow specs?
cluster-global no no (ok)
gateway-global no - should be shared by all flows issued on the same exec node on behalf of the same gateway yes (BAD)
session-scoped no - should be shared by all flows issued on the same exec node on behalf of the same session yes (BAD)
SQL txn-scoped no - should be shared by all flows issued on the same exec node on behalf of the same session yes (BAD)
KV txn-scoped no no (ok)
statement-scoped no - should be shared by all flows issued on the same exec node on behalf of the same statement yes (BAD)
execution node no no (ok)
specific scalar evaluation no no (ok)
Scope On exec node, should be initialized when? Currently initialized when
cluster-global Once per node Once per flow (BAD)
gateway-global The first time a flow is set up on behalf of a gateway Once per flow (BAD)
session-scoped The first time a flow is set up on behalf of a session Once per flow (BAD)
SQL txn-scoped The first time a flow is set up on behalf of a statement Once per flow (BAD)
KV txn-scoped The first time a flow is set up on behalf of a statement Once per flow (BAD)
statement-scoped The first time a flow is set up on behalf of a statement Once per flow (BAD)
execution node Once per node Once per node (ok)
specific scalar evaluation Once per evaluation Once per flow (BAD)

So here you are.

To start, IMHO merely threading client.Txn through builtins is just heating up the top of the iceberg after it has hit the Titanic already.

Here's a strategy that probably works better:

  1. group the current fields of EvalContext in separate structs, once per scope identified above. With the exception of Context, see below.
  2. in EvalContext, link the new structs by reference.
  3. initialize each struct at the specific scope where needs to be initialized. For clarity in the code, place these initializations in suitably-named separate functions.
  4. on each flow set up, merely link the struct references to the EvalContext instance passed to scalar evaluation, as necessary.

In this story the context.Context is a bit special because it is the only one with the most specific scope, potentially changing on every single scalar eval. This is the one which, IMHO, can stay in the EvalContext struct.

However if we ever consider running multiple scalar evaluations for the same statement in parallel in the future, we may want to share a single EvalContext instance in memory but with different context.Context instances. This is when it may become beneficial to thread context.Context through all the Eval calls.

@jordanlewis
Copy link
Member

Nice analysis. Here's a potential counterpoint: per-row Eval already has performance problems that we've solved by moving to the vectorized engine. Adding a txn to Eval therefore shouldn't matter, as we gradually implement all builtins that are important for performance in the vectorized model.

Is this mutually agreeable? If so, I would argue that it's not a big deal to thread an extra argument through. But if you say it's still disagreeable that's fine. I just want us to move away from using "performance of Eval" as an acceptable argument whenever possible.

@andreimatei
Copy link
Contributor Author

I don't have a strong opinion on whether the txn is passed explicitly or implicitly to builtin functions. That object feels special to me, different from anything else in the EvalContext, because it allows the builtins to do arbitrary things. Most builtins don't need it, and so I'd rather have most builtins have a simpler interface that doesn't require it. It seems that in various contexts it's not clear what transaction to pass to these functions (e.g. when backfilling after schema changes, in tests), and so I'd rather have most functions not worry about it - which means both taking it out of the EvalContext and also out of the argument list (except for builtins that need it). But, I haven't rationalized this fully, hence the no strong opinion part.
I don't buy the cost of passing the pointer very much particularly with the vectorized engine (but also I don't want the argument at all for most functions, as I was saying).

I have a broader question than all of this, though. Do we actually ever need the transaction in builtins (or, at least, do the 18 pg_ builtins that currently use it really need to use it)? What is it exactly that needs to be done in the same transaction as the one running the statement? In other words, what writes from said txn need to be seen by the queries done by the builtins? Cause if there's no good reason to use it, I'd rather not use it at all. Is it about seeing dirty schema elements? Cause for that, perhaps we could use a more restricted schema accessor interface.

@knz
Copy link
Contributor

knz commented Dec 2, 2019

So during my travel I thought of something which changed my position a little. See below.

Replying to Andrei first:

do the 18 pg_ builtins that currently use it really need to use it

yes - they inspect the schema descriptors via vtables.

what writes from said txn need to be seen by the queries done by the builtins?

DDL and privilege grants.

perhaps we could use a more restricted schema accessor interface.

The builtins issue SQL queries against pg_catalog tables. Good luck virtualizing all this. (The complexity is not warranted IMHO)

Most builtins don't need it, and so I'd rather have most builtins have a simpler interface that doesn't require it.

I like this argument too.


New learnings on my side:

  • following Jordan's lead I will drop the performance argument for a moment. Still, for the same reason as Andrei I think there is something fundamental about engineering principles here: we should not incur an overhead for the reader of the code in the general case when the benefit is only incurred exceptionally.

  • I also now think it would be beneficial to split the client.Txn away from EvalContext (with no comment yet on the argument passing, see next point fo rthat). The reason for splitting client.Txn is to enable passing a different instance of client.Txn to different distsql processors. @ajwerner and I today realized this would be the mechanism needed for sql: mixed timestamp queries #39275.

  • based on the previous point, for me the question is not any more whether we should split Txn away (yes we should), but rather how, and what is the better solution than an extra argument throughout the call stack (which I don't like, and not just for performance reasons, see above).

Here's an approach. Look at runParams in the sql package. This does not provide any different run-time behavior than 3 separate arguments (ctx, planner, evalCtx) but it combines them visually, so that the reader of the code does not need to know there are 3 things when only just 1 is used in the Next() functions.

So here I'd suggest something like this. an evalParams struct containing both an *EvalContext and a *client.Txn side-by-side. WDYT?

@andreimatei
Copy link
Contributor Author

andreimatei commented Dec 3, 2019

Simply moving the txn alone inside an arguments struct wouldn't make much differences, in my opinion. It's just cosmetics. I think I'd rather have two different interfaces for builtin functions - one simple, one more "extended".
Short of that, different processors already use some sort of copies of the EvalCtx (see FlowCtx.NewEvalCtx()) and so they can override the txn there I think.
On the other hand, if we manage to split the fields of EvalCtx by scope (along the lines you were listing before) or degree of constness (in particular, if we manage to split the iVarContainerStack that NewEvalCtx() mucks with together with the txn), that would seem like an improvement to me indeed.

@knz
Copy link
Contributor

knz commented Dec 3, 2019

Simply moving the txn alone inside an arguments struct wouldn't make much differences, in my opinion. It's just cosmetics

What you call "cosmetics" is another team's cognitive overhead or the avoidance thereof.

On the other hand, if we manage to split the fields of EvalCtx by scope (along the lines you were listing before) or degree of constness (in particular, if we manage to split the iVarContainerStack that NewEvalCtx() mucks with together with the txn), that would seem like an improvement to me indeed

I certainly would support that work too! But maybe there's something in-between where we only extract part of the fields into a differently scoped struct.

Also I'd like to underline that once there are multiple differently-scoped structs we'll need another struct to hold these references and pass them around, which is pretty much what I was suggesting above.

@github-actions
Copy link

github-actions bot commented Jun 4, 2021

We have marked this issue as stale because it has been inactive for
18 months. If this issue is still relevant, removing the stale label
or adding a comment will keep it active. Otherwise, we'll close it in
5 days to keep the issue queue tidy. Thank you for your contribution
to CockroachDB!

@knz
Copy link
Contributor

knz commented Jun 4, 2021

still current

@github-actions
Copy link

We have marked this issue as stale because it has been inactive for
18 months. If this issue is still relevant, removing the stale label
or adding a comment will keep it active. Otherwise, we'll close it in
10 days to keep the issue queue tidy. Thank you for your contribution
to CockroachDB!

@knz
Copy link
Contributor

knz commented Sep 19, 2023

@yuzefovich @rharding6373 @DrewKimball I think we fixed this right?

@yuzefovich
Copy link
Member

I don't think so - Jane and Andrew did some work around the internal executors to tie them to a concrete txn, so perhaps this issue has been mostly addressed. However, this space seems quite fragile and deserves more attention at some point, and this issue has lots of useful of context, so I'll keep it open.

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
C-bug Code not up to spec/doc, specs & docs deemed correct. Solution expected to change code/behavior. T-sql-queries SQL Queries Team
Projects
Status: Backlog
Development

No branches or pull requests

5 participants