Skip to content

Commit

Permalink
Merge #49822
Browse files Browse the repository at this point in the history
49822: opt: change JoinMultiplicity from a Relational prop to a join field r=DrewKimball a=DrewKimball

Previously, the JoinMultiplicity property was stored as a Relational
prop. This is a problem because all expressions in a memo group share
the same Relational props during exploration, and there are exploration
rules that can flip a join's left/right multiplicity.

This patch instead stores JoinMultiplcity as a join field that is
initialized during construction of the join. This fixes the shared
Relational props issue, and also  makes it possible for JoinMultiplicity
to aid in calculating other logical properties.

Fixes #49821

Release note: None

Co-authored-by: Drew Kimball <drewk@cockroachlabs.com>
  • Loading branch information
craig[bot] and DrewKimball committed Jun 5, 2020
2 parents c51b249 + ab412e1 commit 6aaf3d5
Show file tree
Hide file tree
Showing 73 changed files with 1,072 additions and 301 deletions.
7 changes: 5 additions & 2 deletions pkg/sql/opt/exec/execbuilder/testdata/explain
Original file line number Diff line number Diff line change
Expand Up @@ -948,6 +948,7 @@ TABLE t
└── k int not null
inner-join (hash)
├── columns: a:1 b:2 k:4 v:5
├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-more)
├── stats: [rows=990, distinct(1)=99, null(1)=0, distinct(4)=99, null(4)=0]
├── cost: 2219.95
├── fd: (4)-->(5), (1)==(4), (4)==(1)
Expand All @@ -957,15 +958,17 @@ inner-join (hash)
│ ├── stats: [rows=1000, distinct(1)=100, null(1)=10]
│ ├── cost: 1100.02
│ ├── prune: (1,2)
│ └── interesting orderings: (+1)
│ ├── interesting orderings: (+1)
│ └── unfiltered-cols: (1,2)
├── scan t
│ ├── columns: k:4 v:5
│ ├── stats: [rows=1000, distinct(4)=1000, null(4)=0]
│ ├── cost: 1080.02
│ ├── key: (4)
│ ├── fd: (4)-->(5)
│ ├── prune: (4,5)
│ └── interesting orderings: (+4)
│ ├── interesting orderings: (+4)
│ └── unfiltered-cols: (4,5)
└── filters
└── k:4 = a:1 [outer=(1,4), constraints=(/1: (/NULL - ]; /4: (/NULL - ]), fd=(1)==(4), (4)==(1)]

Expand Down
48 changes: 48 additions & 0 deletions pkg/sql/opt/memo/expr.go
Original file line number Diff line number Diff line change
Expand Up @@ -421,6 +421,18 @@ func (jf JoinFlags) String() string {
return b.String()
}

func (ij *InnerJoinExpr) initUnexportedFields(mem *Memo) {
initJoinMultiplicity(ij)
}

func (lj *LeftJoinExpr) initUnexportedFields(mem *Memo) {
initJoinMultiplicity(lj)
}

func (fj *FullJoinExpr) initUnexportedFields(mem *Memo) {
initJoinMultiplicity(fj)
}

func (lj *LookupJoinExpr) initUnexportedFields(mem *Memo) {
// lookupProps are initialized as necessary by the logical props builder.
}
Expand All @@ -434,6 +446,42 @@ func (zj *ZigzagJoinExpr) initUnexportedFields(mem *Memo) {
// builder.
}

// joinWithMultiplicity allows join operators for which JoinMultiplicity is
// supported (currently InnerJoin, LeftJoin, and FullJoin) to be treated
// polymorphically.
type joinWithMultiplicity interface {
setMultiplicity(props.JoinMultiplicity)
getMultiplicity() props.JoinMultiplicity
}

var _ joinWithMultiplicity = &InnerJoinExpr{}
var _ joinWithMultiplicity = &LeftJoinExpr{}
var _ joinWithMultiplicity = &FullJoinExpr{}

func (ij *InnerJoinExpr) setMultiplicity(multiplicity props.JoinMultiplicity) {
ij.multiplicity = multiplicity
}

func (ij *InnerJoinExpr) getMultiplicity() props.JoinMultiplicity {
return ij.multiplicity
}

func (lj *LeftJoinExpr) setMultiplicity(multiplicity props.JoinMultiplicity) {
lj.multiplicity = multiplicity
}

func (lj *LeftJoinExpr) getMultiplicity() props.JoinMultiplicity {
return lj.multiplicity
}

func (fj *FullJoinExpr) setMultiplicity(multiplicity props.JoinMultiplicity) {
fj.multiplicity = multiplicity
}

func (fj *FullJoinExpr) getMultiplicity() props.JoinMultiplicity {
return fj.multiplicity
}

// WindowFrame denotes the definition of a window frame for an individual
// window function, excluding the OFFSET expressions, if present.
type WindowFrame struct {
Expand Down
14 changes: 9 additions & 5 deletions pkg/sql/opt/memo/expr_format.go
Original file line number Diff line number Diff line change
Expand Up @@ -618,6 +618,13 @@ func (f *ExprFmtCtx) formatRelational(e RelExpr, tp treeprinter.Node) {
}
}

if join, ok := e.(joinWithMultiplicity); ok {
mult := join.getMultiplicity()
if s := mult.String(); s != "" {
tp.Childf("multiplicity: %s", s)
}
}

f.Buffer.Reset()
writeFlag := func(name string) {
if f.Buffer.Len() != 0 {
Expand Down Expand Up @@ -703,11 +710,8 @@ func (f *ExprFmtCtx) formatRelational(e RelExpr, tp treeprinter.Node) {
if r.JoinSize > 1 {
tp.Childf("join-size: %d", r.JoinSize)
}
switch e.Op() {
case opt.InnerJoinOp, opt.LeftJoinOp, opt.FullJoinOp:
if s := r.MultiplicityProps.String(); (r.Available&props.MultiplicityProps) != 0 && s != "" {
tp.Childf("multiplicity: %s", s)
}
if !r.UnfilteredCols.Empty() {
tp.Childf("unfiltered-cols: %s", r.UnfilteredCols.String())
}
if withUses := relational.Shared.Rule.WithUses; len(withUses) > 0 {
n := tp.Childf("cte-uses")
Expand Down
138 changes: 83 additions & 55 deletions pkg/sql/opt/memo/multiplicity_builder.go
Original file line number Diff line number Diff line change
Expand Up @@ -18,67 +18,48 @@ import (
"github.com/cockroachdb/errors"
)

// DeriveJoinMultiplicity returns a JoinMultiplicity struct that describes how a
// join operator will affect the rows of its left and right inputs (e.g.
// duplicated and/or filtered). When the function is called on an operator other
// than an InnerJoin, a LeftJoin, or a FullJoin, it simply populates the
// UnfilteredCols field of the JoinMultiplicity for that operator and leaves the
// join fields unchanged.
//
// DeriveJoinMultiplicity recursively derives the UnfilteredCols field and
// populates the props.Relational.Rule.MultiplicityProps field as it goes to
// make future calls faster.
func DeriveJoinMultiplicity(in RelExpr) props.JoinMultiplicity {
// If the MultiplicityProps property has already been derived, return it
// immediately.
relational := in.Relational()
if relational.IsAvailable(props.MultiplicityProps) {
return relational.Rule.MultiplicityProps
}
relational.Rule.Available |= props.MultiplicityProps
var multiplicity props.JoinMultiplicity

// Derive MultiplicityProps now.
// initJoinMultiplicity initializes a JoinMultiplicity for the given InnerJoin,
// LeftJoin or FullJoin and returns it. initJoinMultiplicity should only be
// called during construction of the join by the initUnexportedFields methods.
// Panics if called on an operator other than an InnerJoin, LeftJoin, or
// FullJoin.
func initJoinMultiplicity(in RelExpr) {
switch t := in.(type) {
case *ScanExpr:
// All un-limited, unconstrained output columns are unfiltered columns.
if t.HardLimit == 0 && t.Constraint == nil {
multiplicity.UnfilteredCols = relational.OutputCols
}

case *ProjectExpr:
// Project never filters rows, so it passes through unfiltered columns.
unfilteredCols := DeriveJoinMultiplicity(t.Input).UnfilteredCols
multiplicity.UnfilteredCols = unfilteredCols.Intersection(relational.OutputCols)

case *InnerJoinExpr, *LeftJoinExpr, *FullJoinExpr:
// Calculate JoinMultiplicity.
left := t.Child(0).(RelExpr)
right := t.Child(1).(RelExpr)
filters := *t.Child(2).(*FiltersExpr)
multiplicity = GetJoinMultiplicityFromInputs(t.Op(), left, right, filters)

// Use the JoinMultiplicity to determine whether unfiltered columns can be
// passed through.
if multiplicity.JoinPreservesLeftRows() {
multiplicity.UnfilteredCols.UnionWith(DeriveJoinMultiplicity(left).UnfilteredCols)
}
if multiplicity.JoinPreservesRightRows() {
multiplicity.UnfilteredCols.UnionWith(DeriveJoinMultiplicity(right).UnfilteredCols)
}
multiplicity := DeriveJoinMultiplicityFromInputs(t.Op(), left, right, filters)
t.(joinWithMultiplicity).setMultiplicity(multiplicity)

default:
// An empty JoinMultiplicity is returned.
panic(errors.AssertionFailedf("invalid operator type: %v", t.Op()))
}
relational.Rule.MultiplicityProps = multiplicity
return relational.Rule.MultiplicityProps
}

// GetJoinMultiplicityFromInputs returns a JoinMultiplicity that describes how a
// join of the given type with the given inputs and filters will affect the rows
// of its inputs. When possible, DeriveJoinMultiplicity should be called instead
// because GetJoinMultiplicityFromInputs cannot take advantage of a previously
// calculated JoinMultiplicity.
func GetJoinMultiplicityFromInputs(
// GetJoinMultiplicity returns a JoinMultiplicity struct that describes how a
// join operator will affect the rows of its left and right inputs (e.g.
// duplicated and/or filtered). Panics if the method is called on an operator
// that does not support JoinMultiplicity (any operator other than an InnerJoin,
// LeftJoin, or FullJoin).
func GetJoinMultiplicity(in RelExpr) props.JoinMultiplicity {
if join, ok := in.(joinWithMultiplicity); ok {
// JoinMultiplicity has already been initialized during construction of the
// join, so simply return it.
return join.getMultiplicity()
}
panic(errors.AssertionFailedf("invalid operator type: %v", in.Op()))
}

// DeriveJoinMultiplicityFromInputs returns a JoinMultiplicity that describes
// how a join of the given type with the given inputs and filters will affect
// the rows of its inputs. When possible, GetJoinMultiplicity should be called
// instead because DeriveJoinMultiplicityFromInputs cannot take advantage of a
// previously calculated JoinMultiplicity. The UnfilteredCols Relational
// property is used in calculating the JoinMultiplicity, and is lazily derived
// by a call to deriveUnfilteredCols.
func DeriveJoinMultiplicityFromInputs(
joinOp opt.Operator, left, right RelExpr, filters FiltersExpr,
) props.JoinMultiplicity {

Expand All @@ -101,6 +82,53 @@ func GetJoinMultiplicityFromInputs(
}
}

// deriveUnfilteredCols recursively derives the UnfilteredCols field and
// populates the props.Relational.Rule.UnfilteredCols field as it goes to
// make future calls faster.
func deriveUnfilteredCols(in RelExpr) opt.ColSet {
// If the UnfilteredCols property has already been derived, return it
// immediately.
relational := in.Relational()
if relational.IsAvailable(props.UnfilteredCols) {
return relational.Rule.UnfilteredCols
}
relational.Rule.Available |= props.UnfilteredCols
unfilteredCols := opt.ColSet{}

// Derive UnfilteredCols now.
switch t := in.(type) {
case *ScanExpr:
// All un-limited, unconstrained output columns are unfiltered columns.
if t.HardLimit == 0 && t.Constraint == nil {
unfilteredCols.UnionWith(relational.OutputCols)
}

case *ProjectExpr:
// Project never filters rows, so it passes through unfiltered columns.
unfilteredCols.UnionWith(deriveUnfilteredCols(t.Input).Intersection(relational.OutputCols))

case *InnerJoinExpr, *LeftJoinExpr, *FullJoinExpr:
left := t.Child(0).(RelExpr)
right := t.Child(1).(RelExpr)
filters := *t.Child(2).(*FiltersExpr)
multiplicity := DeriveJoinMultiplicityFromInputs(t.Op(), left, right, filters)

// Use the UnfilteredCols to determine whether unfiltered columns can be
// passed through.
if multiplicity.JoinPreservesLeftRows() {
unfilteredCols.UnionWith(deriveUnfilteredCols(left))
}
if multiplicity.JoinPreservesRightRows() {
unfilteredCols.UnionWith(deriveUnfilteredCols(right))
}

default:
// An empty ColSet is returned.
}
relational.Rule.UnfilteredCols = unfilteredCols
return relational.Rule.UnfilteredCols
}

// getJoinLeftMultiplicityVal returns a MultiplicityValue that describes whether
// a join with the given properties would duplicate or filter the rows of its
// left input.
Expand Down Expand Up @@ -217,11 +245,11 @@ func filtersMatchAllLeftRows(left, right RelExpr, filters FiltersExpr) bool {
// Case 1b: if there is at least one not-null foreign key column referencing
// the unfiltered right columns, return true. Otherwise, false.
return makeForeignKeyMap(
md, left.Relational().NotNullCols, DeriveJoinMultiplicity(right).UnfilteredCols) != nil
md, left.Relational().NotNullCols, deriveUnfilteredCols(right)) != nil
}

leftColIDs := left.Relational().NotNullCols
rightColIDs := DeriveJoinMultiplicity(right).UnfilteredCols
rightColIDs := deriveUnfilteredCols(right)
if rightColIDs.Empty() {
// Right input has no unfiltered columns.
return false
Expand Down Expand Up @@ -355,8 +383,8 @@ func makeForeignKeyMap(
leftCol := fkTableID.ColumnID(leftOrd)
rightCol := refTableID.ColumnID(rightOrd)
if !leftNotNullCols.Contains(leftCol) {
// Not all FK columns are part of the equality conditions. There are two
// cases:
// Not all FK columns are part of the equality conditions. There are
// two cases:
// 1. MATCH SIMPLE/PARTIAL: if this column is nullable, rows from this
// foreign key are not guaranteed to match.
// 2. MATCH FULL: FK rows are still guaranteed to match because the
Expand Down
Loading

0 comments on commit 6aaf3d5

Please sign in to comment.