Error semantics—evaluation order cleanup
Overview
Evaluation order and short-circuiting can affect whether some potential error during query execution 1) actually happens and 2) is actually surfaced to the user. This can cause surprises in many cases (to users, to certain test frameworks, and even to our own optimizer code), and is outright wrong in other cases.
Frank's Take
(From here.)
The high level problem imo is that we do not have semantics for errors; they are instead a byproduct of (render.rs) code. SQL doesn't appear to have strong opinions about errors either; evaluation order is unspecified, and it is hard to be certain on what basis you are allowed / not to hallucinate data and error on it before you filter it out. For example, we have similar issues in decorrelation, where we may compute predicates on entire relations and then join with the keys of interest, perhaps already erroring on rows that would not have been explored in the most common interpretation of how subqueries work.
Alternately: we should have semantics for errors, and someone should figure out what strong opinions SQL has about them. I'm not sure where to find that function in the org (deep understanding of the SQL spec).
Links to some earlier discussions
- Nikhil saying that we can be quite liberal in what we allow the optimizer to do: https://github.com/MaterializeInc/materialize/issues/2418#issuecomment-604791188
- https://materializeinc.slack.com/archives/C02PPB50ZHS/p1673960928197939?thread_ts=1673954403.292819&cid=C02PPB50ZHS
Tasks
(I organized tasks into 1., 2., ..., but this is kind of a naive ordering that won't reflect our actual order of working on these: actually, what we find when looking into 3. and 4. will have to heavily influence our decisions in 1.)
1. Decide what should be our error semantics at the SQL level (which should be stable across optimizer transforms)
Look around
- [ ] We should check the SQL standard.
- [ ] We should check Postgres.
- The relevant Postgres docs: https://www.postgresql.org/docs/9.0/sql-expressions.html#SYNTAX-EXPRESS-EVAL
- And a relevant Postgres mailing list thread: https://www.postgresql.org/message-id/flat/18400-1358382426-292278%40sneakemail.com#5094932246b593dbb7679a47b1b2c3b0
- A weird case in Postgres that would be hard to follow: https://github.com/MaterializeInc/materialize/issues/2418#issuecomment-604791188
Semantics questions
Note that for each of the questions listed below there are three theoretical options for what the answer might be:
- The error must not be shown.
- The error must be propagated upwards.
- It's unspecified whether the system will propagate the error. (The usual constraints on unspecified behavior apply: can change between mz versions; can't be different between replicas running at the same time.) Note that even when it's strictly speaking unspecified, we might still make a reasonable effort to either show or hide the error. This is what we currently actually do in many cases, e.g., by checking for literal errors in
PredicatePushdownbut missing function calls that might error at run time (see below at "Special-casing literal errors"). In some cases, we can do this by harmonizing various evaluation code paths, see at 3. in the task list below.
In many cases, we'll have to choose "unspecified", because guaranteeing to show certain errors would incur an unacceptable performance hit.
A probably incomplete list of specific things we need to think through:
- [ ] 1.1 Order of expressions in a WHERE clause where some expressions might error (e.g.,
WHERE 2 + 2 = 4 OR 5 / 0 = 6)- 1.1.1 Do we allow reordering? This has to be yes...
- 1.1.2 Do we allow short-circuiting to hide errors that would have happened in later conjuncts? Hard to imagine how we would implement "no" without a performance hit. But if we say yes, and we allow reordering in the previous point, then an error in any conjunct might be hidden by short-circuiting, even the first one. This is sad, because then we can't have guarded errors with simple boolean operators, but I don't see a way out of this, and I'm guessing other databases do the same. (However, CASE WHEN should probably function as a guard, see below.)
- [ ] 1.2 Errors in records that will be Filtered out in a later WHERE clause before reaching the root: If we don't allow a later Filter to filter out a Row that involves an error, then PredicatePushdown can change error semantics by pushing a predicate below some node that errors if the predicate filters out the data that causes the error.
- [ ] 1.3 Projected away errors. We should definitely make this unspecified, right?
- [ ] 1.4 Join, where one input errors, but an other is empty. One could argue that we shouldn't propagate the error in this case, but currently we do in both const folding and rendering, see https://github.com/MaterializeInc/materialize/issues/20769
- [ ] 1.5 COUNT, where one of the input records is an error.
- [ ] #19662
- [ ] 1.6 CASE WHEN (and possibly other special scalar expressions) should probably be a special-case, because we don't want to make this unspecified. See below at
MirScalarExpr::If. (Note that Postgres has two limitations when using CASE for guarding errors, but I think we don't have those limitations.) The following issues seem to be violations of this:- [ ] https://github.com/MaterializeInc/materialize/issues/20199
- [ ] https://github.com/MaterializeInc/materialize/issues/19570
- [ ] https://github.com/MaterializeInc/materialize/issues/19594
- [ ] ~~https://github.com/MaterializeInc/materialize/issues/20769: What might have happened here is that
PredicatePushdownpushed anErrorIfNullthat was originally after a CASE WHEN to be executed in some place where the CASE WHEN was not able to guard it.~~ Edit: No, this was something else, see PR that fixed it. - [ ] ~~Also, we have the problem that if MFP CSE pulls out something from the CASE WHEN, then the thing won't be guarded. (Might be the root cause of any of the above issues as well.)~~ Edit: Actually, a code comment is saying that this shouldn't happen. But we should verify this.
- [ ] Since https://github.com/MaterializeInc/materialize/pull/20991, window functions are always pulled out of a CASE WHEN, so an error inside a window function is currently not being guarded by a CASE WHEN that is outside of the window function call.
- [ ] 1.7 Subqueries
- [x] #5670
- [ ] The interaction between the "more than one record produced in subquery" pattern and PredicatePushdown
- [ ] https://github.com/MaterializeInc/materialize/issues/20199
- [ ] https://github.com/MaterializeInc/materialize/issues/20769
- [ ] 1.8 When can errors cancel each other out?
- [ ] 1.8.1 If the query has an EXCEPT / EXCEPT ALL, can errors cancel each other out? E.g., if different records are being divided by zero in the different branches, then they definitely shouldn't cancel each other out. But what if they are referring to the same (local or global) id that has some error. It's easy to say that even these errors should never cancel each other out, but the problem is that we might internally rely on them cancelling each other out.
- [ ] 1.8.2
NOT IN? https://materializeinc.slack.com/archives/C02PPB50ZHS/p1697020517291619?thread_ts=1696944133.205489&cid=C02PPB50ZHS - [ ] 1.8.3
NOT EXISTS? https://materializeinc.slack.com/archives/C02PPB50ZHS/p1697020517291619?thread_ts=1696944133.205489&cid=C02PPB50ZHS
- [ ] 1.9
(Error ...) IS NULL- [ ] #18346
- [ ] 1.10 Null propagation vs. error propagation: When we have a scalar function call with some of the arguments being constants, both seeing a constant error in one of the arguments and seeing a constant null in one of the arguments can lead to a simplification, but to different ones. Which one should take precedence when both are possible?
- [ ] I think this is this issue: #2418
- [ ] Maybe also: https://github.com/MaterializeInc/materialize/issues/17189#issuecomment-1620330425
- [ ] 1.11 Semantics of invalid accumulations, see https://materializeinc.slack.com/archives/C02PPB50ZHS/p1675868356176619?thread_ts=1675864117.612589&cid=C02PPB50ZHS
- [ ] https://github.com/MaterializeInc/materialize/issues/20111
- [ ] 1.12 Reordering numerical expressions with changing overflow behavior. E.g., reordering
x > mz_now() - ytox + y > mz_now(): https://materializeinc.slack.com/archives/C02PPB50ZHS/p1686668220302679?thread_ts=1686243455.580009&cid=C02PPB50ZHS - [x] 1.13
FILTERclause, e.g.:SELECT COUNT(1/x) FILTER (WHERE x != 0): this currently translates to a CASE WHEN, so one could say that this guard should work, but Nikhil says this is not important. - [ ] 1.14
LIMIT 0: CurrentlySELECT * FROM (SELECT * FROM t LIMIT 0)swallows the error, whileSELECT * FROM t LIMIT 0returns it. Also note that this is not just a theoretical problem, because sometimes people wanted to useLIMIT 0to observe whether a mat view or source or something has errors, and the current behavior makes this fragile, because external tools sometimes add an extraSELECT * FROM (...) LIMIT xat the outer level.- [ ] More generally, do we want
LIMIT nto error only if the firstnrecords have an error? - [ ] See also the Postgres inconsistency in https://github.com/MaterializeInc/materialize/issues/26754: Postgres does not error on
LIMIT 0but does onLIMIT 1even if the problematic row is not part of the result set.
- [ ] More generally, do we want
2. Document 1.
- [ ] #19992
- [x] #19789
- (focus on the last point)
3. Look into the error semantics of each of our various evaluation code paths.
In some places we will adjust code, in other places we might need to just verify that it already does what it should according to what we decide in 1. Additionally, we will have to keep all of this list in mind when working on 4. from below.
One thing we should strive for is to make the evaluation of the same thing in different contexts (e.g., normal evaluation vs. constant folding) have the same semantics. This is not going to solve all our problems by itself, but makes life easier when working on 4.
- [ ] scalar functions
- [ ] AND/OR: Probably fine, see here
- [ ] (Check the lazy and eager macros.)
- [ ]
coalesce- [ ] #4969
- [ ] https://github.com/MaterializeInc/materialize/issues/17189#issuecomment-1798032300
- [ ] Introduce more cases in
UnaryFunc::could_error. How many functions we can make this false for will inform other decisions. - [ ] ... is that all?
- [ ]
MirScalarExpr::If: This should probably make sure to surface errors only from the chosen branch, so that when somebody wants to guard an error, they can use CASE WHEN for that. We will also need this for e.g., the "more than one record produced in subquery" pattern that our lowering introduces. Note that we do make some amount of effort already to do this, e.g., both the decorrelation and MirScalarExpr evaluation tries to evaluate a branch only when the condition matches. But there are several issues, see in 1. at CASE WHEN. - [ ] MFP evaluation
- [ ] The implicit ANDing of
MirRelationExpr::Filter's predicates. - [ ] Map expressions that will get projected out.
- [ ] The implicit ANDing of
- [ ] Different evaluations of aggregations:
AggregateFunc.evalcan evaluate any aggregation, butrender/reduce.rsis not using this fn for all aggregations, but instead does its own magic forAccumulableandHierarchicalPlan::MonotonicPlanones. We should make sure thatrender/reduce.rs's implementations matchAggregateFunc.eval, because even those aggregations that are specially handled byrender/reduce.rsare handled byAggregateFunc.evalin other contexts: This is currently (at least)FoldConstantsandLiteralLifting, but later it will be also window aggregations.- [ ] #17758
- [ ] https://github.com/MaterializeInc/materialize/issues/23336
- [ ] Evaluation of join equivalences
- [ ] Constant folding (there are lots of consistency checks in
test/sqllogictest/transform/fold_vs_dataflow)- [ ] Follow-ups from https://github.com/MaterializeInc/materialize/pull/17796#issuecomment-1441881016 (e.g., https://github.com/MaterializeInc/materialize/issues/17189#issuecomment-1623643909)
- [ ]
MirScalarExpr::reduce- [ ] Currently, a
CallBinarywhose any argument is a literal error is replaced by a literal error. - [ ]
CallVariadicalso does this, except forcoalesce. - [ ] #2418
- (I think this is the null propagation or error propagation should take precedence from 1.) Note that this is not so simple as deciding which
ifto have first, the literal null detection or the literal error detection: the problem is not just these ifs "racing" with each other, but that they are also racing with the normal (non-const-folding) evaluation paths!
- (I think this is the null propagation or error propagation should take precedence from 1.) Note that this is not so simple as deciding which
- [ ] https://github.com/MaterializeInc/materialize/issues/17189#issuecomment-1620330425 seems also like a null propagation vs. error propagation in const folding vs. in rendering.
- [ ]
coalesce- [ ] #4969
- [ ] https://github.com/MaterializeInc/materialize/issues/17189#issuecomment-1798032300
- [ ]
MirScalarExpr::If: probably ok? Still need to check the normal evaluation. - [ ] There are various function specializations which error out when the partial evaluation fails. This is ok, right? (Btw. which item from 1. covers this?)
- [ ] Currently, a
- [ ]
FoldConstants: We should check all the cases in that big match. Some notable ones:- [ ]
Reduce: see above atAggregateFunc::eval. - [ ]
Filter: This is currently very weird: It doespredicates.sort_by_key(|p| p.is_literal_err());. This is probably not wrong, but whatever it is trying to solve, it is not completely solving, see below at "Special-casing literal errors". To put it in an other way, if not having this line would introduce some issue, then we probably have that same issue in a weaker form already even when having this line. (This would also be a good opportunity for testing: comment out this line and see if something breaks.) - [ ]
Union: Does the exact opposite of Filter: if there is a literal error in any branch, then it fires that error. (This might be wrong depending on what we decide in 1., because this error might have been cancelled out by errors in an other, non-const branch that we wipe away. Although note that the current rendering code of Negate doesn't negate the error stream.) - [ ]
Join: if there is a literal error in any branch, then it fires that error. This might be wrong, depending on what we decide in 1.: If an other join input is empty, do we want to fire this error? (Note that theife.is_empty()doesn't do much to mitigate this, because it might very well be a non-const empty, which won't be detected by that if.) (PredicatePushdown is struggling with the same issue, see below.)
- [ ]
- [x] interpret.rs (It's currently conservative, which seems perfectly fine for the abstract interpreter.)
- [ ] #17758
- [ ] Look into how the rendering of various operators handle the error stream.
- [ ] E.g.,
Negatedoesn't negate the error stream! This means that an erroring Filter can't be pushed past a Negate, which is very sad. But unfortunately, we can't change this to make it negate, see here: Marcos: "it is not clear to me that the rendering should negate the error stream. Suppose you have A - B where a division by zero occurs in A and another one occurs in B. Wouldn’t we be at risk of not reporting this error?" (See 1.8 above.) - [ ] We might have to change Join to not propagate an error from one input if some other input is empty.
- [ ] ... Are there more similar mines hidden?
- [ ] E.g.,
4. Adjust optimizer transforms to not change error behavior when transforming a plan in ways that would violate the error semantics.
This might be the hardest part, maybe even impossible in some cases without either a big performance hit, or a substantial rewrite of rendering's error handling, or some complicated tricks in the optimizer.
- [ ] This is definitely a bug regardless of what we decide in 1.:
- [ ] #9284
- [ ] Special-casing literal errors: Various transforms treat literal errors as barriers in situations where any function call that
can_errorcan cause a similar problem as a literal error. It's unclear why only literal errors are barriers and not any function call thatcan_error.- [ ]
PredicatePushdown- [ ] #20003
- This is a bug regardless of what we decide in 1., because https://github.com/MaterializeInc/materialize/issues/19179 might happen with not just literal errors.
- [ ] https://materializeinc.slack.com/archives/C02PPB50ZHS/p1675864117612589
- [ ] #20003
- [ ] Is
JoinImplementationlifting up an erroring mfp ok? Again, it tries to avoid lifting literal errors, but this might not be enough if we are aiming here to provide any sort of meaningful guarantee. - [ ] All uses of
extract_non_errors_from_expr,extract_non_errors_from_expr_ref_mut, andextract_non_errors_from_expr_mut(17 uses in total) are also sus of the above issue of specially handling only literal errors where the same special handling might be required for any expression that can error...
- [ ]
- [ ]
ReduceElisionhttps://github.com/MaterializeInc/materialize/issues/19662#issuecomment-1573517182 - [ ]
NonNullRequirementscan eliminate a constant null row which would have triggered an error if not eliminated, see https://materializeinc.slack.com/archives/C02PPB50ZHS/p1686919212265659?thread_ts=1686919149.040909&cid=C02PPB50ZHS - [ ]
NonNullable"turns any IsNull(Col(X)) into literal false, but it shouldn't if it is in an If::cond because it could derive from predicate behavior, or itself be an expression that can migrate below whatever constraint introduced it." (Frank) - [ ]
LiteralLiftingand other transforms that move scalar exprs around https://materializeinc.slack.com/archives/C02PPB50ZHS/p1673962206660309?thread_ts=1673954403.292819&cid=C02PPB50ZHS - [ ] ... there are probably other cases I haven't yet thought about
(FoldConstants is discussed above in 3.)
5. Adjust our "more than one record produced in subquery" pattern.
- [ ] #19993
Instead of relying on shady Filter evaluation order rules, we should use a construct that has a well-defined evaluation order, e.g., IF-THEN-ELSE. Maybe this will solve https://github.com/MaterializeInc/materialize/issues/9284 ? Also, fixing this would unblock https://github.com/MaterializeInc/materialize/issues/17142.
But wait, https://github.com/MaterializeInc/materialize/issues/20199 might mean that currently IF-THEN-ELSE is not actually able to fulfill its role of rock-solidly guarding errors. That issue should be investigated first.
6. Adjust test frameworks to not alert on things that are in spec for whatever we decide for 1.
- [ ] SQLancer (and maybe SQLsmith?): e.g., https://github.com/MaterializeInc/materialize/issues/18346
- [ ] We have some automated tests for constant folding consistency with normal evaluation. A case mentioned here: https://github.com/MaterializeInc/materialize/issues/19662#issuecomment-1573618794
We should also check the all_errors stuff in PredicatePushdown.
Also, check https://materializeinc.slack.com/archives/C02PPB50ZHS/p1675864117612589
The relevant Postgres docs: https://www.postgresql.org/docs/9.0/sql-expressions.html#SYNTAX-EXPRESS-EVAL This is a really thorough explanation of all these issues in Postgres.
(And a relevant Postgres mailing list thread: https://www.postgresql.org/message-id/flat/18400-1358382426-292278%40sneakemail.com#5094932246b593dbb7679a47b1b2c3b0 )
#17796 takes a big step towards eliminating the confusing error semantics of our logical operators, but there are some follow-up todos there: https://github.com/MaterializeInc/materialize/pull/17796#issuecomment-1441881016
More instances of this, frustrating nightly runs of SQLancer: https://github.com/MaterializeInc/materialize/issues/18346
Philip has some suggestions there for addressing some specific instances of the problem.
Some earlier discussions (collected by Philip):
- https://github.com/MaterializeInc/materialize/issues/5670
- https://github.com/MaterializeInc/materialize/issues/4969
- https://github.com/MaterializeInc/materialize/issues/2418
- https://github.com/MaterializeInc/materialize/issues/9284
!! https://materializeinc.slack.com/archives/C02PPB50ZHS/p1683899532068649
Came from https://github.com/MaterializeInc/materialize/issues/19179, but it's a more general problem.
For the issue in the previous comment: We have a couple of checks in PredicatePushdown to not push literal errors past certain operators. (And I will now add one more for Union to address https://github.com/MaterializeInc/materialize/issues/19179.) However, these are just band aids as far as I can see, because the same problem can occur when pushing down any function call that can error. This is not obvious to solve, because we would lose too many pushdowns if we simply not push all these. However, we might have a rather tricky solution (from a discussion with @mgree and @aalexandrov):
- We push down a modified version of the function call, which produces a null instead of erroring. Edit: Or maybe some other null-like special value, to avoid IS NULL picking up the errors. (Similarly to this situation.)
- Additionally, we leave the original function call at its original place.
By this, we can have our cake and eat it too: 1. means that we get almost all of the benefits of a traditional pushdown and we don't get spurious errors from an early evaluation, and 2. means that any of the real, original errors we still properly get.
By this, we can have our cake and eat it too [...]
Not sure if this is entirely true, though. We still have to leave the original predicate in place, which definitely will have consequences for downstream optimizations (in a sense, the term gets more complicated in case of simpler / closer to a canonical form).
Btw. basically all uses of extract_non_errors_from_expr, extract_non_errors_from_expr_ref_mut, and extract_non_errors_from_expr_mut are also sus of the above issue of specially handling only literal errors where the same special handling might be required for any expression that can error.
From #19378:
As part of evaluating an MFP's filters to determine whether it will filter out an entire set of rows, we use AND to combine the results of multiple filters. However, MFP's filter evaluation is strict: if any filter errors the overall evaluate call will fail.
One more tricky case: https://github.com/MaterializeInc/materialize/issues/19662
count manages to get lazy and swallow an error.
As part of evaluating an MFP's filters to determine whether it will filter out an entire set of rows, we use AND to combine the results of multiple filters. However, MFP's filter evaluation is strict: if any filter errors the overall evaluate call will fail.
Actually, it's more complicated than that, more details here.
I've incorporated everything from the above comments into the issue description or into other issues that are linked from the issue description.
Another one. @ggevay, please make sure it is included in the issue description.
string_agg(NULL, chr(-32768)) fails with dataflow-rendering and shortcuts to NULL with constant_folding.
Reproduction:
DROP TABLE IF EXISTS dataflow_rendering;
CREATE TABLE dataflow_rendering (row_index INT, int2_neg_max INT2, int4_null INT4, int4_one INT4, text_null TEXT);
INSERT INTO dataflow_rendering VALUES (0, -32768::INT2, NULL::INT4, 1::INT4, NULL::TEXT);
CREATE OR REPLACE VIEW constant_folding (row_index, int2_neg_max, int4_null, int4_one, text_null)
AS SELECT 0, -32768::INT2, NULL::INT4, 1::INT4, NULL::TEXT;
SELECT
string_agg(NULL, chr(int2_neg_max) ORDER BY row_index)
FROM
dataflow_rendering;
SELECT
string_agg(NULL, chr(int2_neg_max) ORDER BY row_index)
FROM
constant_folding;
Thanks, added it to the "Null propagation vs. error propagation" category.
@ggevay: A further case for the sake of completeness.
It is known that constant folding produces a result in some cases where dataflow-rendering fails because of shortcuts that constant folding takes. However, it can also be the case that dataflow-rendering succeeds and constant folding fails because of a different evaluation order in the where clause.
Given the where clause WHERE (row_index IN (3, 5)) AND (interval_val = justify_days(interval_val)),
- dataflow-rendering succeeds in my examples because it applies the first condition (
row_indexcolumn filtering) first - constant folding fails in my examples because it does not start with the
row_indexcondition and rows with indices notIN (3, 5)contain values that causejustify_days(interval_val)to fail
Reproduction:
DROP TABLE IF EXISTS dataflow_rendering;
CREATE TABLE dataflow_rendering (row_index INT, interval_val INTERVAL, text_val TEXT);
INSERT INTO dataflow_rendering VALUES (1, '-178956970 years -8 months -2147483648 days -2562047788:00:54.775808'::INTERVAL, ''::TEXT);
INSERT INTO dataflow_rendering VALUES (3, '2 years 3 months 4 days 11:22:33.456789'::INTERVAL, 'abc'::TEXT);
INSERT INTO dataflow_rendering VALUES (9, '100 months 100 days'::INTERVAL, 'xAAx'::TEXT);
CREATE OR REPLACE VIEW constant_folding (row_index, interval_val, text_val)
AS SELECT 0, NULL::INTERVAL, NULL::TEXT
UNION SELECT 1, '-178956970 years -8 months -2147483648 days -2562047788:00:54.775808'::INTERVAL, ''::TEXT
UNION SELECT 3, '2 years 3 months 4 days 11:22:33.456789'::INTERVAL, 'abc'::TEXT
UNION SELECT 9, '100 months 100 days'::INTERVAL, 'xAAx'::TEXT;
SELECT
char_length(text_val)
FROM
dataflow_rendering
WHERE (row_index IN (3, 5)) AND (interval_val = justify_days(interval_val))
ORDER BY row_index ASC;
SELECT
char_length(text_val)
FROM
constant_folding
WHERE (row_index IN (3, 5)) AND (interval_val = justify_days(interval_val))
ORDER BY row_index ASC;
Thanks, I've added this in the issue description.
Another related issue: #22529.
Further COALESCE example (data-flow rendering evaluates other params and fails at invalid regex, constant folding does not):
DROP TABLE IF EXISTS dataflow_rendering;
CREATE TABLE dataflow_rendering (row_index INT, bool_true BOOL, text_empty TEXT);
INSERT INTO dataflow_rendering VALUES (0, TRUE::BOOL, ''::TEXT);
CREATE OR REPLACE VIEW constant_folding (row_index, bool_true, text_empty)
AS SELECT 0, TRUE::BOOL, ''::TEXT;
SELECT
coalesce(NOT (bool_true), text_empty ~* 'ab(')
FROM
dataflow_rendering;
SELECT
coalesce(NOT (bool_true), text_empty ~* 'ab(')
FROM
constant_folding;
Thx, added a link to your comment to the issue description.
Newer Slack thread on Datum::Error: https://materializeinc.slack.com/archives/C077FUD1F9C/p1718287744657919?thread_ts=1718221293.477699&cid=C077FUD1F9C
Idea for how we could even keep error behavior unchanged across any reorderings:
- For any expression that could error (an "error source"), we'd assign an "error source id" before optimization starts reordering things.
- At the same time, we'd build an "error source precedence graph", which would capture the original relationships between error sources, i.e., before optimization starts reordering things.
- When optimization moves stuff around, the error source ids would move around with the expressions. E.g., every MirScalarExpr node would have an additional field that records the error source id. (Also, every scalar subquery would have an error source id, which would propagate to the error literal expression in
Map (error("more than one record produced in subquery")).) Datum::Errorwould include the error source id.- During evaluation, when there is an operation that has multiple inputs, and more than one input are Datum::Errors, we'd look at their error source ids and consult the error source precedence graph to know which one to propagate.
So then for the (1 / 0) + 'x'::int example, we'd record that errors coming from the (1 / 0) have precedence over the errors coming from the 'x'::int, and then even if the optimizer reorders the inputs of the +, we'd remember these precedences, and propagate the correct error during the evaluation of the +.
Another example would be moving around expressions that were originally in WHERE clauses. Let's say we want to guarantee that for the expression x1/x2 == 6 AND y::int == 7, we want any errors in the first argument of the AND to win over any errors in the second argument. With the above solution, even if the two arguments are pushed down to two different inputs of a join (because, let's say x1 and x2 come from one join input, and y comes from another join input), the Join evaluation would still know which error to propagate when it is putting together an output row from a matching pair of input rows where we have both of the above errors.
In addition to precedences, the error source precedence graph could record other kinds of relationships between expressions: To make CASE WHEN reliably hide errors from branches that officially shouldn't be getting evaluated, we'd record which branch of which CASE WHEN an expression was originally in, and make the CASE WHEN evaluation propagate or suppress errors based on this.
- When optimization moves stuff around, the error source ids would move around with the expressions. E.g., every MirScalarExpr node would have an additional field that records the error source id. (Also, every scalar subquery would have an error source id, which would propagate to the error literal expression in
Map (error("more than one record produced in subquery")).)
Is this "source location information" or something more specific? That is, is each MIR node's error source id unique to it?
In addition to precedences, the error source precedence graph could record other kinds of relationships between expressions: To make
CASE WHENreliably hide errors from branches that officially shouldn't be getting evaluated, we'd record which branch of whichCASE WHENan expression was originally in, and make theCASE WHENevaluation propagate or suppress errors based on this.
What do you mean by this? Turning a CASE WHEN e1 THEN e2 ELSE e3 END into Map (e2, e3, CASE WHEN e1 THEN col_of_e2 ELSE col_of_e3 END followed by projecting the last column? (Which I guess could be useful if e2 and e3 shared subexpressions?)
- When optimization moves stuff around, the error source ids would move around with the expressions. E.g., every MirScalarExpr node would have an additional field that records the error source id. (Also, every scalar subquery would have an error source id, which would propagate to the error literal expression in
Map (error("more than one record produced in subquery")).)
Is this "source location information" or something more specific?
Yes, it's basically source location information.
That is, is each MIR node's error source id unique to it?
Mostly yes, but there will be some anomalies, because in some cases an MirScalarExpr gets copied to multiple locations in the plan. Off the top of my head:
- When a Filter is pushed into a source, a copy also stays above the source.
- https://github.com/MaterializeInc/materialize/pull/14725
- ...
In addition to precedences, the error source precedence graph could record other kinds of relationships between expressions: To make
CASE WHENreliably hide errors from branches that officially shouldn't be getting evaluated, we'd record which branch of whichCASE WHENan expression was originally in, and make theCASE WHENevaluation propagate or suppress errors based on this.
What do you mean by this? Turning a
CASE WHEN e1 THEN e2 ELSE e3 ENDintoMap (e2, e3, CASE WHEN e1 THEN col_of_e2 ELSE col_of_e3 ENDfollowed by projecting the last column? (Which I guess could be useful ife2ande3shared subexpressions?)
Yes, in some cases e2 and e3 get pulled out in this way, for example when they involve window function calls, and this currently messes up the error behavior. We could recover the error behavior by noting down that these things were originally in case branches. Also, CASE WHEN has a non-trivial lowering pattern when the scalar expressions involve (possibly correlated) subqueries, and I think there are bugs around handling errors in these subqueries as well.
Mostly yes, but there will be some anomalies, because in some cases an MirScalarExpr gets copied to multiple locations in the plan. Off the top of my head:
- When a Filter is pushed into a source, a copy also stays above the source.
- https://github.com/MaterializeInc/materialize/pull/14725
- ...
When we copy a node, its source location information should stay the same, if "source" here means the thing the user wrote. The thing I'm not 100% on is if "source location information" and "error provenance information" are the same. In both of these cases, if the duplicated code is going to error in one place, it will error in the other---so it's fine to have the same information. Will that always be the case? The following makes me worried it might not be...
Yes, in some cases e2 and e3 get pulled out in this way, for example when they involve window function calls, and this currently messes up the error behavior. We could recover the error behavior by noting down that these things were originally in case branches.
What do you mean by "noting down that these things were originally in case branches"? I don't know window functions well enough to be confident that "source location" and "error provenance" will be the same.
Further case: Join with join constraint and where condition.
In the following query, constant folding seems to prioritize the WHERE condition in contrast to data-flow rendering, which potentially evaluates the JOIN constraint first.
-- Value 1 (Dataflow rendering): 'QueryFailure' (type: <class 'str'>)
-- Value 2 (Constant folding): 'QueryResult' (type: <class 'str'>)
-- Error 1: 'Evaluation error: "0001-02-03 11:00:00" mz_timestamp out of range'
-- Query 1: SELECT (octet_length(s0.bytea_val) / s1.real_val) FROM t_dfr_vert_0 s0 FULL OUTER JOIN t_dfr_vert_1 s1 ON (s0.timestamp_val = s1.interval_val) WHERE (s0.row_index IN (2, 11, 13)) ORDER BY s0.row_index ASC, s1.row_index ASC;
-- Query 2: SELECT (octet_length(s0.bytea_val) / s1.real_val) FROM v_ctf_vert_0 s0 FULL OUTER JOIN v_ctf_vert_1 s1 ON (s0.timestamp_val = s1.interval_val) WHERE (s0.row_index IN (2, 11, 13)) ORDER BY s0.row_index ASC, s1.row_index ASC;
Hmm, I'm not really sure what is happening in this query.
constant folding seems to prioritize the WHERE condition in contrast to data-flow rendering, which potentially evaluates the JOIN constraint first.
Why does the ordering between the WHERE condition evaluation and JOIN constraint evaluation matter? Both of these should be evaluated before the SELECT clause, and I'd guess the error (a division by 0) happens in the SELECT clause.
Do you have a full repro that I could debug? I'd need to run the optimizer trace tool.
Well, the point is that the evaluation of the join condition fails. Constant folding does not / no longer evaluate that.
That's my assumption based on the error message "0001-02-03 11:00:00" mz_timestamp out of range and usage of interval_val in the join constraint.
Do you still need the full repro in this case?
Ah, sorry, you are right, I missed the error msg somehow.
But I still have no idea how can this happen. I'd expect that the full outer join has to run the join condition first (in both const folding and dataflow), so something mysterious is going on.
But in any case, unfortunately, this is probably in spec: I don't think it's realistic to strive for guaranteeing evaluation order between a join condition and a where clause, because then we'd lose pushdowns.