Skip to content

Commit

Permalink
opt: don't add reordered join to group when filters weren't pushed down
Browse files Browse the repository at this point in the history
The `JoinOrderBuilder` builds reordered join plans from the bottom up, and
expects filters to be pushed down as far as possible at each step. It also
reuses the original matched joins when possible, to avoid duplicate work.
This could previously cause filters to be dropped in the case when they
weren't pushed all the way down.

This commit adds a check to the `JoinOrderBuilder` to identify cases where
filters 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 in rare cases.
  • Loading branch information
DrewKimball committed Sep 27, 2022
1 parent c55586b commit 6bdb468
Show file tree
Hide file tree
Showing 2 changed files with 220 additions and 78 deletions.
69 changes: 58 additions & 11 deletions pkg/sql/opt/xform/join_order_builder.go
Original file line number Diff line number Diff line change
Expand Up @@ -316,6 +316,14 @@ type JoinOrderBuilder struct {
// assembling filters.
equivs props.EquivSet

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

onReorderFunc OnReorderFunc

onAddJoinFunc OnAddJoinFunc
Expand Down Expand Up @@ -354,6 +362,10 @@ func (jb *JoinOrderBuilder) Reorder(join memo.RelExpr) {
// the best plan.
jb.ensureClosure(join)

// Ensure that if any of the joins did not successfully push filters down as
// far as possible, all joins apart from the root join will be rebuilt.
jb.ensurePushdown()

if jb.onReorderFunc != nil {
// Hook for testing purposes.
jb.callOnReorderFunc(join)
Expand All @@ -368,6 +380,37 @@ func (jb *JoinOrderBuilder) Reorder(join memo.RelExpr) {
}
}

// ensurePushdown checks whether each edge applies to its original join. If any
// do not, push-down rules failed to push the filter(s) down as far as possible,
// and it is not valid to add new reordered joins to the original joins apart
// from the root join. When this is the case, all joins except for the root join
// need to be removed from the plans map.
func (jb *JoinOrderBuilder) ensurePushdown() {
for i := range jb.edges {
if jb.rebuildAllJoins {
break
}
e := &jb.edges[i]
isFilter := len(e.filters) > 0 && e.filters[0].ScalarProps().OuterCols.Contains(3) && e.filters[0].ScalarProps().OuterCols.Contains(9)
_ = isFilter
if e.op.joinType == opt.InnerJoinOp {
jb.rebuildAllJoins = !e.checkInnerJoin(e.op.leftVertexes, e.op.rightVertexes)
} else {
jb.rebuildAllJoins = !e.checkNonInnerJoin(e.op.leftVertexes, e.op.rightVertexes)
}
}
if jb.rebuildAllJoins {
for vertexes := range jb.plans {
if vertexes.isSingleton() || vertexes == jb.allVertexes() {
// Do not remove the plan if it is for a base relation (not a join) or
// it is the root join.
continue
}
delete(jb.plans, vertexes)
}
}
}

// populateGraph traverses the given subtree up to ReorderJoinsLimit and
// initializes the vertexes and edges of the join hypergraph. populateGraph
// returns the sets of vertexes and edges that were added to the graph during
Expand Down Expand Up @@ -528,7 +571,7 @@ func (jb *JoinOrderBuilder) addJoins(s1, s2 vertexSet) {
if !joinIsRedundant {
// If this edge was originally part of a join between relation sets s1 and
// s2, any other edges that apply will also be part of that original join.
joinIsRedundant = e.joinIsRedundant(s1, s2)
joinIsRedundant = e.joinIsRedundant(jb, s1, s2)
}
for j := range e.filters {
jb.equivs.AddFromFDs(&e.filters[j].ScalarProps().FuncDeps)
Expand All @@ -549,7 +592,7 @@ func (jb *JoinOrderBuilder) addJoins(s1, s2 vertexSet) {
// Construct a non-inner join. If any inner join filters also apply to the
// pair of relationSets, construct a select on top of the join with the
// inner join filters.
jb.addJoin(e.op.joinType, s1, s2, e.filters, innerJoinFilters, e.joinIsRedundant(s1, s2))
jb.addJoin(e.op.joinType, s1, s2, e.filters, innerJoinFilters, e.joinIsRedundant(jb, s1, s2))
return
}
if e.checkNonInnerJoin(s2, s1) {
Expand All @@ -575,7 +618,7 @@ func (jb *JoinOrderBuilder) addJoins(s1, s2 vertexSet) {
// 010 on the right. 101 is larger than 111 / 2, so we will not enumerate
// this plan unless we consider a join with s2 on the left and s1 on the
// right.
jb.addJoin(e.op.joinType, s2, s1, e.filters, innerJoinFilters, e.joinIsRedundant(s2, s1))
jb.addJoin(e.op.joinType, s2, s1, e.filters, innerJoinFilters, e.joinIsRedundant(jb, s2, s1))
return
}
}
Expand Down Expand Up @@ -642,6 +685,16 @@ func (jb *JoinOrderBuilder) makeTransitiveEdge(col1, col2 opt.ColumnID) {
return
}

originalJoin, ok := jb.plans[op.leftVertexes.union(op.rightVertexes)]
if !ok {
panic(errors.AssertionFailedf("failed to find expected join plan"))
}
if !originalJoin.Relational().FuncDeps.AreColsEquiv(col1, col2) {
// This inferred filter was not pushed down as far as possible. All joins
// apart from the root will have to be rebuilt.
jb.rebuildAllJoins = true
}

// Construct the edge.
var1 := jb.f.ConstructVariable(col1)
var2 := jb.f.ConstructVariable(col2)
Expand Down Expand Up @@ -754,12 +807,6 @@ func (jb *JoinOrderBuilder) addToGroup(
) {
if len(selectFilters) > 0 {
joinExpr := jb.memoize(op, left, right, on, nil)
if joinExpr.FirstExpr() == grp.FirstExpr() {
// In rare cases, the select filters may be redundant. In this case,
// adding a select to the group with the redundant filters would create a
// memo cycle (see #80901).
return
}
selectExpr := &memo.SelectExpr{
Input: joinExpr,
Filters: selectFilters,
Expand Down Expand Up @@ -1356,8 +1403,8 @@ func (e *edge) checkRules(s1, s2 vertexSet) bool {
// joinIsRedundant returns true if a join between the two sets of base relations
// was already present in the original join tree. If so, enumerating this join
// would be redundant, so it should be skipped.
func (e *edge) joinIsRedundant(s1, s2 vertexSet) bool {
return e.op.leftVertexes == s1 && e.op.rightVertexes == s2
func (e *edge) joinIsRedundant(jb *JoinOrderBuilder, s1, s2 vertexSet) bool {
return !jb.rebuildAllJoins && e.op.leftVertexes == s1 && e.op.rightVertexes == s2
}

// commute returns true if the given join operator type is commutable.
Expand Down
Loading

0 comments on commit 6bdb468

Please sign in to comment.