Skip to content

Commit 73e8b28

Browse files
committed
opt: don't add reordered join with extra filters to original memo group
The `JoinOrderBuilder` builds reordered join plans from the bottom up. It expects filters to be pushed down as far as possible at each step, and that transitive closure has been calculated over Inner Join equality filters (e.g. `a=b` and `b=c` => `a=c`). It also reuses the original matched joins when possible to avoid duplicate work by adding to the original memo groups. This could previously cause filters to be dropped in the case when the original join tree did not compute transitive closure and push filters down as far as possible. More specifically, the `JoinOrderBuilder` could add new reordered joins with new filters synthesized and pushed down as far as possible to an original memo group that didn't have one of those filters. Subsequent joins would then expect the filter to be part of the memo group, and so it wouldn't be added later on in the plan. In the rare case when the expression without the filter was chosen, this could manifest as a dropped filter in the final plan. This was rare because dropping a filter usually does not produce a lower-cost plan. As an example, take this original join tree: ``` (xy join ab on true) join uv on x = u and a = u; ``` Here it is possible to sythesize and push down a `x = a` filter, and so the `JoinOrderBuilder` would do this and add it to the group: ``` group (xy join ab on true), (xy join ab on x = a) ``` Later joins would use this group as an input, an expect the `x = a` filter to be present. If costing happened to choose the first expression in the group, we would end up choosing a plan like this: ``` (xy join ab on true) join uv on x = u ``` Where the `a = u` filter isn't included in the top-level join because it would be redundant to add it when `x = u` and `x = a` are already present. This is a bit of a simplification, but is essentially the problem fixed by this commit. This commit adds a check to the `JoinOrderBuilder` to identify cases where filters (including ones sythesized from the transitive closure) weren't pushed all the way down in the original join tree. When this is true, none of the originally matched joins can be reused when reordered joins are built except for the root join. This solution may perform some duplicate work when filters aren't pushed down, but it shouldn't matter because this case is rare (and should be avoided whenever possible). Fixes #88659 Release note (bug fix): Fixed a bug introduced in 20.2 that could cause filters to be dropped from a query plan with many joins in rare cases.
1 parent aaca5ce commit 73e8b28

2 files changed

Lines changed: 263 additions & 85 deletions

File tree

pkg/sql/opt/xform/join_order_builder.go

Lines changed: 108 additions & 18 deletions
Original file line numberDiff line numberDiff line change
@@ -316,6 +316,14 @@ type JoinOrderBuilder struct {
316316
// assembling filters.
317317
equivs props.EquivSet
318318

319+
// rebuildAllJoins is true when the filters in the original matched join tree
320+
// were not pushed down as far as possible. When this is true, all joins
321+
// except the root join need to be re-built, possibly with additional filters
322+
// pushed down. While technically it is sufficient to only do this for the
323+
// joins that would be changed by a successful push-down, it is simpler to
324+
// handle things this way (and the problem is rare).
325+
rebuildAllJoins bool
326+
319327
onReorderFunc OnReorderFunc
320328

321329
onAddJoinFunc OnAddJoinFunc
@@ -354,6 +362,12 @@ func (jb *JoinOrderBuilder) Reorder(join memo.RelExpr) {
354362
// the best plan.
355363
jb.ensureClosure(join)
356364

365+
// Ensure that the JoinOrderBuilder will not add reordered joins to the
366+
// original memo groups (apart from the root) in the case when doing so
367+
// would add filters that weren't present in the original joins. See the
368+
// validateEdges comment for more information.
369+
jb.validateEdges()
370+
357371
if jb.onReorderFunc != nil {
358372
// Hook for testing purposes.
359373
jb.callOnReorderFunc(join)
@@ -466,6 +480,72 @@ func (jb *JoinOrderBuilder) ensureClosure(join memo.RelExpr) {
466480
}
467481
}
468482

483+
// validateEdges checks whether each edge applies to its original join. If any
484+
// do not, normalization rules failed to synthesize and push a filter down as
485+
// far as possible, and it is not valid to add new reordered joins to the
486+
// original memo groups. When this is the case, all joins except for the root
487+
// join need to be removed from the plans map. This prevents cases where a join
488+
// is added to a memo group that isn't logically equivalent.
489+
//
490+
// This is necessary because the JoinOrderBuilder expects each join tree for a
491+
// given set of relations to contain all filters that apply to those relations.
492+
// When a new join is constructed, it doesn't contain "degenerate" filters -
493+
// filters that only refer to one side of the join. So if the original join tree
494+
// had an implicit filter that could have been synthesized and pushed down the
495+
// tree, but wasn't, using the original join group that *should* have that
496+
// filter when building a new join would cause a filter to be dropped.
497+
//
498+
// Take the following (simplified) example of a join tree where filter push-down
499+
// rules have failed:
500+
//
501+
// (xy join ab on true) join uv on x = u and a = u
502+
//
503+
// Here, the JoinOrderBuilder will synthesize an 'x = a' filter that will be
504+
// used to join xy and ab. If it was added to the original group, we would have
505+
// a memo group that looks like this:
506+
//
507+
// group: (xy join ab on true), (xy join ab on x = a)
508+
//
509+
// Later joins that are constructed using this group would expect the 'x = a'
510+
// filter to be present, and would avoid adding redundant filters. Therefore,
511+
// a join tree like the following would be added to the memo.
512+
//
513+
// (xy join ab on true) join uv on x = u
514+
//
515+
// Notice how the 'a = u' filter has been dropped because it would be redundant
516+
// when 'x = u' and 'x = a' are already present. We prevent this from happening
517+
// by not reusing the original memo groups in the case when the JoinOrderBuilder
518+
// is able to synthesize and/or push down filters that weren't in the original
519+
// join tree.
520+
func (jb *JoinOrderBuilder) validateEdges() {
521+
for i := range jb.edges {
522+
if jb.rebuildAllJoins {
523+
break
524+
}
525+
e := &jb.edges[i]
526+
if e.op.joinType == opt.InnerJoinOp {
527+
jb.rebuildAllJoins = !e.checkInnerJoin(e.op.leftVertexes, e.op.rightVertexes)
528+
} else {
529+
jb.rebuildAllJoins = !e.checkNonInnerJoin(e.op.leftVertexes, e.op.rightVertexes)
530+
}
531+
}
532+
if jb.rebuildAllJoins {
533+
for vertexes := range jb.plans {
534+
if vertexes.isSingleton() || vertexes == jb.allVertexes() {
535+
// Do not remove the plan if it is for a base relation (not a join) or
536+
// it is the root join. Adding to the root join group is correct because
537+
// the JoinOrderBuilder will only consider filters that were present
538+
// (even if only implicitly) in the root join tree. It is also necessary
539+
// because the purpose of the JoinOrderBuilder is to add equivalent join
540+
// plans to the root join group - otherwise, any new joins would be
541+
// disconnected from the main query plan.
542+
continue
543+
}
544+
delete(jb.plans, vertexes)
545+
}
546+
}
547+
}
548+
469549
// dpSube carries out the DPSube algorithm (citations: [8] figure 4). All
470550
// disjoint pairs of subsets of base relations are enumerated and checked for
471551
// validity. If valid, the pair of subsets is used along with the edges
@@ -526,9 +606,10 @@ func (jb *JoinOrderBuilder) addJoins(s1, s2 vertexSet) {
526606
continue
527607
}
528608
if !joinIsRedundant {
529-
// If this edge was originally part of a join between relation sets s1 and
530-
// s2, any other edges that apply will also be part of that original join.
531-
joinIsRedundant = e.joinIsRedundant(s1, s2)
609+
// If this edge was originally part of a join between relation sets s1
610+
// and s2, any other edges that apply will also be part of that original
611+
// join.
612+
joinIsRedundant = jb.joinIsRedundant(e, s1, s2)
532613
}
533614
for j := range e.filters {
534615
jb.equivs.AddFromFDs(&e.filters[j].ScalarProps().FuncDeps)
@@ -549,7 +630,7 @@ func (jb *JoinOrderBuilder) addJoins(s1, s2 vertexSet) {
549630
// Construct a non-inner join. If any inner join filters also apply to the
550631
// pair of relationSets, construct a select on top of the join with the
551632
// inner join filters.
552-
jb.addJoin(e.op.joinType, s1, s2, e.filters, innerJoinFilters, e.joinIsRedundant(s1, s2))
633+
jb.addJoin(e.op.joinType, s1, s2, e.filters, innerJoinFilters, jb.joinIsRedundant(e, s1, s2))
553634
return
554635
}
555636
if e.checkNonInnerJoin(s2, s1) {
@@ -575,7 +656,7 @@ func (jb *JoinOrderBuilder) addJoins(s1, s2 vertexSet) {
575656
// 010 on the right. 101 is larger than 111 / 2, so we will not enumerate
576657
// this plan unless we consider a join with s2 on the left and s1 on the
577658
// right.
578-
jb.addJoin(e.op.joinType, s2, s1, e.filters, innerJoinFilters, e.joinIsRedundant(s2, s1))
659+
jb.addJoin(e.op.joinType, s2, s1, e.filters, innerJoinFilters, jb.joinIsRedundant(e, s2, s1))
579660
return
580661
}
581662
}
@@ -642,6 +723,19 @@ func (jb *JoinOrderBuilder) makeTransitiveEdge(col1, col2 opt.ColumnID) {
642723
return
643724
}
644725

726+
originalJoin, ok := jb.plans[op.leftVertexes.union(op.rightVertexes)]
727+
if !ok {
728+
panic(errors.AssertionFailedf("failed to find expected join plan"))
729+
}
730+
if !originalJoin.Relational().FuncDeps.AreColsEquiv(col1, col2) {
731+
// This inferred filter was not pushed down as far as possible. All joins
732+
// apart from the root will have to be rebuilt. We have to do this check
733+
// here because we set the op for this edge to the join to which the filter
734+
// *would* have been pushed down if it existed, so the applicable check will
735+
// always succeed for that join.
736+
jb.rebuildAllJoins = true
737+
}
738+
645739
// Construct the edge.
646740
var1 := jb.f.ConstructVariable(col1)
647741
var2 := jb.f.ConstructVariable(col2)
@@ -754,12 +848,6 @@ func (jb *JoinOrderBuilder) addToGroup(
754848
) {
755849
if len(selectFilters) > 0 {
756850
joinExpr := jb.memoize(op, left, right, on, nil)
757-
if joinExpr.FirstExpr() == grp.FirstExpr() {
758-
// In rare cases, the select filters may be redundant. In this case,
759-
// adding a select to the group with the redundant filters would create a
760-
// memo cycle (see #80901).
761-
return
762-
}
763851
selectExpr := &memo.SelectExpr{
764852
Input: joinExpr,
765853
Filters: selectFilters,
@@ -904,6 +992,15 @@ func (jb *JoinOrderBuilder) addBaseRelation(rel memo.RelExpr) {
904992
jb.plans[relSet] = rel
905993
}
906994

995+
// joinIsRedundant returns true if a join between the two sets of base relations
996+
// was already present in the original join tree. If so, enumerating this join
997+
// would be redundant, so it should be skipped.
998+
func (jb *JoinOrderBuilder) joinIsRedundant(e *edge, s1, s2 vertexSet) bool {
999+
// The join is never redundant when rebuildAllJoins is true, because
1000+
// rebuildAllJoins indicates we don't want to reuse the original joins.
1001+
return !jb.rebuildAllJoins && e.op.leftVertexes == s1 && e.op.rightVertexes == s2
1002+
}
1003+
9071004
// checkSize panics if the number of relations is greater than or equal to
9081005
// MaxReorderJoinsLimit. checkSize should be called before a vertex is added to
9091006
// the join graph.
@@ -1353,13 +1450,6 @@ func (e *edge) checkRules(s1, s2 vertexSet) bool {
13531450
return true
13541451
}
13551452

1356-
// joinIsRedundant returns true if a join between the two sets of base relations
1357-
// was already present in the original join tree. If so, enumerating this join
1358-
// would be redundant, so it should be skipped.
1359-
func (e *edge) joinIsRedundant(s1, s2 vertexSet) bool {
1360-
return e.op.leftVertexes == s1 && e.op.rightVertexes == s2
1361-
}
1362-
13631453
// commute returns true if the given join operator type is commutable.
13641454
func commute(op opt.Operator) bool {
13651455
return op == opt.InnerJoinOp || op == opt.FullJoinOp

0 commit comments

Comments
 (0)