opt: fixup CTE stats on placeholder queries#99433
opt: fixup CTE stats on placeholder queries#99433craig[bot] merged 1 commit intocockroachdb:masterfrom
Conversation
|
It looks like your PR touches production code but doesn't add or edit any test code. Did you consider adding tests to your PR? 🦉 Hoot! I am a Blathers, a bot for CockroachDB. My owner is dev-inf. |
ca5da91 to
9c807dc
Compare
mgartner
left a comment
There was a problem hiding this comment.
Great find!
It's a bit unprecedented for a normalization rule to muck with statistics. Maybe that's ok, but I'm not 100% sure yet. I'm trying to think of a more idiomatic way to achieve the same goal as this.
Reviewable status:
complete! 0 of 0 LGTMs obtained (waiting on @cucaroach and @msirek)
pkg/sql/opt/norm/with_funcs.go line 301 at r2 (raw file):
} func (c *CustomFuncs) ApplyBindingRowCount(
nit: add comment
pkg/sql/opt/norm/with_funcs.go line 315 at r2 (raw file):
} func (c *CustomFuncs) RowCountDifferent(
nit: add comment explaining this function.
pkg/sql/opt/norm/rules/with.opt line 68 at r2 (raw file):
) # ApplyBindingRowCountToCTE makes sure to that stats changes are propagated on
nit: typo in "makes sure to that"
pkg/sql/opt/xform/testdata/rules/cte line 23 at r2 (raw file):
# Regression test for #99389. assign-placeholders-opt query-args=(1) format=show-stats
Can you move this test to pkg/sql/opt/norm/testdata/rules/with and use assign-placeholder-norm instead of assign-placeholders-opt?
msirek
left a comment
There was a problem hiding this comment.
Nice fix!
Reviewable status:
complete! 0 of 0 LGTMs obtained (waiting on @cucaroach and @mgartner)
pkg/sql/opt/norm/rules/with.opt line 79 at r2 (raw file):
=> (RecursiveCTE (ApplyBindingRowCount $binding $initial)
It looks like this will update the original binding expression with a new row count.
Shouldn't an expression in the memo be treated as immutable? The updated expression would have a different hash than what was computed when it was placed in the internCache.
We may want to make a new FakeRel and update its row count.
@msirek was playing around with an update to optgen -- did that work? |
msirek
left a comment
There was a problem hiding this comment.
One possible downside to using a normalization rule is use of more memory. We memoize new expressions in CopyAndReplaceDefault and then memoize again in the normalization rule.
Reviewable status:
complete! 0 of 0 LGTMs obtained (waiting on @cucaroach and @mgartner)
msirek
left a comment
There was a problem hiding this comment.
@msirek was playing around with an update to optgen -- did that work?
I haven't tested that yet, but I think it should work if we want to go that route.
Reviewable status:
complete! 0 of 0 LGTMs obtained (waiting on @cucaroach and @mgartner)
DrewKimball
left a comment
There was a problem hiding this comment.
Reviewable status:
complete! 0 of 0 LGTMs obtained (waiting on @cucaroach, @mgartner, and @msirek)
pkg/sql/opt/norm/rules/with.opt line 79 at r2 (raw file):
Previously, msirek (Mark Sirek) wrote…
It looks like this will update the original binding expression with a new row count.
Shouldn't an expression in the memo be treated as immutable? The updated expression would have a different hash than what was computed when it was placed in theinternCache.We may want to make a new FakeRel and update its row count.
+1 - we should make a new one and update the binding stored in the metadata. That would probably require adding a new method to opt.Metadata similar to AddWithBinding that would instead update an existing binding.
pkg/sql/opt/norm/rules/with.opt line 81 at r2 (raw file):
(ApplyBindingRowCount $binding $initial) $initial $recursive
I think the recursive branch will also need to be reconstructed. The test case didn't catch it because the recursive branch is a contradiction that gets simplified, but a more interesting case should do the trick.
Would be interesting to know if that approach is simpler / would avoid the potential downsides of a normalization rule |
Actually, |
msirek
left a comment
There was a problem hiding this comment.
I ran the regression test in this PR against the CopyAndReplaceDefault fix and got the same results.
Reviewable status:
complete! 0 of 0 LGTMs obtained (waiting on @cucaroach, @DrewKimball, and @mgartner)
|
Is an optgen language change necessary? We already have the |
|
Forgot the code pointer: cockroach/pkg/sql/opt/norm/factory.go Lines 391 to 416 in 63d9aed |
That looks like just the thing! I'll give that a whirl. |
msirek
left a comment
There was a problem hiding this comment.
Is an optgen language change necessary? We already have the onConstructRelational hook which is called at the end of Construct... functions. Can we put this logic there?
onConstructRelational is called after the expression is memoized, so if we alter the expression, we'll have to memoize it again (which may add another expression to the memo), unless we swap the order of the MemoizeXXX call and onConstructRelational.
Reviewable status:
complete! 0 of 0 LGTMs obtained (waiting on @cucaroach, @DrewKimball, and @mgartner)
I think if we change something we have to re memoize and intern anyway no? FakeRel interns by props pointer value and it seems unsound to just mutate it. And like Drew said we need to reconstruct the recursive branch as well. |
9c807dc to
4ecd597
Compare
|
Okay based on Drew's feedback I think we have to rebuild stuff (and trying to avoid rebuilding seems silly given that the model of assign-placeholders/replace is that if we change something all the ancestors are rebuilt). So I tried I cleaned it up and basically went with same model. RFAL! |
4ecd597 to
b092bd6
Compare
60d0ff2 to
9d3d8b3
Compare
|
New approach inspired by Mark's approach, what do we think? |
pkg/sql/opt/norm/factory.go
Outdated
| if newInitial != rcte.Initial { | ||
| newBinding := f.ConstructFakeRel(&memo.FakeRelPrivate{ | ||
| Props: MakeBindingPropsForRecursiveCTE( | ||
| props.AnyCardinality, rcte.Binding.Relational().OutputCols, newInitial.Relational().Statistics().RowCount)}) |
There was a problem hiding this comment.
See fixup for possible improvement to cardinality arg.
DrewKimball
left a comment
There was a problem hiding this comment.
I like this,
AssignPlaceholders feels like the right place to solve it.
Reviewed 1 of 2 files at r5, 3 of 3 files at r6, all commit messages.
Reviewable status:complete! 1 of 0 LGTMs obtained (waiting on @cucaroach, @mgartner, @msirek, and @rytaft)
-- commits line 6 at r6:
I think the commit message needs an update.
pkg/sql/opt/norm/factory.go line 363 at r6 (raw file):
Previously, cucaroach (Tommy Reilly) wrote…
See fixup for possible improvement to cardinality arg.
I don't think it should be necessary to set the cardinality here, since there are already rules that should handle it (ApplyLimitToRecursiveCTEScan and TryAddLimitToRecursiveBranch).
pkg/sql/opt/norm/factory.go line 359 at r7 (raw file):
if rcte, ok := e.(*memo.RecursiveCTEExpr); ok { newInitial := f.CopyAndReplaceDefault(rcte.Initial, replaceFn).(memo.RelExpr) if newInitial != rcte.Initial {
We could also check if the row-counts are different here.
41518e5 to
314d1bf
Compare
cucaroach
left a comment
There was a problem hiding this comment.
Reviewable status:
complete! 0 of 0 LGTMs obtained (and 1 stale) (waiting on @DrewKimball, @mgartner, @msirek, and @rytaft)
Previously, DrewKimball (Drew Kimball) wrote…
I think the commit message needs an update.
Done.
pkg/sql/opt/norm/factory.go line 363 at r6 (raw file):
Previously, DrewKimball (Drew Kimball) wrote…
I don't think it should be necessary to set the cardinality here, since there are already rules that should handle it (
ApplyLimitToRecursiveCTEScanandTryAddLimitToRecursiveBranch).
👍
pkg/sql/opt/norm/factory.go line 359 at r7 (raw file):
Previously, DrewKimball (Drew Kimball) wrote…
We could also check if the row-counts are different here.
I thought about it but what if CopyAndReplaceDefault rebuilt the thing and changed something else but not the stats, doing this seemed more sane.
pkg/sql/opt/xform/testdata/rules/cte line 23 at r2 (raw file):
Previously, mgartner (Marcus Gartner) wrote…
Can you move this test to
pkg/sql/opt/norm/testdata/rules/withand useassign-placeholder-norminstead ofassign-placeholders-opt?
Done.
mgartner
left a comment
There was a problem hiding this comment.
I agree with @DrewKimball, this seems like the most natural solution. Great work!
Reviewed 1 of 2 files at r5, 1 of 1 files at r8, all commit messages.
Reviewable status:complete! 1 of 0 LGTMs obtained (and 1 stale) (waiting on @cucaroach, @DrewKimball, @msirek, and @rytaft)
-- commits line 10 at r8:
nit: Should we have a performance improvement release note here?
pkg/sql/opt/norm/factory.go line 359 at r8 (raw file):
} if rcte, ok := e.(*memo.RecursiveCTEExpr); ok { newInitial := f.CopyAndReplaceDefault(rcte.Initial, replaceFn).(memo.RelExpr)
nit: A comment explaining what we're doing here would be good, since it's not obvious.
During optbuilder phase we copy the initial expressions stats into the fake-rel but this value can change when placeholders are assigned so add code in AssignPlaceholders to rebuild the cte if the stats change. Fixes: cockroachdb#99389 Epic: none Release note (sql change): Prepared statements using placeholders in recursive CTEs sometimes did not re-optimize correctly after plugging in the parameters leading to poor plan choices, this has been fixed.
314d1bf to
2014cf2
Compare
|
TFTRs! Nice teamwork! |
|
Build failed (retrying...): |
|
Build failed (retrying...): |
|
Build failed (retrying...): |
|
Build failed (retrying...): |
|
Build failed: |
|
bors retry |
|
Build succeeded: |
|
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 2014cf2 to blathers/backport-release-22.1-99433: 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.x failed. See errors above. error creating merge commit from 2014cf2 to blathers/backport-release-22.2-99433: 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.x failed. See errors above. 🦉 Hoot! I am a Blathers, a bot for CockroachDB. My owner is dev-inf. |
During optbuilder phase we copy the initial expressions stats into the
fake-rel but this value can change when placeholders are assigned so add
code in AssignPlaceholders to rebuild the cte if the stats change.
Fixes: #99389
Epic: none
Release note: none