Skip to content

Commit

Permalink
opt: don't drop LeftJoin filter during join ordering
Browse files Browse the repository at this point in the history
This patch fixes a bug in the join reordering logic that can lead to
incorrect results due to a dropped filter and incorrect conversion of
a left join to an inner join. The bug can occur when the join tree
contains an inner join with a left join as an input, where the inner
join has two separate conjuncts in its ON condition that reference
both inputs of the left join. Additionally, the inner join filters
must not filter NULL values from the right side of the left join
(or alternatively null-rejection rules must be disabled).

The incorrect transformation looks something like this:
```
(INNER JOIN xy (LEFT JOIN ab (INNER JOIN uv wz ON v = w) ON b = v) ON a = x AND u = x)
```
=>
```
(INNER JOIN ab (INNER JOIN xy (INNER JOIN uv wz ON v = w) ON u = x) ON a = x)
```
Notice how `xy` has been "pushed" into the right side of the left
join and the left join's `b = v` filter (and the left join itself)
dropped in the process.

To understand what causes the bug, it is necessary to understand three
points about the join reordering algorithm:
1. Cross products are never introduced in the enumerated plans. So, for
   two sub-plans, a join is only considered between them if there is an
   applicable edge between those sub-plans.
2. The original paper associates each join with exactly one edge in the
   hypergraph that encodes "reorderability" properties.
3. The `JoinOrderBuilder` departs from the paper by associating each
   inner join *conjunct* with a hypergraph edge. This allows each
   conjunct to be independently reordered from the others. See the
   `Special handling of inner joins` section in the `JoinOrderBuilder`
   comment for more details.

(1) combined with (2) implies that a reordered join tree is only
considered if every edge in the hypergraph could be applied to form joins
in the join tree. This allows the original algorithm to prevent invalid
orderings by making just a single edge inapplicable. However, because
of (3) the same is no longer true for the `JoinOrderBuilder`. In the
example given above, the left join fails the applicability check,
indicating an invalid plan. However, the inner join's `a = x` filter
passes the check and ends up replacing the left join. This prevents
the the check in (1) from catching the invalid plan.

This patch fixes the bug by keeping track of the edges that *should*
be applied somewhere in each join tree based on the TES of each edge.
This is then compared against the actual edges that are applied in
the construction of the join tree. If the edge sets aren't equal,
the plan is invalid and cannot be added to the memo. This allows the
`JoinOrderBuilder` to recover the property that an inapplicable edge
invalidates an enumerated plan.

Fixes #90761

Release note (bug fix): Fixed a bug existing since 20.2 that could
cause incorrect results in rare cases for queries with inner joins
and left joins. For the bug to occur, the left join had to be in
the input of the inner join and the inner join filters had to
reference both inputs of the left join, and not filter NULL values
from the right input of the left join. Additionally, the right input
of the left join had to contain at least one join, with one input not
referenced by the left join's ON condition.
  • Loading branch information
DrewKimball committed Nov 2, 2022
1 parent 7dd9b8a commit 8486195
Show file tree
Hide file tree
Showing 2 changed files with 259 additions and 25 deletions.
119 changes: 94 additions & 25 deletions pkg/sql/opt/xform/join_order_builder.go
Original file line number Diff line number Diff line change
Expand Up @@ -307,6 +307,11 @@ type JoinOrderBuilder struct {
// The group for a single base relation is simply the base relation itself.
plans map[vertexSet]memo.RelExpr

// applicableEdges maps from each (sub)set of vertexes to the set of edges
// that must be used when building join trees for the set. See
// checkAppliedEdges for more information.
applicableEdges map[vertexSet]edgeSet

// joinCount counts the number of joins that have been added to the join
// graph. It is used to ensure that the number of joins that are reordered at
// once does not exceed the session limit.
Expand Down Expand Up @@ -336,12 +341,13 @@ func (jb *JoinOrderBuilder) Init(f *norm.Factory, evalCtx *eval.Context) {
// This initialization pattern ensures that fields are not unwittingly
// reused. Field reuse must be explicit.
*jb = JoinOrderBuilder{
f: f,
evalCtx: evalCtx,
plans: make(map[vertexSet]memo.RelExpr),
onReorderFunc: jb.onReorderFunc,
onAddJoinFunc: jb.onAddJoinFunc,
equivs: props.NewEquivSet(),
f: f,
evalCtx: evalCtx,
plans: make(map[vertexSet]memo.RelExpr),
applicableEdges: make(map[vertexSet]edgeSet),
onReorderFunc: jb.onReorderFunc,
onAddJoinFunc: jb.onAddJoinFunc,
equivs: props.NewEquivSet(),
}
}

Expand Down Expand Up @@ -559,6 +565,8 @@ func (jb *JoinOrderBuilder) dpSube() {
// relation. We need at least two relations in order to create a new join.
continue
}
jb.setApplicableEdges(subset)

// Enumerate all possible pairwise-disjoint binary partitions of the subset,
// s1 AND s2. These represent sets of relations that may be joined together.
//
Expand All @@ -577,6 +585,19 @@ func (jb *JoinOrderBuilder) dpSube() {
}
}

// setApplicableEdges initializes applicableEdges with all edges that must show
// up in any join tree that is constructed for the given set of vertexes. See
// checkAppliedEdges for how this information is used.
func (jb *JoinOrderBuilder) setApplicableEdges(s vertexSet) {
applicableEdges := edgeSet{}
for i := range jb.edges {
if jb.edges[i].tes.isSubsetOf(s) {
applicableEdges.Add(i)
}
}
jb.applicableEdges[s] = applicableEdges
}

// addJoins iterates through the edges of the join graph and checks whether any
// joins can be constructed between the memo groups for the two given sets of
// base relations without creating an invalid plan or introducing cross joins.
Expand All @@ -586,36 +607,32 @@ func (jb *JoinOrderBuilder) addJoins(s1, s2 vertexSet) {
// Both inputs must have plans.
return
}
// Keep track of which edges are applicable to this join.
var appliedEdges edgeSet

jb.equivs.Reset()
jb.equivs.AddFromFDs(&jb.plans[s1].Relational().FuncDeps)
jb.equivs.AddFromFDs(&jb.plans[s2].Relational().FuncDeps)

// Gather all inner edges that connect the left and right relation sets.
var innerJoinFilters memo.FiltersExpr
var addInnerJoin bool
var joinIsRedundant bool
for i, ok := jb.innerEdges.Next(0); ok; i, ok = jb.innerEdges.Next(i + 1) {
e := &jb.edges[i]

// Ensure that this edge forms a valid connection between the two sets. See
// the checkNonInnerJoin and checkInnerJoin comments for more information.
if e.checkInnerJoin(s1, s2) {
// Record this edge as applied even if it's redundant, since redundant
// edges are trivially applied.
appliedEdges.Add(i)
if areFiltersRedundant(&jb.equivs, e.filters) {
// Avoid adding redundant filters.
continue
}
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 = jb.joinIsRedundant(e, s1, s2)
}
for j := range e.filters {
jb.equivs.AddFromFDs(&e.filters[j].ScalarProps().FuncDeps)
}
innerJoinFilters = append(innerJoinFilters, e.filters...)
addInnerJoin = true
}
}

Expand All @@ -627,13 +644,17 @@ func (jb *JoinOrderBuilder) addJoins(s1, s2 vertexSet) {
// Ensure that this edge forms a valid connection between the two sets. See
// the checkNonInnerJoin and checkInnerJoin comments for more information.
if e.checkNonInnerJoin(s1, s2) {
appliedEdges.Add(i)

// 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, jb.joinIsRedundant(e, s1, s2))
jb.addJoin(e.op.joinType, s1, s2, e.filters, innerJoinFilters, appliedEdges)
return
}
if e.checkNonInnerJoin(s2, s1) {
appliedEdges.Add(i)

// If joining s1, s2 is not valid, try s2, s1. We only do this if the
// s1, s2 join fails, because commutation is handled by the addJoin
// function. This is necessary because we only iterate s1 up to subset / 2
Expand All @@ -656,17 +677,17 @@ 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, jb.joinIsRedundant(e, s2, s1))
jb.addJoin(e.op.joinType, s2, s1, e.filters, innerJoinFilters, appliedEdges)
return
}
}

if addInnerJoin {
if !appliedEdges.Empty() {
// Construct an inner join. Don't add in the case when a non-inner join has
// already been constructed, because doing so can lead to a case where a
// non-inner join operator 'disappears' because an inner join has replaced
// it.
jb.addJoin(opt.InnerJoinOp, s1, s2, innerJoinFilters, nil /* selectFilters */, joinIsRedundant)
jb.addJoin(opt.InnerJoinOp, s1, s2, innerJoinFilters, nil /* selectFilters */, appliedEdges)
}
}

Expand Down Expand Up @@ -774,6 +795,39 @@ func (jb *JoinOrderBuilder) hasEqEdge(leftCol, rightCol opt.ColumnID) bool {
return false
}

// checkAppliedEdges checks that each join plan includes every edge for which
// the TES is a subset of the relations that are joined together by the plan.
// This is necessary to recover a property which the original algorithm relies
// on - namely that if any edge cannot be applied in a given plan, that plan
// must be invalid. Consider the following three points:
//
// 1. The join reordering algorithm never includes a cross-product in an
// enumerated plan unless it was part of the original join tree. This
// means that a join between two sub-plans is only considered if there is
// an applicable edge that can be used to construct the join.
//
// 2. The original paper associates each join in the original join tree with
// exactly one edge in the join hypergraph.
//
// 3. The JoinOrderBuilder departs from the paper by associating each inner
// join conjunct with an edge. This means that each join can be associated
// with one or more edges. See the section in the JoinOrderBuilder comment
// titled "Special handling of inner joins" for details.
//
// (1) and (2) together imply that a reordered join tree is only considered if
// every edge in the hypergraph could be applied to construct a join for every
// subtree. This allows the original algorithm to prevent invalid orderings by
// making a single edge inapplicable. However, because of (3) the same is no
// longer true for the `JoinOrderBuilder`. checkAppliedEdges corrects for this
// by explicitly checking that all applicable edges have been applied when a
// join plan is considered.
func (jb *JoinOrderBuilder) checkAppliedEdges(s1, s2 vertexSet, appliedEdges edgeSet) bool {
leftApplied, rightApplied := jb.applicableEdges[s1], jb.applicableEdges[s2]
allAppliedEdges := appliedEdges.Union(leftApplied).Union(rightApplied)
expectedAppliedEdges := jb.applicableEdges[s1.union(s2)]
return allAppliedEdges.Equals(expectedAppliedEdges)
}

// addJoin adds a join between the given left and right subsets of relations on
// the given set of edges. If the group containing joins between this set of
// relations is already contained in the plans field, the new join is added to
Expand All @@ -784,11 +838,14 @@ func (jb *JoinOrderBuilder) addJoin(
op opt.Operator,
s1, s2 vertexSet,
joinFilters, selectFilters memo.FiltersExpr,
joinIsRedundant bool,
appliedEdges edgeSet,
) {
if s1.intersects(s2) {
panic(errors.AssertionFailedf("sets are not disjoint"))
}
if !jb.checkAppliedEdges(s1, s2, appliedEdges) {
return
}
if jb.onAddJoinFunc != nil {
// Hook for testing purposes.
jb.callOnAddJoinFunc(s1, s2, joinFilters, selectFilters, op)
Expand All @@ -797,7 +854,7 @@ func (jb *JoinOrderBuilder) addJoin(
left := jb.plans[s1]
right := jb.plans[s2]
union := s1.union(s2)
if !joinIsRedundant {
if !jb.joinIsRedundant(s1, s2, appliedEdges) {
if jb.plans[union] != nil {
jb.addToGroup(op, left, right, joinFilters, selectFilters, jb.plans[union])
} else {
Expand Down Expand Up @@ -995,10 +1052,22 @@ func (jb *JoinOrderBuilder) addBaseRelation(rel memo.RelExpr) {
// 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 (jb *JoinOrderBuilder) joinIsRedundant(e *edge, s1, s2 vertexSet) bool {
// The join is never redundant when rebuildAllJoins is true, because
// rebuildAllJoins indicates we don't want to reuse the original joins.
return !jb.rebuildAllJoins && e.op.leftVertexes == s1 && e.op.rightVertexes == s2
func (jb *JoinOrderBuilder) joinIsRedundant(s1, s2 vertexSet, appliedEdges edgeSet) bool {
if jb.rebuildAllJoins {
// The join is never redundant when rebuildAllJoins is true, because
// rebuildAllJoins indicates we don't want to reuse the original joins.
return false
}
for i, ok := appliedEdges.Next(0); ok; i, ok = appliedEdges.Next(i + 1) {
e := &jb.edges[i]
if e.op.leftVertexes == s1 && e.op.rightVertexes == s2 {
// 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.
return true
}
}
return false
}

// checkSize panics if the number of relations is greater than or equal to
Expand Down
Loading

0 comments on commit 8486195

Please sign in to comment.