Skip to content

Commit

Permalink
Merge #51164
Browse files Browse the repository at this point in the history
51164: opt: build partial index predicates as FiltersExprs r=RaduBerinde a=mgartner

This commit updates optbuilder to build a partial index predicate as
`FiltersExpr` instead of an arbitrary ScalarExpr. The primary reason for
doing so is to make it easier to calculate statistics for partial index
scans. By representing the predicate as a `FiltersExpr`, the statistics
builder can generate stats for partial index scans using existing
functions that help generate statistics for Selects.

`SimplifyFilters` and `ConsolidateFilters` are manually called on the
predicate so that it is similar in form to a normalized Select filter.
Flattening the expression and generating constraints for each top-level
conjunction will make it simpler and cheaper to calculate stats
accurately for unapplied conjunctions.

This commit also updates `Implicator` to handle predicates that are
`FiltersExpr`s and have `RangeExpr`s in them. This change benefits the
performance of `Implicator` significantly in some of the slowest
benchmark cases because flattening predicate conjunctions means that the
exact-match fast path can prove implication in more cases. This change
only worsens performance significantly in some of the fastest cases that
take <400ns/op.

    name                                               old time/op  new time/op  delta
    Implicator/single-exact-match-16                   61.7ns ± 0%  89.1ns ± 0%  +44.47%  (p=0.008 n=5+5)
    Implicator/single-inexact-match-16                  878ns ± 0%   912ns ± 1%   +3.94%  (p=0.008 n=5+5)
    Implicator/range-inexact-match-16                  2.32µs ± 2%  2.40µs ± 1%   +3.78%  (p=0.008 n=5+5)
    Implicator/single-exact-match-extra-filters-16      321ns ± 1%   371ns ± 1%  +15.70%  (p=0.008 n=5+5)
    Implicator/single-inexact-match-extra-filters-16   4.00µs ± 0%  4.11µs ± 1%   +2.78%  (p=0.008 n=5+5)
    Implicator/multi-column-and-exact-match-16          455ns ± 0%   100ns ± 1%  -78.02%  (p=0.016 n=4+5)
    Implicator/multi-column-and-inexact-match-16       1.94µs ± 1%  1.97µs ± 0%   +1.32%  (p=0.040 n=5+5)
    Implicator/multi-column-or-exact-match-16          62.1ns ± 0%  91.4ns ± 0%  +47.23%  (p=0.008 n=5+5)
    Implicator/multi-column-or-exact-match-reverse-16  1.72µs ± 0%  1.79µs ± 0%   +4.08%  (p=0.008 n=5+5)
    Implicator/multi-column-or-inexact-match-16        2.18µs ± 0%  2.25µs ± 1%   +3.56%  (p=0.008 n=5+5)
    Implicator/and-filters-do-not-imply-pred-16        3.65µs ± 2%  3.76µs ± 5%     ~     (p=0.056 n=5+5)
    Implicator/or-filters-do-not-imply-pred-16          774ns ± 1%   850ns ± 1%   +9.71%  (p=0.008 n=5+5)
    Implicator/many-columns-exact-match10-16           4.93µs ± 1%  0.30µs ± 1%  -93.87%  (p=0.008 n=5+5)
    Implicator/many-columns-inexact-match10-16         11.3µs ± 1%  11.1µs ± 0%   -1.86%  (p=0.008 n=5+5)
    Implicator/many-columns-exact-match100-16           352µs ± 1%    18µs ± 1%  -94.92%  (p=0.008 n=5+5)
    Implicator/many-columns-inexact-match100-16         382µs ± 1%   372µs ± 1%   -2.47%  (p=0.008 n=5+5)

Release note: None

Co-authored-by: Marcus Gartner <[email protected]>
  • Loading branch information
craig[bot] and mgartner committed Jul 9, 2020
2 parents ba2deac + be87984 commit e84d0cb
Show file tree
Hide file tree
Showing 7 changed files with 168 additions and 49 deletions.
7 changes: 5 additions & 2 deletions pkg/sql/opt/norm/testdata/rules/prune_cols
Original file line number Diff line number Diff line change
Expand Up @@ -1956,8 +1956,11 @@ delete partial_indexes
├── scan partial_indexes
│ ├── columns: a:4!null b:5 c:6
│ ├── partial index predicates
│ │ ├── secondary: c:6 = 'foo'
│ │ └── secondary: (a:4 > b:5) AND (c:6 = 'bar')
│ │ ├── secondary: filters
│ │ │ └── c:6 = 'foo' [outer=(6), constraints=(/6: [/'foo' - /'foo']; tight), fd=()-->(6)]
│ │ └── secondary: filters
│ │ ├── a:4 > b:5 [outer=(4,5), constraints=(/4: (/NULL - ]; /5: (/NULL - ])]
│ │ └── c:6 = 'bar' [outer=(6), constraints=(/6: [/'bar' - /'bar']; tight), fd=()-->(6)]
│ ├── key: (4)
│ └── fd: (4)-->(5,6)
└── projections
Expand Down
30 changes: 25 additions & 5 deletions pkg/sql/opt/optbuilder/select.go
Original file line number Diff line number Diff line change
Expand Up @@ -702,16 +702,36 @@ func (b *Builder) addPartialIndexPredicatesForTable(tabMeta *opt.TableMeta) {
}

texpr := tableScope.resolveAndRequireType(expr, types.Bool)

var scalar opt.ScalarExpr
b.factory.FoldingControl().TemporarilyDisallowStableFolds(func() {
scalar = b.buildScalar(texpr, tableScope, nil, nil, nil)
})
// Check if the expression contains non-immutable operators.
var sharedProps props.Shared
memo.BuildSharedProps(scalar, &sharedProps)
if !sharedProps.VolatilitySet.HasStable() && !sharedProps.VolatilitySet.HasVolatile() {
tabMeta.AddPartialIndexPredicate(indexOrd, scalar)

// Wrap the scalar in a FiltersItem.
filter := b.factory.ConstructFiltersItem(scalar)

// If the expression contains non-immutable operators, do not add it to
// the table metadata.
if filter.ScalarProps().VolatilitySet.HasStable() || filter.ScalarProps().VolatilitySet.HasVolatile() {
return
}

// Wrap the filter in a FiltersExpr.
//
// Run SimplifyFilters so that adjacent top-level AND expressions are
// flattened into individual FiltersItems, like they would be during
// normalization of a SELECT query.
//
// Run ConsolidateFilters so that adjacent top-level FiltersItems that
// constrain a single variable are combined into a RangeExpr, like they
// would be during normalization of a SELECT query.
filters := memo.FiltersExpr{filter}
filters = b.factory.CustomFuncs().SimplifyFilters(filters)
filters = b.factory.CustomFuncs().ConsolidateFilters(filters)

// Add the filters to the table metadata.
tabMeta.AddPartialIndexPredicate(indexOrd, &filters)
}
}

Expand Down
7 changes: 5 additions & 2 deletions pkg/sql/opt/optbuilder/testdata/delete
Original file line number Diff line number Diff line change
Expand Up @@ -464,8 +464,11 @@ delete partial_indexes
├── scan partial_indexes
│ ├── columns: a:4!null b:5 c:6
│ └── partial index predicates
│ ├── secondary: c:6 = 'foo'
│ └── secondary: (a:4 > b:5) AND (c:6 = 'bar')
│ ├── secondary: filters
│ │ └── c:6 = 'foo'
│ └── secondary: filters
│ ├── a:4 > b:5
│ └── c:6 = 'bar'
└── projections
├── c:6 = 'foo' [as=partial_index_del1:7]
└── (a:4 > b:5) AND (c:6 = 'bar') [as=partial_index_del2:8]
7 changes: 5 additions & 2 deletions pkg/sql/opt/optbuilder/testdata/select
Original file line number Diff line number Diff line change
Expand Up @@ -1289,5 +1289,8 @@ project
└── scan partial_index
├── columns: k:1!null u:2 v:3
└── partial index predicates
├── u: u:2 = 1
└── v: ((v:3 > 100) AND (v:3 < 200)) AND (u:2 > 50)
├── u: filters
│ └── u:2 = 1
└── v: filters
├── (v:3 > 100) AND (v:3 < 200)
└── u:2 > 50
160 changes: 125 additions & 35 deletions pkg/sql/opt/partialidx/implicator.go
Original file line number Diff line number Diff line change
Expand Up @@ -16,6 +16,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/opt/memo"
"github.com/cockroachdb/cockroach/pkg/sql/opt/norm"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/util"
)

// Implicator is used to 1) prove that query filters imply a partial index
Expand Down Expand Up @@ -152,61 +153,94 @@ func (im *Implicator) Init(f *norm.Factory, md *opt.Metadata, evalCtx *tree.Eval
// proven, nil and false are returned. See Implicator for more details on how
// implication is proven and how the remaining filters are determined.
func (im *Implicator) FiltersImplyPredicate(
filters memo.FiltersExpr, pred opt.ScalarExpr,
filters memo.FiltersExpr, pred memo.FiltersExpr,
) (remainingFilters memo.FiltersExpr, ok bool) {
// Empty filters are equivalent to True, which only implies True.
if len(filters) == 0 && pred == memo.TrueSingleton {
return filters, true
// An empty FiltersExpr is equivalent to True, which is only implied by
// True.
if len(pred) == 0 {
return filters, len(filters) == 0
}

// Next, check for exact matches at the root FiltersExpr. This check is not
// Check for exact matches for all FiltersItems in pred. This check is not
// necessary for correctness because the recursive approach below handles
// all cases. However, this is a faster path for common cases where
// expressions in filters are exact matches to the entire predicate.
for i := range filters {
c := filters[i].Condition

// If the FiltersItem's condition is an exact match to the predicate,
// remove the FilterItem from the remaining filters and return true.
if c == pred {
return filters.RemoveFiltersItem(&filters[i]), true
}

// If the FiltersItem's condition is a RangeExpr, unbox it and check for
// an exact match. RangeExprs are only created in the
// ConsolidateSelectFilters normalization rule and only exist as direct
// children of a FiltersItem. The predicate will not contain a
// RangeExpr, but the predicate may be an exact match to a RangeExpr's
// child.
if r, ok := c.(*memo.RangeExpr); ok {
if r.And == pred {
return filters.RemoveFiltersItem(&filters[i]), true
}
}
if remFilters, ok := im.filtersImplyPredicateFastPath(filters, pred); ok {
return remFilters, true
}

// Populate the constraint cache with any constraints already generated for
// FiltersItems that are atoms.
for _, f := range filters {
op := f.Condition.Op()
if f.ScalarProps().Constraints != nil && op != opt.AndOp && op != opt.OrOp && op != opt.RangeOp {
im.cacheConstraint(f.Condition, f.ScalarProps().Constraints, f.ScalarProps().TightConstraints)
}
}
im.warmCache(filters)
im.warmCache(pred)

// If no exact match was found, recursively check the sub-expressions of the
// filters and predicate. Use exactMatches to keep track of expressions in
// filters that exactly matches expressions in pred, so that the can be
// removed from the remaining filters.
exactMatches := make(map[opt.Expr]struct{})
if im.scalarExprImpliesPredicate(&filters, pred, exactMatches) {
if im.scalarExprImpliesPredicate(&filters, &pred, exactMatches) {
remainingFilters = im.simplifyFiltersExpr(filters, exactMatches)
return remainingFilters, true
}

return nil, false
}

// filtersImplyPredicateFastPath returns remaining filters and true if every
// FiltersItem condition in pred exists in filters. This is a faster path for
// proving implication in common cases where expressions in filters are exact
// matches to expressions in the predicate.
//
// If this function returns false it is NOT proven that the filters do not imply
// pred. Instead, it indicates that the slower recursive walk of both expression
// trees is required to prove or disprove implication.
func (im *Implicator) filtersImplyPredicateFastPath(
filters memo.FiltersExpr, pred memo.FiltersExpr,
) (remainingFilters memo.FiltersExpr, ok bool) {
var filtersToRemove util.FastIntSet

// For every FiltersItem in pred, search for a matching FiltersItem in
// filters.
for i := range pred {
predCondition := pred[i].Condition
exactMatchFound := false
for j := range filters {
filterCondition := filters[j].Condition

// If there is a match, track the index of the filter so that it can
// be removed from the remaining filters and move on to the next
// predicate FiltersItem.
if predCondition == filterCondition {
exactMatchFound = true
filtersToRemove.Add(j)
break
}
}

// If an exact match to the predicate filter was not found in filters,
// then implication cannot be proven.
if !exactMatchFound {
return nil, false
}
}

// Return an empty FiltersExpr if all filters are to be removed.
if len(filters) == filtersToRemove.Len() {
return memo.FiltersExpr{}, true
}

// Build the remaining filters from FiltersItems in filters which did not
// have matches in the predicate.
remainingFilters = make(memo.FiltersExpr, 0, len(filters)-filtersToRemove.Len())
for i := range filters {
if !filtersToRemove.Contains(i) {
remainingFilters = append(remainingFilters, filters[i])
}
}
return remainingFilters, true
}

// scalarExprImpliesPredicate returns true if the expression e implies the
// ScalarExpr pred. If e or any of its encountered sub-expressions are exact
// matches to expressions within pred, they are added to the exactMatches set
Expand Down Expand Up @@ -256,6 +290,21 @@ func (im *Implicator) filtersExprImpliesPredicate(
e *memo.FiltersExpr, pred opt.ScalarExpr, exactMatches map[opt.Expr]struct{},
) bool {
switch pt := pred.(type) {
case *memo.FiltersExpr:
// AND-expr A => AND-expr B iff A => each of B's children.
for i := range *pt {
if !im.filtersExprImpliesPredicate(e, (*pt)[i].Condition, exactMatches) {
return false
}
}
return true

case *memo.RangeExpr:
// AND-expr A => AND-expr B iff A => each of B's children.
and := pt.And.(*memo.AndExpr)
return im.filtersExprImpliesPredicate(e, and.Left, exactMatches) &&
im.filtersExprImpliesPredicate(e, and.Right, exactMatches)

case *memo.AndExpr:
// AND-expr A => AND-expr B iff A => each of B's children.
return im.filtersExprImpliesPredicate(e, pt.Left, exactMatches) &&
Expand Down Expand Up @@ -317,6 +366,21 @@ func (im *Implicator) andExprImpliesPredicate(
// exactMatches. See FiltersImplyPredicate (rule #3) for more details.
func (im *Implicator) orExprImpliesPredicate(e *memo.OrExpr, pred opt.ScalarExpr) bool {
switch pt := pred.(type) {
case *memo.FiltersExpr:
// OR-expr A => AND-expr B iff A => each of B's children.
for i := range *pt {
if !im.orExprImpliesPredicate(e, (*pt)[i].Condition) {
return false
}
}
return true

case *memo.RangeExpr:
// OR-expr A => AND-expr B iff A => each of B's children.
and := pt.And.(*memo.AndExpr)
return im.orExprImpliesPredicate(e, and.Left) &&
im.orExprImpliesPredicate(e, and.Right)

case *memo.AndExpr:
// OR-expr A => AND-expr B iff A => each of B's children.
return im.orExprImpliesPredicate(e, pt.Left) &&
Expand Down Expand Up @@ -358,11 +422,25 @@ func (im *Implicator) atomImpliesPredicate(
e opt.ScalarExpr, pred opt.ScalarExpr, exactMatches map[opt.Expr]struct{},
) bool {
switch pt := pred.(type) {
case *memo.FiltersExpr:
// atom A => AND-expr B iff A => each of B's children.
for i := range *pt {
if !im.atomImpliesPredicate(e, (*pt)[i].Condition, exactMatches) {
return false
}
}
return true

case *memo.RangeExpr:
// atom A => AND-expr B iff A => each of B's children.
and := pt.And.(*memo.AndExpr)
return im.atomImpliesPredicate(e, and.Left, exactMatches) &&
im.atomImpliesPredicate(e, and.Right, exactMatches)

case *memo.AndExpr:
// atom A => AND-expr B iff A => each of B's children.
leftPredImplied := im.atomImpliesPredicate(e, pt.Left, exactMatches)
rightPredImplied := im.atomImpliesPredicate(e, pt.Right, exactMatches)
return leftPredImplied && rightPredImplied
return im.atomImpliesPredicate(e, pt.Left, exactMatches) &&
im.atomImpliesPredicate(e, pt.Right, exactMatches)

case *memo.OrExpr:
// atom A => OR-expr B iff A => any of B's children.
Expand Down Expand Up @@ -452,6 +530,18 @@ func (im *Implicator) fetchConstraint(e opt.ScalarExpr) (_ *constraint.Set, tigh
return nil, false, false
}

// warmCache adds top-level atom constraints of filters to the cache. This
// prevents rebuilding constraints that have already have been built, reducing
// the overhead of proving implication.
func (im *Implicator) warmCache(filters memo.FiltersExpr) {
for _, f := range filters {
op := f.Condition.Op()
if f.ScalarProps().Constraints != nil && op != opt.AndOp && op != opt.OrOp && op != opt.RangeOp {
im.cacheConstraint(f.Condition, f.ScalarProps().Constraints, f.ScalarProps().TightConstraints)
}
}
}

// simplifyFiltersExpr returns a new FiltersExpr with any expressions in e that
// exist in exactMatches removed.
//
Expand Down
4 changes: 2 additions & 2 deletions pkg/sql/opt/partialidx/implicator_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -101,7 +101,7 @@ func TestImplicator(t *testing.T) {
}

// Build the predicate from the second split, everything after "=>".
pred, err := makeScalarExpr(splitInput[1], &semaCtx, &evalCtx, &f)
pred, err := makeFiltersExpr(splitInput[1], &semaCtx, &evalCtx, &f)
if err != nil {
d.Fatalf(t, "unexpected error while building predicate: %v\n", err)
}
Expand Down Expand Up @@ -266,7 +266,7 @@ func BenchmarkImplicator(b *testing.B) {
}

// Build the predicate.
pred, err := makeScalarExpr(tc.pred, &semaCtx, &evalCtx, &f)
pred, err := makeFiltersExpr(tc.pred, &semaCtx, &evalCtx, &f)
if err != nil {
b.Fatalf("unexpected error while building predicate: %v\n", err)
}
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/opt/xform/custom_funcs.go
Original file line number Diff line number Diff line change
Expand Up @@ -203,7 +203,7 @@ func (c *CustomFuncs) GeneratePartialIndexScans(
iter := makeScanIndexIter(c.e.mem, scanPrivate, rejectNonPartialIndexes)
for iter.Next() {
pred := tabMeta.PartialIndexPredicates[iter.IndexOrdinal()]
remainingFilters, ok := c.im.FiltersImplyPredicate(filters, pred)
remainingFilters, ok := c.im.FiltersImplyPredicate(filters, *pred.(*memo.FiltersExpr))
if !ok {
// The filters do not imply the predicate, so the partial index
// cannot be used.
Expand Down

0 comments on commit e84d0cb

Please sign in to comment.