opt: fix memo cycle caused by join ordering#83875
opt: fix memo cycle caused by join ordering#83875craig[bot] merged 1 commit intocockroachdb:masterfrom
Conversation
mgartner
left a comment
There was a problem hiding this comment.
Great find and explanation! Thanks for getting to the bottom of this one.
This is because the a=d filter should rejects nulls, so the LeftJoin
should have been simplified.
A couple follow-up questions for my curiosity:
- If null rejection was working for this filter, what would the LeftJoin be simplified to?
- Why isn't null-rejection working in this case? Should we fix that at some point too?
Reviewed 4 of 4 files at r1, all commit messages.
Reviewable status:complete! 1 of 0 LGTMs obtained (waiting on @DrewKimball)
pkg/sql/opt/testutils/opttester/reorder_joins.go line 85 at r1 (raw file):
var selString string if len(selRefs) > 0 { selString = fmt.Sprintf(" [select, refs=%s]", jof.formatVertexSet(selRefs))
Should we add a join reordering test that uses this new formatting?
| CREATE TABLE table80901_3 (col3_2 OID, col3_4 FLOAT8, col3_9 STRING); | ||
| ---- | ||
|
|
||
| memo memo-cycles |
There was a problem hiding this comment.
nit: If there is a cycle, the optimizer should panic here and opttester should print out the error in the test output. So maybe the memo-cycles option is unnecessary?
In some rare cases when null-rejection rules fail to fire, a redundant filter
can be inferred in an `InnerJoin` - `LeftJoin` complex. This could previously
result in the `JoinOrderBuilder` attempting to add a `Select` to the same memo
group as its input, which would create a memo cycle. This patch prevents the
`JoinOrderBuilder` from adding the `Select` to the memo in such cases.
What follows is a more detailed explanation of the conditions that could
previously cause a memo cycle.
`InnerJoin` operators have two properties that make them more 'reorderable'
than other types of joins: (1) their conjuncts can be reordered separately
and (2) new conjuncts can be inferred from equalities. As a special case of
(1), an `InnerJoin` can be pushed into the left side of a `LeftJoin`, leaving
behind any `Select` conjuncts that reference the right side of the `LeftJoin`.
This allows the `JoinOrderBuilder` to make the following transformation:
```
(InnerJoin
A
(InnerJoin
B
(LeftJoin
C
D
c=d
)
b=c
)
a=b, a=d
)
=>
(InnerJoin
A
(Select
(LeftJoin
(InnerJoin
B
C
b=c
)
D
c=d
)
b=d // Inferred filter!
)
a=b, a=d
)
```
Note the new `b=d` filter that was inferred and subsequently left on
a `Select` operator after the reordering. The crucial point is that
this filter is redundant - the input to the `Select` is already a
valid reordering of the `BCD` join complex.
The `JoinOrderBuilder` avoids adding redundant filters to `InnerJoin`
operators, but does not do the same for the `Select` case because it
was assumed that the filters left on the `Select` would never be redundant.
This is because the `a=d` filter *should* rejects nulls, so the `LeftJoin`
should have been simplified. However, in rare cases null-rejection does not
take place. Because the input to the `Select` is already a valid reordering,
the `JoinOrderBuilder` ends up trying to add the `Select` to the same group
as its input - namely, the `BCD` join group. This causes a cycle in the memo.
Fixes cockroachdb#80901
Release note (bug fix): Fixed a bug that could cause an optimizer
panic in rare cases when a query had a left join in the input of
an inner join.
DrewKimball
left a comment
There was a problem hiding this comment.
If null rejection was working for this filter, what would the LeftJoin be simplified to?
The LeftJoin should become an InnerJoin because there is an equality filter (null-rejecting) that references its right input.
Why isn't null-rejection working in this case? Should we fix that at some point too?
It's because the query has outer columns. We have to be careful about when we push down IS NOT NULL filters to prevent interactions with decorrelation rules that attempt to pull those filters back up - see #35171. I do have an idea of how to fix it; instead of physically adding an IS NOT NULL filter, we can traverse the operator tree wherever it would be correct to add an IS NOT NULL filter and perform outer-join simplification as we go.
Reviewable status:
complete! 1 of 0 LGTMs obtained (waiting on @mgartner)
pkg/sql/opt/testutils/opttester/reorder_joins.go line 85 at r1 (raw file):
Previously, mgartner (Marcus Gartner) wrote…
Should we add a join reordering test that uses this new formatting?
Good point, I'm surprised there wasn't already one. Done.
pkg/sql/opt/xform/testdata/rules/join_order line 2622 at r1 (raw file):
Previously, mgartner (Marcus Gartner) wrote…
nit: If there is a cycle, the optimizer should panic here and opttester should print out the error in the test output. So maybe the
memo-cyclesoption is unnecessary?
It doesn't seem to set off a cycle in the optimization pattern; only in the structure of the memo. So, you can run without the memo-cycles command and it will happily print the memo without any errors.
|
Previously, DrewKimball (Drew Kimball) wrote…
Do you mean without the fix or with the fix? The panic should occur during optimization like it did in #80901, right? |
mgartner
left a comment
There was a problem hiding this comment.
The LeftJoin should become an InnerJoin because there is an equality filter (null-rejecting) that references its right input.
Oh right. I originally thought that but somehow convinced myself that would not be equivalent.
It's because the query has outer columns. We have to be careful about when we push down IS NOT NULL filters to prevent interactions with decorrelation rules that attempt to pull those filters back up - see #35171. I do have an idea of how to fix it; instead of physically adding an IS NOT NULL filter, we can traverse the operator tree wherever it would be correct to add an IS NOT NULL filter and perform outer-join simplification as we go.
I see. No need to prioritize this, but feel free to document your idea in an issue so we have a record of it.
Reviewed 1 of 1 files at r2, all commit messages.
Reviewable status:complete! 0 of 0 LGTMs obtained (and 1 stale) (waiting on @DrewKimball)
DrewKimball
left a comment
There was a problem hiding this comment.
Reviewable status:
complete! 0 of 0 LGTMs obtained (and 1 stale) (waiting on @mgartner)
pkg/sql/opt/xform/testdata/rules/join_order line 2622 at r1 (raw file):
Previously, mgartner (Marcus Gartner) wrote…
Do you mean without the fix or with the fix? The panic should occur during optimization like it did in #80901, right?
Hm... I mean without the fix. When running through the opttester the memo prints as usual with no error or detected cycle, but with the cycle visible within the memo. I assumed this was an artifact of the opttester, but maybe not. Similarly, there is no problem running the actual query on a cluster, but I get the error when I try to run explain or explain (opt) from the command line. I'm also using an M1 mac...
Some printf debugging in the case that doesn't throw the error showed that the number of passes never exceeded 1, and we never even reached that maxGroupPasses check more than 20 times.
mgartner
left a comment
There was a problem hiding this comment.
Reviewable status:
complete! 0 of 0 LGTMs obtained (and 1 stale) (waiting on @DrewKimball)
pkg/sql/opt/xform/testdata/rules/join_order line 2622 at r1 (raw file):
Previously, DrewKimball (Drew Kimball) wrote…
Hm... I mean without the fix. When running through the opttester the memo prints as usual with no error or detected cycle, but with the cycle visible within the memo. I assumed this was an artifact of the opttester, but maybe not. Similarly, there is no problem running the actual query on a cluster, but I get the error when I try to run
explainorexplain (opt)from the command line. I'm also using an M1 mac...Some printf debugging in the case that doesn't throw the error showed that the number of passes never exceeded 1, and we never even reached that
maxGroupPassescheck more than 20 times.
Interesting. I am getting similar results to you. Expect I do see the cycle error when running these in cockroach demo. Maybe it is a problem with opttester or with the test catalog.
Let's leave this as-is for now. I've created #83996 to track the broken cycle detection.
|
Previously, mgartner (Marcus Gartner) wrote…
Expect => Except*** |
|
TFTR! |
|
bors r+ |
|
Build succeeded: |
|
blathers backport 22.2 22.1 |
|
Encountered an error creating backports. Some common things that can go wrong:
You might need to create your backport manually using the backport tool. error creating merge commit from ac77889 to blathers/backport-release-22.2-83875: POST https://api.github.com/repos/cockroachdb/cockroach/merges: 409 Merge conflict [] you may need to manually resolve merge conflicts with the backport tool. Backport to branch 22.2 failed. See errors above. error creating merge commit from ac77889 to blathers/backport-release-22.1-83875: POST https://api.github.com/repos/cockroachdb/cockroach/merges: 409 Merge conflict [] you may need to manually resolve merge conflicts with the backport tool. Backport to branch 22.1 failed. See errors above. 🦉 Hoot! I am a Blathers, a bot for CockroachDB. My owner is otan. |
|
No need to backport this, since #88779 already fixed the bug. |
In some rare cases when null-rejection rules fail to fire, a redundant filter
can be inferred in an
InnerJoin-LeftJoincomplex. This could previouslyresult in the
JoinOrderBuilderattempting to add aSelectto the same memogroup as its input, which would create a memo cycle. This patch prevents the
JoinOrderBuilderfrom adding theSelectto the memo in such cases.What follows is a more detailed explanation of the conditions that could
previously cause a memo cycle.
InnerJoinoperators have two properties that make them more 'reorderable'than other types of joins: (1) their conjuncts can be reordered separately
and (2) new conjuncts can be inferred from equalities. As a special case of
(1), an
InnerJoincan be pushed into the left side of aLeftJoin, leavingbehind any
Selectconjuncts that reference the right side of theLeftJoin.This allows the
JoinOrderBuilderto make the following transformation:Note the new
b=dfilter that was inferred and subsequently left ona
Selectoperator after the reordering. The crucial point is thatthis filter is redundant - the input to the
Selectis already avalid reordering of the
BCDjoin complex.The
JoinOrderBuilderavoids adding redundant filters toInnerJoinoperators, but does not do the same for the
Selectcase because itwas assumed that the filters left on the
Selectwould never be redundant.This is because the
a=dfilter should rejects nulls, so theLeftJoinshould have been simplified. However, in rare cases null-rejection does not
take place. Because the input to the
Selectis already a valid reordering,the
JoinOrderBuilderends up trying to add theSelectto the same groupas its input - namely, the
BCDjoin group. This causes a cycle in the memo.Fixes #80901
Release note (bug fix): Fixed a bug that could cause an optimizer
panic in rare cases when a query had a left join in the input of
an inner join.