diff --git a/pkg/sql/opt/memo/best_expr.go b/pkg/sql/opt/memo/best_expr.go index 1542508e1fcb..bf0efa735b25 100644 --- a/pkg/sql/opt/memo/best_expr.go +++ b/pkg/sql/opt/memo/best_expr.go @@ -82,6 +82,15 @@ func (be *BestExpr) Operator() opt.Operator { return be.op } +// Expr returns the memo expression referenced by this best expression. Note +// that if the best expression is an enforcer (like a Sort), then the memo +// expression is wrapped by the enforcer (maybe even by multiple enforcers). +// This means that the same ExprID can be returned by different best expressions +// in the same group, each of which would have a different Operator type. +func (be *BestExpr) Expr() ExprID { + return be.eid +} + // Group returns the memo group which contains this best expression. func (be *BestExpr) Group() GroupID { return be.eid.Group diff --git a/pkg/sql/opt/memo/expr.go b/pkg/sql/opt/memo/expr.go index 98e86605deb9..016aa5d52fcf 100644 --- a/pkg/sql/opt/memo/expr.go +++ b/pkg/sql/opt/memo/expr.go @@ -39,6 +39,10 @@ type ExprID struct { Expr ExprOrdinal } +// InvalidExprID is the uninitialized ExprID that never points to a valid +// expression. +var InvalidExprID = ExprID{} + // MakeNormExprID returns the id of the normalized expression for the given // group. func MakeNormExprID(group GroupID) ExprID { diff --git a/pkg/sql/opt/memo/expr_view.go b/pkg/sql/opt/memo/expr_view.go index ea3637f95802..a842747cce1a 100644 --- a/pkg/sql/opt/memo/expr_view.go +++ b/pkg/sql/opt/memo/expr_view.go @@ -120,7 +120,7 @@ func (ev ExprView) Physical() *PhysicalProps { if ev.best == normBestOrdinal { panic("physical properties are not available when traversing the normalized tree") } - return ev.mem.LookupPhysicalProps(ev.lookupBestExpr().required) + return ev.mem.LookupPhysicalProps(ev.bestExpr().required) } // Group returns the memo group containing this expression. @@ -138,7 +138,7 @@ func (ev ExprView) Child(nth int) ExprView { group := ev.ChildGroup(nth) return MakeNormExprView(ev.mem, group) } - return MakeExprView(ev.mem, ev.lookupBestExpr().Child(nth)) + return MakeExprView(ev.mem, ev.bestExpr().Child(nth)) } // ChildCount returns the number of expressions that are inputs to this @@ -147,7 +147,7 @@ func (ev ExprView) ChildCount() int { if ev.best == normBestOrdinal { return ev.mem.NormExpr(ev.group).ChildCount() } - return ev.lookupBestExpr().ChildCount() + return ev.bestExpr().ChildCount() } // ChildGroup returns the memo group containing the nth child of this parent @@ -156,7 +156,7 @@ func (ev ExprView) ChildGroup(nth int) GroupID { if ev.best == normBestOrdinal { return ev.mem.NormExpr(ev.group).ChildGroup(ev.mem, nth) } - return ev.lookupBestExpr().Child(nth).group + return ev.bestExpr().Child(nth).group } // Private returns any private data associated with this expression, or nil if @@ -165,7 +165,7 @@ func (ev ExprView) Private() interface{} { if ev.best == normBestOrdinal { return ev.mem.NormExpr(ev.group).Private(ev.mem) } - return ev.mem.Expr(ev.lookupBestExpr().eid).Private(ev.mem) + return ev.mem.Expr(ev.bestExpr().eid).Private(ev.mem) } // Metadata returns the metadata that's specific to this expression tree. Some @@ -175,11 +175,21 @@ func (ev ExprView) Metadata() *opt.Metadata { return ev.mem.metadata } -func (ev ExprView) lookupChildGroup(nth int) *group { +// Cost returns the cost of executing this expression tree, as estimated by the +// optimizer. It is not available when the ExprView is traversing the normalized +// expression tree. +func (ev ExprView) Cost() Cost { + if ev.best == normBestOrdinal { + panic("Cost is not available when traversing the normalized tree") + } + return ev.mem.bestExpr(BestExprID{group: ev.group, ordinal: ev.best}).cost +} + +func (ev ExprView) childGroup(nth int) *group { return ev.mem.group(ev.ChildGroup(nth)) } -func (ev ExprView) lookupBestExpr() *BestExpr { +func (ev ExprView) bestExpr() *BestExpr { return ev.mem.group(ev.group).bestExpr(ev.best) } @@ -328,7 +338,7 @@ func (ev ExprView) formatRelational(tp treeprinter.Node, flags ExprFmtFlags) { } if !flags.HasFlags(ExprFmtHideCost) && ev.best != normBestOrdinal { - tp.Childf("cost: %.2f", ev.lookupBestExpr().cost) + tp.Childf("cost: %.2f", ev.bestExpr().cost) } // Format weak keys. diff --git a/pkg/sql/opt/memo/logical_props_factory.go b/pkg/sql/opt/memo/logical_props_factory.go index 78c823616354..3be2e111dfe5 100644 --- a/pkg/sql/opt/memo/logical_props_factory.go +++ b/pkg/sql/opt/memo/logical_props_factory.go @@ -116,7 +116,7 @@ func (f logicalPropsFactory) constructScanProps(ev ExprView) LogicalProps { func (f logicalPropsFactory) constructSelectProps(ev ExprView) LogicalProps { props := LogicalProps{Relational: &RelationalProps{}} - inputProps := ev.lookupChildGroup(0).logical.Relational + inputProps := ev.childGroup(0).logical.Relational // Inherit input properties as starting point. *props.Relational = *inputProps @@ -129,7 +129,7 @@ func (f logicalPropsFactory) constructSelectProps(ev ExprView) LogicalProps { func (f logicalPropsFactory) constructProjectProps(ev ExprView) LogicalProps { props := LogicalProps{Relational: &RelationalProps{}} - inputProps := ev.lookupChildGroup(0).logical.Relational + inputProps := ev.childGroup(0).logical.Relational // Use output columns from projection list. props.Relational.OutputCols = opt.ColListToSet(ev.Child(1).Private().(opt.ColList)) @@ -154,8 +154,8 @@ func (f logicalPropsFactory) constructProjectProps(ev ExprView) LogicalProps { func (f logicalPropsFactory) constructJoinProps(ev ExprView) LogicalProps { props := LogicalProps{Relational: &RelationalProps{}} - leftProps := ev.lookupChildGroup(0).logical.Relational - rightProps := ev.lookupChildGroup(1).logical.Relational + leftProps := ev.childGroup(0).logical.Relational + rightProps := ev.childGroup(1).logical.Relational // Output columns are union of columns from left and right inputs, except // in case of semi and anti joins, which only project the left columns. @@ -199,7 +199,7 @@ func (f logicalPropsFactory) constructJoinProps(ev ExprView) LogicalProps { func (f logicalPropsFactory) constructGroupByProps(ev ExprView) LogicalProps { props := LogicalProps{Relational: &RelationalProps{}} - inputProps := ev.lookupChildGroup(0).logical.Relational + inputProps := ev.childGroup(0).logical.Relational // Output columns are the union of grouping columns with columns from the // aggregate projection list. @@ -241,8 +241,8 @@ func (f logicalPropsFactory) constructGroupByProps(ev ExprView) LogicalProps { func (f logicalPropsFactory) constructSetProps(ev ExprView) LogicalProps { props := LogicalProps{Relational: &RelationalProps{}} - leftProps := ev.lookupChildGroup(0).logical.Relational - rightProps := ev.lookupChildGroup(1).logical.Relational + leftProps := ev.childGroup(0).logical.Relational + rightProps := ev.childGroup(1).logical.Relational colMap := *ev.Private().(*SetOpColMap) if len(colMap.Out) != len(colMap.Left) || len(colMap.Out) != len(colMap.Right) { panic(fmt.Errorf("lists in SetOpColMap are not all the same length. new:%d, left:%d, right:%d", @@ -311,7 +311,7 @@ func (f logicalPropsFactory) constructMax1RowProps(ev ExprView) LogicalProps { func (f logicalPropsFactory) passThroughRelationalProps(ev ExprView, childIdx int) LogicalProps { // Properties are immutable after construction, so just inherit relational // props pointer from child. - return LogicalProps{Relational: ev.lookupChildGroup(childIdx).logical.Relational} + return LogicalProps{Relational: ev.childGroup(childIdx).logical.Relational} } func (f logicalPropsFactory) constructScalarProps(ev ExprView) LogicalProps { @@ -326,7 +326,7 @@ func (f logicalPropsFactory) constructScalarProps(ev ExprView) LogicalProps { // By default, union outer cols from all children, both relational and scalar. for i := 0; i < ev.ChildCount(); i++ { - logical := &ev.lookupChildGroup(i).logical + logical := &ev.childGroup(i).logical if logical.Scalar != nil { props.Scalar.OuterCols.UnionWith(logical.Scalar.OuterCols) } else { diff --git a/pkg/sql/opt/memo/memo.go b/pkg/sql/opt/memo/memo.go index 4e9d7461388e..e53ce4e38aa3 100644 --- a/pkg/sql/opt/memo/memo.go +++ b/pkg/sql/opt/memo/memo.go @@ -188,8 +188,8 @@ func (m *Memo) GroupProperties(group GroupID) *LogicalProps { return &m.groups[group].logical } -// GroupByFingerprint returns the group of the expression that has the -// given fingerprint. +// GroupByFingerprint returns the group of the expression that has the given +// fingerprint. func (m *Memo) GroupByFingerprint(f Fingerprint) GroupID { return m.exprMap[f] } @@ -252,7 +252,6 @@ func (m *Memo) MemoizeNormExpr(evalCtx *tree.EvalContext, norm Expr) GroupID { if m.exprMap[norm.Fingerprint()] != 0 { panic("normalized expression has been entered into the memo more than once") } - mgrp := m.newGroup(norm) ev := MakeNormExprView(m, mgrp.id) logPropsFactory := logicalPropsFactory{evalCtx: evalCtx} diff --git a/pkg/sql/opt/norm/factory.go b/pkg/sql/opt/norm/factory.go index 80eedfbbd557..7777fa5706da 100644 --- a/pkg/sql/opt/norm/factory.go +++ b/pkg/sql/opt/norm/factory.go @@ -24,6 +24,21 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/sem/types" ) +// MatchedRuleFunc defines the callback function for the NotifyOnMatchedRule +// event supported by the optimizer and factory. It is invoked each time an +// optimization rule (Normalize or Explore) has been matched. The name of the +// matched rule is passed as a parameter. If the function returns false, then +// the rule is not applied (i.e. skipped). +type MatchedRuleFunc func(ruleName opt.RuleName) bool + +// AppliedRuleFunc defines the callback function for the NotifyOnAppliedRule +// event supported by the optimizer and factory. It is invoked each time an +// optimization rule (Normalize or Explore) has been applied. The function is +// called with the name of the rule and the memo group it affected. If the rule +// was an exploration rule, then the added parameter gives the number of +// expressions added to the group by the rule. +type AppliedRuleFunc func(ruleName opt.RuleName, group memo.GroupID, added int) + //go:generate optgen -out factory.og.go factory ../ops/*.opt rules/*.opt // Factory constructs a normalized expression tree within the memo. As each @@ -57,10 +72,15 @@ type Factory struct { // map, and will skip application of the rule. ruleCycles map[memo.Fingerprint]bool - // onRuleMatch is the callback function that is invoked each time a normalize + // matchedRule is the callback function that is invoked each time a normalize // rule has been matched by the factory. It can be set via a call to the - // SetOnRuleMatch method. - onRuleMatch func(ruleName opt.RuleName) bool + // NotifyOnMatchedRule method. + matchedRule MatchedRuleFunc + + // appliedRule is the callback function which is invoked each time a normalize + // rule has been applied by the factory. It can be set via a call to the + // NotifyOnAppliedRule method. + appliedRule AppliedRuleFunc } // NewFactory returns a new Factory structure with a new, blank memo structure @@ -77,17 +97,23 @@ func NewFactory(evalCtx *tree.EvalContext) *Factory { // expression tree becomes the output expression tree (because no transforms // are applied). func (f *Factory) DisableOptimizations() { - f.SetOnRuleMatch(func(opt.RuleName) bool { return false }) + f.NotifyOnMatchedRule(func(opt.RuleName) bool { return false }) } -// SetOnRuleMatch sets a callback function which is invoked each time a -// normalize rule has been matched by the factory. If the function returns -// false, then the rule is not applied. By default, all rules are applied, but -// callers can set the callback function to override the default behavior. In +// NotifyOnMatchedRule sets a callback function which is invoked each time a +// normalize rule has been matched by the factory. If matchedRule is nil, then +// no further notifications are sent, and all rules are applied by default. In // addition, callers can invoke the DisableOptimizations convenience method to // disable all rules. -func (f *Factory) SetOnRuleMatch(onRuleMatch func(ruleName opt.RuleName) bool) { - f.onRuleMatch = onRuleMatch +func (f *Factory) NotifyOnMatchedRule(matchedRule MatchedRuleFunc) { + f.matchedRule = matchedRule +} + +// NotifyOnAppliedRule sets a callback function which is invoked each time a +// normalize rule has been applied by the factory. If appliedRule is nil, then +// no further notifications are sent. +func (f *Factory) NotifyOnAppliedRule(appliedRule AppliedRuleFunc) { + f.appliedRule = appliedRule } // Memo returns the memo structure that the factory is operating upon. diff --git a/pkg/sql/opt/norm/factory.og.go b/pkg/sql/opt/norm/factory.og.go index 4aee24fce48f..8f23aeddc2c7 100644 --- a/pkg/sql/opt/norm/factory.og.go +++ b/pkg/sql/opt/norm/factory.og.go @@ -140,7 +140,7 @@ func (_f *Factory) ConstructSelect( _andExpr := _f.mem.NormExpr(filter).AsAnd() if _andExpr != nil { conditions := _andExpr.Conditions() - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.EnsureSelectFiltersAnd) { + if _f.matchedRule == nil || _f.matchedRule(opt.EnsureSelectFiltersAnd) { _group = _f.ConstructSelect( input, _f.ConstructFilters( @@ -148,6 +148,9 @@ func (_f *Factory) ConstructSelect( ), ) _f.mem.AddAltFingerprint(_selectExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.EnsureSelectFiltersAnd, _group, 0) + } return _group } } @@ -157,7 +160,7 @@ func (_f *Factory) ConstructSelect( { _expr := _f.mem.NormExpr(filter) if !(_expr.Operator() == opt.FiltersOp || _expr.Operator() == opt.AndOp || _expr.Operator() == opt.TrueOp || _expr.Operator() == opt.FalseOp) { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.EnsureSelectFilters) { + if _f.matchedRule == nil || _f.matchedRule(opt.EnsureSelectFilters) { _group = _f.ConstructSelect( input, _f.ConstructFilters( @@ -165,6 +168,9 @@ func (_f *Factory) ConstructSelect( ), ) _f.mem.AddAltFingerprint(_selectExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.EnsureSelectFilters, _group, 0) + } return _group } } @@ -174,9 +180,12 @@ func (_f *Factory) ConstructSelect( { _trueExpr := _f.mem.NormExpr(filter).AsTrue() if _trueExpr != nil { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.EliminateSelect) { + if _f.matchedRule == nil || _f.matchedRule(opt.EliminateSelect) { _group = input _f.mem.AddAltFingerprint(_selectExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.EliminateSelect, _group, 0) + } return _group } } @@ -188,12 +197,15 @@ func (_f *Factory) ConstructSelect( if _selectExpr2 != nil { input := _selectExpr2.Input() innerFilter := _selectExpr2.Filter() - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.MergeSelects) { + if _f.matchedRule == nil || _f.matchedRule(opt.MergeSelects) { _group = _f.ConstructSelect( input, _f.concatFilters(innerFilter, filter), ) _f.mem.AddAltFingerprint(_selectExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.MergeSelects, _group, 0) + } return _group } } @@ -212,7 +224,7 @@ func (_f *Factory) ConstructSelect( for _, _item := range _f.mem.LookupList(_filtersExpr.Conditions()) { condition := _item if !_f.isCorrelatedCols(condition, _f.synthesizedCols(project)) { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.PushSelectIntoProject) { + if _f.matchedRule == nil || _f.matchedRule(opt.PushSelectIntoProject) { _group = _f.ConstructSelect( _f.projectNoCycle(_f.ConstructSelect( input, @@ -225,6 +237,9 @@ func (_f *Factory) ConstructSelect( ), ) _f.mem.AddAltFingerprint(_selectExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.PushSelectIntoProject, _group, 0) + } return _group } } @@ -246,7 +261,7 @@ func (_f *Factory) ConstructSelect( for _, _item := range _f.mem.LookupList(_filtersExpr.Conditions()) { condition := _item if !_f.isCorrelated(condition, right) { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.PushSelectIntoJoinLeft) { + if _f.matchedRule == nil || _f.matchedRule(opt.PushSelectIntoJoinLeft) { _group = _f.ConstructSelect( _f.DynamicConstruct( _f.mem.NormExpr(input).Operator(), @@ -266,6 +281,9 @@ func (_f *Factory) ConstructSelect( ), ) _f.mem.AddAltFingerprint(_selectExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.PushSelectIntoJoinLeft, _group, 0) + } return _group } } @@ -287,7 +305,7 @@ func (_f *Factory) ConstructSelect( for _, _item := range _f.mem.LookupList(_filtersExpr.Conditions()) { condition := _item if !_f.isCorrelated(condition, left) { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.PushSelectIntoJoinRight) { + if _f.matchedRule == nil || _f.matchedRule(opt.PushSelectIntoJoinRight) { _group = _f.ConstructSelect( _f.DynamicConstruct( _f.mem.NormExpr(input).Operator(), @@ -307,6 +325,9 @@ func (_f *Factory) ConstructSelect( ), ) _f.mem.AddAltFingerprint(_selectExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.PushSelectIntoJoinRight, _group, 0) + } return _group } } @@ -322,7 +343,7 @@ func (_f *Factory) ConstructSelect( left := _expr.ChildGroup(_f.mem, 0) right := _expr.ChildGroup(_f.mem, 1) on := _expr.ChildGroup(_f.mem, 2) - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.MergeSelectInnerJoin) { + if _f.matchedRule == nil || _f.matchedRule(opt.MergeSelectInnerJoin) { _group = _f.DynamicConstruct( _f.mem.NormExpr(input).Operator(), memo.DynamicOperands{ @@ -332,6 +353,9 @@ func (_f *Factory) ConstructSelect( }, ) _f.mem.AddAltFingerprint(_selectExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.MergeSelectInnerJoin, _group, 0) + } return _group } } @@ -351,7 +375,7 @@ func (_f *Factory) ConstructSelect( for _, _item := range _f.mem.LookupList(_filtersExpr.Conditions()) { condition := _item if !_f.isCorrelated(condition, aggregations) { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.PushSelectIntoGroupBy) { + if _f.matchedRule == nil || _f.matchedRule(opt.PushSelectIntoGroupBy) { _group = _f.ConstructSelect( _f.ConstructGroupBy( _f.ConstructSelect( @@ -368,6 +392,9 @@ func (_f *Factory) ConstructSelect( ), ) _f.mem.AddAltFingerprint(_selectExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.PushSelectIntoGroupBy, _group, 0) + } return _group } } @@ -399,9 +426,12 @@ func (_f *Factory) ConstructProject( // [EliminateProject] { if _f.hasSameCols(input, projections) { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.EliminateProject) { + if _f.matchedRule == nil || _f.matchedRule(opt.EliminateProject) { _group = input _f.mem.AddAltFingerprint(_projectExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.EliminateProject, _group, 0) + } return _group } } @@ -413,12 +443,15 @@ func (_f *Factory) ConstructProject( if _projectExpr2 != nil { innerInput := _projectExpr2.Input() if _f.hasSubsetCols(input, innerInput) { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.EliminateProjectProject) { + if _f.matchedRule == nil || _f.matchedRule(opt.EliminateProjectProject) { _group = _f.ConstructProject( innerInput, projections, ) _f.mem.AddAltFingerprint(_projectExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.EliminateProjectProject, _group, 0) + } return _group } } @@ -432,7 +465,7 @@ func (_f *Factory) ConstructProject( innerInput := _projectExpr2.Input() innerProjections := _projectExpr2.Projections() if _f.hasUnusedColumns(innerProjections, _f.neededCols(projections)) { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.FilterUnusedProjectCols) { + if _f.matchedRule == nil || _f.matchedRule(opt.FilterUnusedProjectCols) { _group = _f.ConstructProject( _f.ConstructProject( innerInput, @@ -441,6 +474,9 @@ func (_f *Factory) ConstructProject( projections, ) _f.mem.AddAltFingerprint(_projectExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.FilterUnusedProjectCols, _group, 0) + } return _group } } @@ -452,12 +488,15 @@ func (_f *Factory) ConstructProject( _scanExpr := _f.mem.NormExpr(input).AsScan() if _scanExpr != nil { if _f.hasUnusedColumns(input, _f.neededCols(projections)) { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.FilterUnusedScanCols) { + if _f.matchedRule == nil || _f.matchedRule(opt.FilterUnusedScanCols) { _group = _f.ConstructProject( _f.filterUnusedColumns(input, _f.neededCols(projections)), projections, ) _f.mem.AddAltFingerprint(_projectExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.FilterUnusedScanCols, _group, 0) + } return _group } } @@ -472,7 +511,7 @@ func (_f *Factory) ConstructProject( filter := _selectExpr.Filter() if _f.hasUnusedColumns(input, _f.neededCols2(projections, filter)) { if !_f.ruleCycles[_projectExpr.Fingerprint()] { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.FilterUnusedSelectCols) { + if _f.matchedRule == nil || _f.matchedRule(opt.FilterUnusedSelectCols) { _f.ruleCycles[_projectExpr.Fingerprint()] = true _group = _f.ConstructProject( _f.ConstructSelect( @@ -485,6 +524,9 @@ func (_f *Factory) ConstructProject( if _f.mem.GroupByFingerprint(_projectExpr.Fingerprint()) == 0 { _f.mem.AddAltFingerprint(_projectExpr.Fingerprint(), _group) } + if _f.appliedRule != nil { + _f.appliedRule(opt.FilterUnusedSelectCols, _group, 0) + } return _group } } @@ -500,12 +542,15 @@ func (_f *Factory) ConstructProject( limit := _limitExpr.Limit() ordering := _limitExpr.Ordering() if _f.hasUnusedColumns(input, _f.neededColsLimit(projections, ordering)) { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.FilterUnusedLimitCols) { + if _f.matchedRule == nil || _f.matchedRule(opt.FilterUnusedLimitCols) { _group = _f.ConstructProject( _f.limitNoCycle(_f.filterUnusedColumns(input, _f.neededColsLimit(projections, ordering)), limit, ordering), projections, ) _f.mem.AddAltFingerprint(_projectExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.FilterUnusedLimitCols, _group, 0) + } return _group } } @@ -520,12 +565,15 @@ func (_f *Factory) ConstructProject( offset := _offsetExpr.Offset() ordering := _offsetExpr.Ordering() if _f.hasUnusedColumns(input, _f.neededColsLimit(projections, ordering)) { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.FilterUnusedOffsetCols) { + if _f.matchedRule == nil || _f.matchedRule(opt.FilterUnusedOffsetCols) { _group = _f.ConstructProject( _f.offsetNoCycle(_f.filterUnusedColumns(input, _f.neededColsLimit(projections, ordering)), offset, ordering), projections, ) _f.mem.AddAltFingerprint(_projectExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.FilterUnusedOffsetCols, _group, 0) + } return _group } } @@ -540,7 +588,7 @@ func (_f *Factory) ConstructProject( right := _expr.ChildGroup(_f.mem, 1) on := _expr.ChildGroup(_f.mem, 2) if _f.hasUnusedColumns(left, _f.neededCols3(projections, right, on)) { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.FilterUnusedJoinLeftCols) { + if _f.matchedRule == nil || _f.matchedRule(opt.FilterUnusedJoinLeftCols) { _group = _f.ConstructProject( _f.DynamicConstruct( _f.mem.NormExpr(input).Operator(), @@ -553,6 +601,9 @@ func (_f *Factory) ConstructProject( projections, ) _f.mem.AddAltFingerprint(_projectExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.FilterUnusedJoinLeftCols, _group, 0) + } return _group } } @@ -567,7 +618,7 @@ func (_f *Factory) ConstructProject( right := _expr.ChildGroup(_f.mem, 1) on := _expr.ChildGroup(_f.mem, 2) if _f.hasUnusedColumns(right, _f.neededCols2(projections, on)) { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.FilterUnusedJoinRightCols) { + if _f.matchedRule == nil || _f.matchedRule(opt.FilterUnusedJoinRightCols) { _group = _f.ConstructProject( _f.DynamicConstruct( _f.mem.NormExpr(input).Operator(), @@ -580,6 +631,9 @@ func (_f *Factory) ConstructProject( projections, ) _f.mem.AddAltFingerprint(_projectExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.FilterUnusedJoinRightCols, _group, 0) + } return _group } } @@ -594,7 +648,7 @@ func (_f *Factory) ConstructProject( aggregations := _groupByExpr.Aggregations() groupingCols := _groupByExpr.GroupingCols() if _f.hasUnusedColumns(aggregations, _f.neededCols(projections)) { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.FilterUnusedAggCols) { + if _f.matchedRule == nil || _f.matchedRule(opt.FilterUnusedAggCols) { _group = _f.ConstructProject( _f.ConstructGroupBy( innerInput, @@ -604,6 +658,9 @@ func (_f *Factory) ConstructProject( projections, ) _f.mem.AddAltFingerprint(_projectExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.FilterUnusedAggCols, _group, 0) + } return _group } } @@ -615,12 +672,15 @@ func (_f *Factory) ConstructProject( _valuesExpr := _f.mem.NormExpr(input).AsValues() if _valuesExpr != nil { if _f.hasUnusedColumns(input, _f.neededCols(projections)) { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.FilterUnusedValueCols) { + if _f.matchedRule == nil || _f.matchedRule(opt.FilterUnusedValueCols) { _group = _f.ConstructProject( _f.filterUnusedColumns(input, _f.neededCols(projections)), projections, ) _f.mem.AddAltFingerprint(_projectExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.FilterUnusedValueCols, _group, 0) + } return _group } } @@ -652,7 +712,7 @@ func (_f *Factory) ConstructInnerJoin( _andExpr := _f.mem.NormExpr(on).AsAnd() if _andExpr != nil { conditions := _andExpr.Conditions() - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.EnsureJoinFiltersAnd) { + if _f.matchedRule == nil || _f.matchedRule(opt.EnsureJoinFiltersAnd) { _group = _f.ConstructInnerJoin( left, right, @@ -661,6 +721,9 @@ func (_f *Factory) ConstructInnerJoin( ), ) _f.mem.AddAltFingerprint(_innerJoinExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.EnsureJoinFiltersAnd, _group, 0) + } return _group } } @@ -671,7 +734,7 @@ func (_f *Factory) ConstructInnerJoin( filter := on _expr := _f.mem.NormExpr(on) if !(_expr.Operator() == opt.FiltersOp || _expr.Operator() == opt.AndOp || _expr.Operator() == opt.TrueOp || _expr.Operator() == opt.FalseOp) { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.EnsureJoinFilters) { + if _f.matchedRule == nil || _f.matchedRule(opt.EnsureJoinFilters) { _group = _f.ConstructInnerJoin( left, right, @@ -680,6 +743,9 @@ func (_f *Factory) ConstructInnerJoin( ), ) _f.mem.AddAltFingerprint(_innerJoinExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.EnsureJoinFilters, _group, 0) + } return _group } } @@ -693,7 +759,7 @@ func (_f *Factory) ConstructInnerJoin( for _, _item := range _f.mem.LookupList(_filtersExpr.Conditions()) { condition := _item if !_f.isCorrelated(condition, right) { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.PushFilterIntoJoinLeft) { + if _f.matchedRule == nil || _f.matchedRule(opt.PushFilterIntoJoinLeft) { _group = _f.ConstructInnerJoin( _f.ConstructSelect( left, @@ -707,6 +773,9 @@ func (_f *Factory) ConstructInnerJoin( ), ) _f.mem.AddAltFingerprint(_innerJoinExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.PushFilterIntoJoinLeft, _group, 0) + } return _group } } @@ -722,7 +791,7 @@ func (_f *Factory) ConstructInnerJoin( for _, _item := range _f.mem.LookupList(_filtersExpr.Conditions()) { condition := _item if !_f.isCorrelated(condition, left) { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.PushFilterIntoJoinRight) { + if _f.matchedRule == nil || _f.matchedRule(opt.PushFilterIntoJoinRight) { _group = _f.ConstructInnerJoin( left, _f.ConstructSelect( @@ -736,6 +805,9 @@ func (_f *Factory) ConstructInnerJoin( ), ) _f.mem.AddAltFingerprint(_innerJoinExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.PushFilterIntoJoinRight, _group, 0) + } return _group } } @@ -763,7 +835,7 @@ func (_f *Factory) ConstructLeftJoin( _andExpr := _f.mem.NormExpr(on).AsAnd() if _andExpr != nil { conditions := _andExpr.Conditions() - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.EnsureJoinFiltersAnd) { + if _f.matchedRule == nil || _f.matchedRule(opt.EnsureJoinFiltersAnd) { _group = _f.ConstructLeftJoin( left, right, @@ -772,6 +844,9 @@ func (_f *Factory) ConstructLeftJoin( ), ) _f.mem.AddAltFingerprint(_leftJoinExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.EnsureJoinFiltersAnd, _group, 0) + } return _group } } @@ -782,7 +857,7 @@ func (_f *Factory) ConstructLeftJoin( filter := on _expr := _f.mem.NormExpr(on) if !(_expr.Operator() == opt.FiltersOp || _expr.Operator() == opt.AndOp || _expr.Operator() == opt.TrueOp || _expr.Operator() == opt.FalseOp) { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.EnsureJoinFilters) { + if _f.matchedRule == nil || _f.matchedRule(opt.EnsureJoinFilters) { _group = _f.ConstructLeftJoin( left, right, @@ -791,6 +866,9 @@ func (_f *Factory) ConstructLeftJoin( ), ) _f.mem.AddAltFingerprint(_leftJoinExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.EnsureJoinFilters, _group, 0) + } return _group } } @@ -804,7 +882,7 @@ func (_f *Factory) ConstructLeftJoin( for _, _item := range _f.mem.LookupList(_filtersExpr.Conditions()) { condition := _item if !_f.isCorrelated(condition, left) { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.PushFilterIntoJoinRight) { + if _f.matchedRule == nil || _f.matchedRule(opt.PushFilterIntoJoinRight) { _group = _f.ConstructLeftJoin( left, _f.ConstructSelect( @@ -818,6 +896,9 @@ func (_f *Factory) ConstructLeftJoin( ), ) _f.mem.AddAltFingerprint(_leftJoinExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.PushFilterIntoJoinRight, _group, 0) + } return _group } } @@ -845,7 +926,7 @@ func (_f *Factory) ConstructRightJoin( _andExpr := _f.mem.NormExpr(on).AsAnd() if _andExpr != nil { conditions := _andExpr.Conditions() - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.EnsureJoinFiltersAnd) { + if _f.matchedRule == nil || _f.matchedRule(opt.EnsureJoinFiltersAnd) { _group = _f.ConstructRightJoin( left, right, @@ -854,6 +935,9 @@ func (_f *Factory) ConstructRightJoin( ), ) _f.mem.AddAltFingerprint(_rightJoinExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.EnsureJoinFiltersAnd, _group, 0) + } return _group } } @@ -864,7 +948,7 @@ func (_f *Factory) ConstructRightJoin( filter := on _expr := _f.mem.NormExpr(on) if !(_expr.Operator() == opt.FiltersOp || _expr.Operator() == opt.AndOp || _expr.Operator() == opt.TrueOp || _expr.Operator() == opt.FalseOp) { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.EnsureJoinFilters) { + if _f.matchedRule == nil || _f.matchedRule(opt.EnsureJoinFilters) { _group = _f.ConstructRightJoin( left, right, @@ -873,6 +957,9 @@ func (_f *Factory) ConstructRightJoin( ), ) _f.mem.AddAltFingerprint(_rightJoinExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.EnsureJoinFilters, _group, 0) + } return _group } } @@ -886,7 +973,7 @@ func (_f *Factory) ConstructRightJoin( for _, _item := range _f.mem.LookupList(_filtersExpr.Conditions()) { condition := _item if !_f.isCorrelated(condition, right) { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.PushFilterIntoJoinLeft) { + if _f.matchedRule == nil || _f.matchedRule(opt.PushFilterIntoJoinLeft) { _group = _f.ConstructRightJoin( _f.ConstructSelect( left, @@ -900,6 +987,9 @@ func (_f *Factory) ConstructRightJoin( ), ) _f.mem.AddAltFingerprint(_rightJoinExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.PushFilterIntoJoinLeft, _group, 0) + } return _group } } @@ -927,7 +1017,7 @@ func (_f *Factory) ConstructFullJoin( _andExpr := _f.mem.NormExpr(on).AsAnd() if _andExpr != nil { conditions := _andExpr.Conditions() - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.EnsureJoinFiltersAnd) { + if _f.matchedRule == nil || _f.matchedRule(opt.EnsureJoinFiltersAnd) { _group = _f.ConstructFullJoin( left, right, @@ -936,6 +1026,9 @@ func (_f *Factory) ConstructFullJoin( ), ) _f.mem.AddAltFingerprint(_fullJoinExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.EnsureJoinFiltersAnd, _group, 0) + } return _group } } @@ -946,7 +1039,7 @@ func (_f *Factory) ConstructFullJoin( filter := on _expr := _f.mem.NormExpr(on) if !(_expr.Operator() == opt.FiltersOp || _expr.Operator() == opt.AndOp || _expr.Operator() == opt.TrueOp || _expr.Operator() == opt.FalseOp) { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.EnsureJoinFilters) { + if _f.matchedRule == nil || _f.matchedRule(opt.EnsureJoinFilters) { _group = _f.ConstructFullJoin( left, right, @@ -955,6 +1048,9 @@ func (_f *Factory) ConstructFullJoin( ), ) _f.mem.AddAltFingerprint(_fullJoinExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.EnsureJoinFilters, _group, 0) + } return _group } } @@ -980,7 +1076,7 @@ func (_f *Factory) ConstructSemiJoin( _andExpr := _f.mem.NormExpr(on).AsAnd() if _andExpr != nil { conditions := _andExpr.Conditions() - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.EnsureJoinFiltersAnd) { + if _f.matchedRule == nil || _f.matchedRule(opt.EnsureJoinFiltersAnd) { _group = _f.ConstructSemiJoin( left, right, @@ -989,6 +1085,9 @@ func (_f *Factory) ConstructSemiJoin( ), ) _f.mem.AddAltFingerprint(_semiJoinExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.EnsureJoinFiltersAnd, _group, 0) + } return _group } } @@ -999,7 +1098,7 @@ func (_f *Factory) ConstructSemiJoin( filter := on _expr := _f.mem.NormExpr(on) if !(_expr.Operator() == opt.FiltersOp || _expr.Operator() == opt.AndOp || _expr.Operator() == opt.TrueOp || _expr.Operator() == opt.FalseOp) { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.EnsureJoinFilters) { + if _f.matchedRule == nil || _f.matchedRule(opt.EnsureJoinFilters) { _group = _f.ConstructSemiJoin( left, right, @@ -1008,6 +1107,9 @@ func (_f *Factory) ConstructSemiJoin( ), ) _f.mem.AddAltFingerprint(_semiJoinExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.EnsureJoinFilters, _group, 0) + } return _group } } @@ -1033,7 +1135,7 @@ func (_f *Factory) ConstructAntiJoin( _andExpr := _f.mem.NormExpr(on).AsAnd() if _andExpr != nil { conditions := _andExpr.Conditions() - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.EnsureJoinFiltersAnd) { + if _f.matchedRule == nil || _f.matchedRule(opt.EnsureJoinFiltersAnd) { _group = _f.ConstructAntiJoin( left, right, @@ -1042,6 +1144,9 @@ func (_f *Factory) ConstructAntiJoin( ), ) _f.mem.AddAltFingerprint(_antiJoinExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.EnsureJoinFiltersAnd, _group, 0) + } return _group } } @@ -1052,7 +1157,7 @@ func (_f *Factory) ConstructAntiJoin( filter := on _expr := _f.mem.NormExpr(on) if !(_expr.Operator() == opt.FiltersOp || _expr.Operator() == opt.AndOp || _expr.Operator() == opt.TrueOp || _expr.Operator() == opt.FalseOp) { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.EnsureJoinFilters) { + if _f.matchedRule == nil || _f.matchedRule(opt.EnsureJoinFilters) { _group = _f.ConstructAntiJoin( left, right, @@ -1061,6 +1166,9 @@ func (_f *Factory) ConstructAntiJoin( ), ) _f.mem.AddAltFingerprint(_antiJoinExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.EnsureJoinFilters, _group, 0) + } return _group } } @@ -1089,7 +1197,7 @@ func (_f *Factory) ConstructInnerJoinApply( _andExpr := _f.mem.NormExpr(on).AsAnd() if _andExpr != nil { conditions := _andExpr.Conditions() - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.EnsureJoinFiltersAnd) { + if _f.matchedRule == nil || _f.matchedRule(opt.EnsureJoinFiltersAnd) { _group = _f.ConstructInnerJoinApply( left, right, @@ -1098,6 +1206,9 @@ func (_f *Factory) ConstructInnerJoinApply( ), ) _f.mem.AddAltFingerprint(_innerJoinApplyExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.EnsureJoinFiltersAnd, _group, 0) + } return _group } } @@ -1108,7 +1219,7 @@ func (_f *Factory) ConstructInnerJoinApply( filter := on _expr := _f.mem.NormExpr(on) if !(_expr.Operator() == opt.FiltersOp || _expr.Operator() == opt.AndOp || _expr.Operator() == opt.TrueOp || _expr.Operator() == opt.FalseOp) { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.EnsureJoinFilters) { + if _f.matchedRule == nil || _f.matchedRule(opt.EnsureJoinFilters) { _group = _f.ConstructInnerJoinApply( left, right, @@ -1117,6 +1228,9 @@ func (_f *Factory) ConstructInnerJoinApply( ), ) _f.mem.AddAltFingerprint(_innerJoinApplyExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.EnsureJoinFilters, _group, 0) + } return _group } } @@ -1130,7 +1244,7 @@ func (_f *Factory) ConstructInnerJoinApply( for _, _item := range _f.mem.LookupList(_filtersExpr.Conditions()) { condition := _item if !_f.isCorrelated(condition, right) { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.PushFilterIntoJoinLeft) { + if _f.matchedRule == nil || _f.matchedRule(opt.PushFilterIntoJoinLeft) { _group = _f.ConstructInnerJoinApply( _f.ConstructSelect( left, @@ -1144,6 +1258,9 @@ func (_f *Factory) ConstructInnerJoinApply( ), ) _f.mem.AddAltFingerprint(_innerJoinApplyExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.PushFilterIntoJoinLeft, _group, 0) + } return _group } } @@ -1159,7 +1276,7 @@ func (_f *Factory) ConstructInnerJoinApply( for _, _item := range _f.mem.LookupList(_filtersExpr.Conditions()) { condition := _item if !_f.isCorrelated(condition, left) { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.PushFilterIntoJoinRight) { + if _f.matchedRule == nil || _f.matchedRule(opt.PushFilterIntoJoinRight) { _group = _f.ConstructInnerJoinApply( left, _f.ConstructSelect( @@ -1173,6 +1290,9 @@ func (_f *Factory) ConstructInnerJoinApply( ), ) _f.mem.AddAltFingerprint(_innerJoinApplyExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.PushFilterIntoJoinRight, _group, 0) + } return _group } } @@ -1200,7 +1320,7 @@ func (_f *Factory) ConstructLeftJoinApply( _andExpr := _f.mem.NormExpr(on).AsAnd() if _andExpr != nil { conditions := _andExpr.Conditions() - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.EnsureJoinFiltersAnd) { + if _f.matchedRule == nil || _f.matchedRule(opt.EnsureJoinFiltersAnd) { _group = _f.ConstructLeftJoinApply( left, right, @@ -1209,6 +1329,9 @@ func (_f *Factory) ConstructLeftJoinApply( ), ) _f.mem.AddAltFingerprint(_leftJoinApplyExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.EnsureJoinFiltersAnd, _group, 0) + } return _group } } @@ -1219,7 +1342,7 @@ func (_f *Factory) ConstructLeftJoinApply( filter := on _expr := _f.mem.NormExpr(on) if !(_expr.Operator() == opt.FiltersOp || _expr.Operator() == opt.AndOp || _expr.Operator() == opt.TrueOp || _expr.Operator() == opt.FalseOp) { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.EnsureJoinFilters) { + if _f.matchedRule == nil || _f.matchedRule(opt.EnsureJoinFilters) { _group = _f.ConstructLeftJoinApply( left, right, @@ -1228,6 +1351,9 @@ func (_f *Factory) ConstructLeftJoinApply( ), ) _f.mem.AddAltFingerprint(_leftJoinApplyExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.EnsureJoinFilters, _group, 0) + } return _group } } @@ -1241,7 +1367,7 @@ func (_f *Factory) ConstructLeftJoinApply( for _, _item := range _f.mem.LookupList(_filtersExpr.Conditions()) { condition := _item if !_f.isCorrelated(condition, left) { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.PushFilterIntoJoinRight) { + if _f.matchedRule == nil || _f.matchedRule(opt.PushFilterIntoJoinRight) { _group = _f.ConstructLeftJoinApply( left, _f.ConstructSelect( @@ -1255,6 +1381,9 @@ func (_f *Factory) ConstructLeftJoinApply( ), ) _f.mem.AddAltFingerprint(_leftJoinApplyExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.PushFilterIntoJoinRight, _group, 0) + } return _group } } @@ -1282,7 +1411,7 @@ func (_f *Factory) ConstructRightJoinApply( _andExpr := _f.mem.NormExpr(on).AsAnd() if _andExpr != nil { conditions := _andExpr.Conditions() - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.EnsureJoinFiltersAnd) { + if _f.matchedRule == nil || _f.matchedRule(opt.EnsureJoinFiltersAnd) { _group = _f.ConstructRightJoinApply( left, right, @@ -1291,6 +1420,9 @@ func (_f *Factory) ConstructRightJoinApply( ), ) _f.mem.AddAltFingerprint(_rightJoinApplyExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.EnsureJoinFiltersAnd, _group, 0) + } return _group } } @@ -1301,7 +1433,7 @@ func (_f *Factory) ConstructRightJoinApply( filter := on _expr := _f.mem.NormExpr(on) if !(_expr.Operator() == opt.FiltersOp || _expr.Operator() == opt.AndOp || _expr.Operator() == opt.TrueOp || _expr.Operator() == opt.FalseOp) { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.EnsureJoinFilters) { + if _f.matchedRule == nil || _f.matchedRule(opt.EnsureJoinFilters) { _group = _f.ConstructRightJoinApply( left, right, @@ -1310,6 +1442,9 @@ func (_f *Factory) ConstructRightJoinApply( ), ) _f.mem.AddAltFingerprint(_rightJoinApplyExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.EnsureJoinFilters, _group, 0) + } return _group } } @@ -1323,7 +1458,7 @@ func (_f *Factory) ConstructRightJoinApply( for _, _item := range _f.mem.LookupList(_filtersExpr.Conditions()) { condition := _item if !_f.isCorrelated(condition, right) { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.PushFilterIntoJoinLeft) { + if _f.matchedRule == nil || _f.matchedRule(opt.PushFilterIntoJoinLeft) { _group = _f.ConstructRightJoinApply( _f.ConstructSelect( left, @@ -1337,6 +1472,9 @@ func (_f *Factory) ConstructRightJoinApply( ), ) _f.mem.AddAltFingerprint(_rightJoinApplyExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.PushFilterIntoJoinLeft, _group, 0) + } return _group } } @@ -1364,7 +1502,7 @@ func (_f *Factory) ConstructFullJoinApply( _andExpr := _f.mem.NormExpr(on).AsAnd() if _andExpr != nil { conditions := _andExpr.Conditions() - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.EnsureJoinFiltersAnd) { + if _f.matchedRule == nil || _f.matchedRule(opt.EnsureJoinFiltersAnd) { _group = _f.ConstructFullJoinApply( left, right, @@ -1373,6 +1511,9 @@ func (_f *Factory) ConstructFullJoinApply( ), ) _f.mem.AddAltFingerprint(_fullJoinApplyExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.EnsureJoinFiltersAnd, _group, 0) + } return _group } } @@ -1383,7 +1524,7 @@ func (_f *Factory) ConstructFullJoinApply( filter := on _expr := _f.mem.NormExpr(on) if !(_expr.Operator() == opt.FiltersOp || _expr.Operator() == opt.AndOp || _expr.Operator() == opt.TrueOp || _expr.Operator() == opt.FalseOp) { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.EnsureJoinFilters) { + if _f.matchedRule == nil || _f.matchedRule(opt.EnsureJoinFilters) { _group = _f.ConstructFullJoinApply( left, right, @@ -1392,6 +1533,9 @@ func (_f *Factory) ConstructFullJoinApply( ), ) _f.mem.AddAltFingerprint(_fullJoinApplyExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.EnsureJoinFilters, _group, 0) + } return _group } } @@ -1417,7 +1561,7 @@ func (_f *Factory) ConstructSemiJoinApply( _andExpr := _f.mem.NormExpr(on).AsAnd() if _andExpr != nil { conditions := _andExpr.Conditions() - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.EnsureJoinFiltersAnd) { + if _f.matchedRule == nil || _f.matchedRule(opt.EnsureJoinFiltersAnd) { _group = _f.ConstructSemiJoinApply( left, right, @@ -1426,6 +1570,9 @@ func (_f *Factory) ConstructSemiJoinApply( ), ) _f.mem.AddAltFingerprint(_semiJoinApplyExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.EnsureJoinFiltersAnd, _group, 0) + } return _group } } @@ -1436,7 +1583,7 @@ func (_f *Factory) ConstructSemiJoinApply( filter := on _expr := _f.mem.NormExpr(on) if !(_expr.Operator() == opt.FiltersOp || _expr.Operator() == opt.AndOp || _expr.Operator() == opt.TrueOp || _expr.Operator() == opt.FalseOp) { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.EnsureJoinFilters) { + if _f.matchedRule == nil || _f.matchedRule(opt.EnsureJoinFilters) { _group = _f.ConstructSemiJoinApply( left, right, @@ -1445,6 +1592,9 @@ func (_f *Factory) ConstructSemiJoinApply( ), ) _f.mem.AddAltFingerprint(_semiJoinApplyExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.EnsureJoinFilters, _group, 0) + } return _group } } @@ -1470,7 +1620,7 @@ func (_f *Factory) ConstructAntiJoinApply( _andExpr := _f.mem.NormExpr(on).AsAnd() if _andExpr != nil { conditions := _andExpr.Conditions() - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.EnsureJoinFiltersAnd) { + if _f.matchedRule == nil || _f.matchedRule(opt.EnsureJoinFiltersAnd) { _group = _f.ConstructAntiJoinApply( left, right, @@ -1479,6 +1629,9 @@ func (_f *Factory) ConstructAntiJoinApply( ), ) _f.mem.AddAltFingerprint(_antiJoinApplyExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.EnsureJoinFiltersAnd, _group, 0) + } return _group } } @@ -1489,7 +1642,7 @@ func (_f *Factory) ConstructAntiJoinApply( filter := on _expr := _f.mem.NormExpr(on) if !(_expr.Operator() == opt.FiltersOp || _expr.Operator() == opt.AndOp || _expr.Operator() == opt.TrueOp || _expr.Operator() == opt.FalseOp) { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.EnsureJoinFilters) { + if _f.matchedRule == nil || _f.matchedRule(opt.EnsureJoinFilters) { _group = _f.ConstructAntiJoinApply( left, right, @@ -1498,6 +1651,9 @@ func (_f *Factory) ConstructAntiJoinApply( ), ) _f.mem.AddAltFingerprint(_antiJoinApplyExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.EnsureJoinFilters, _group, 0) + } return _group } } @@ -1527,9 +1683,12 @@ func (_f *Factory) ConstructGroupBy( { if _f.hasNoCols(aggregations) { if _f.colsAreKey(groupingCols, input) { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.EliminateDistinct) { + if _f.matchedRule == nil || _f.matchedRule(opt.EliminateDistinct) { _group = input _f.mem.AddAltFingerprint(_groupByExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.EliminateDistinct, _group, 0) + } return _group } } @@ -1539,13 +1698,16 @@ func (_f *Factory) ConstructGroupBy( // [FilterUnusedGroupByCols] { if _f.hasUnusedColumns(input, _f.neededColsGroupBy(aggregations, groupingCols)) { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.FilterUnusedGroupByCols) { + if _f.matchedRule == nil || _f.matchedRule(opt.FilterUnusedGroupByCols) { _group = _f.ConstructGroupBy( _f.filterUnusedColumns(input, _f.neededColsGroupBy(aggregations, groupingCols)), aggregations, groupingCols, ) _f.mem.AddAltFingerprint(_groupByExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.FilterUnusedGroupByCols, _group, 0) + } return _group } } @@ -1737,7 +1899,7 @@ func (_f *Factory) ConstructLimit( input := _projectExpr.Input() projections := _projectExpr.Projections() if !_f.ruleCycles[_limitExpr.Fingerprint()] { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.PushLimitIntoProject) { + if _f.matchedRule == nil || _f.matchedRule(opt.PushLimitIntoProject) { _f.ruleCycles[_limitExpr.Fingerprint()] = true _group = _f.ConstructProject( _f.ConstructLimit( @@ -1751,6 +1913,9 @@ func (_f *Factory) ConstructLimit( if _f.mem.GroupByFingerprint(_limitExpr.Fingerprint()) == 0 { _f.mem.AddAltFingerprint(_limitExpr.Fingerprint(), _group) } + if _f.appliedRule != nil { + _f.appliedRule(opt.PushLimitIntoProject, _group, 0) + } return _group } } @@ -1781,7 +1946,7 @@ func (_f *Factory) ConstructOffset( input := _projectExpr.Input() projections := _projectExpr.Projections() if !_f.ruleCycles[_offsetExpr.Fingerprint()] { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.PushOffsetIntoProject) { + if _f.matchedRule == nil || _f.matchedRule(opt.PushOffsetIntoProject) { _f.ruleCycles[_offsetExpr.Fingerprint()] = true _group = _f.ConstructProject( _f.ConstructOffset( @@ -1795,6 +1960,9 @@ func (_f *Factory) ConstructOffset( if _f.mem.GroupByFingerprint(_offsetExpr.Fingerprint()) == 0 { _f.mem.AddAltFingerprint(_offsetExpr.Fingerprint(), _group) } + if _f.appliedRule != nil { + _f.appliedRule(opt.PushOffsetIntoProject, _group, 0) + } return _group } } @@ -2084,9 +2252,12 @@ func (_f *Factory) ConstructFilters( // [EliminateEmptyAnd] { if conditions.Length == 0 { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.EliminateEmptyAnd) { + if _f.matchedRule == nil || _f.matchedRule(opt.EliminateEmptyAnd) { _group = _f.ConstructTrue() _f.mem.AddAltFingerprint(_filtersExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.EliminateEmptyAnd, _group, 0) + } return _group } } @@ -2097,9 +2268,12 @@ func (_f *Factory) ConstructFilters( for _, _item := range _f.mem.LookupList(conditions) { _expr := _f.mem.NormExpr(_item) if _expr.Operator() == opt.AndOp || _expr.Operator() == opt.TrueOp || _expr.Operator() == opt.FalseOp || _expr.Operator() == opt.NullOp { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.SimplifyFilters) { + if _f.matchedRule == nil || _f.matchedRule(opt.SimplifyFilters) { _group = _f.simplifyFilters(conditions) _f.mem.AddAltFingerprint(_filtersExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.SimplifyFilters, _group, 0) + } return _group } } @@ -2125,9 +2299,12 @@ func (_f *Factory) ConstructAnd( // [EliminateEmptyAnd] { if conditions.Length == 0 { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.EliminateEmptyAnd) { + if _f.matchedRule == nil || _f.matchedRule(opt.EliminateEmptyAnd) { _group = _f.ConstructTrue() _f.mem.AddAltFingerprint(_andExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.EliminateEmptyAnd, _group, 0) + } return _group } } @@ -2138,9 +2315,12 @@ func (_f *Factory) ConstructAnd( if conditions.Length == 1 { _item := _f.mem.LookupList(conditions)[0] item := _item - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.EliminateSingletonAndOr) { + if _f.matchedRule == nil || _f.matchedRule(opt.EliminateSingletonAndOr) { _group = item _f.mem.AddAltFingerprint(_andExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.EliminateSingletonAndOr, _group, 0) + } return _group } } @@ -2151,9 +2331,12 @@ func (_f *Factory) ConstructAnd( for _, _item := range _f.mem.LookupList(conditions) { _expr := _f.mem.NormExpr(_item) if _expr.Operator() == opt.AndOp || _expr.Operator() == opt.TrueOp || _expr.Operator() == opt.FalseOp { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.SimplifyAnd) { + if _f.matchedRule == nil || _f.matchedRule(opt.SimplifyAnd) { _group = _f.simplifyAnd(conditions) _f.mem.AddAltFingerprint(_andExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.SimplifyAnd, _group, 0) + } return _group } } @@ -2167,11 +2350,14 @@ func (_f *Factory) ConstructAnd( _nullExpr := _f.mem.NormExpr(_item).AsNull() if _nullExpr != nil { if _f.listOnlyHasNulls(conditions) { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.FoldNullAndOr) { + if _f.matchedRule == nil || _f.matchedRule(opt.FoldNullAndOr) { _group = _f.ConstructNull( _f.boolType(), ) _f.mem.AddAltFingerprint(_andExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.FoldNullAndOr, _group, 0) + } return _group } } @@ -2198,9 +2384,12 @@ func (_f *Factory) ConstructOr( // [EliminateEmptyOr] { if conditions.Length == 0 { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.EliminateEmptyOr) { + if _f.matchedRule == nil || _f.matchedRule(opt.EliminateEmptyOr) { _group = _f.ConstructFalse() _f.mem.AddAltFingerprint(_orExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.EliminateEmptyOr, _group, 0) + } return _group } } @@ -2211,9 +2400,12 @@ func (_f *Factory) ConstructOr( if conditions.Length == 1 { _item := _f.mem.LookupList(conditions)[0] item := _item - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.EliminateSingletonAndOr) { + if _f.matchedRule == nil || _f.matchedRule(opt.EliminateSingletonAndOr) { _group = item _f.mem.AddAltFingerprint(_orExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.EliminateSingletonAndOr, _group, 0) + } return _group } } @@ -2224,9 +2416,12 @@ func (_f *Factory) ConstructOr( for _, _item := range _f.mem.LookupList(conditions) { _expr := _f.mem.NormExpr(_item) if _expr.Operator() == opt.OrOp || _expr.Operator() == opt.TrueOp || _expr.Operator() == opt.FalseOp { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.SimplifyOr) { + if _f.matchedRule == nil || _f.matchedRule(opt.SimplifyOr) { _group = _f.simplifyOr(conditions) _f.mem.AddAltFingerprint(_orExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.SimplifyOr, _group, 0) + } return _group } } @@ -2240,11 +2435,14 @@ func (_f *Factory) ConstructOr( _nullExpr := _f.mem.NormExpr(_item).AsNull() if _nullExpr != nil { if _f.listOnlyHasNulls(conditions) { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.FoldNullAndOr) { + if _f.matchedRule == nil || _f.matchedRule(opt.FoldNullAndOr) { _group = _f.ConstructNull( _f.boolType(), ) _f.mem.AddAltFingerprint(_orExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.FoldNullAndOr, _group, 0) + } return _group } } @@ -2275,9 +2473,12 @@ func (_f *Factory) ConstructNot( right := _expr.ChildGroup(_f.mem, 1) _expr2 := _f.mem.NormExpr(input) if !(_expr2.Operator() == opt.ContainsOp || _expr2.Operator() == opt.JsonExistsOp || _expr2.Operator() == opt.JsonSomeExistsOp || _expr2.Operator() == opt.JsonAllExistsOp) { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.NegateComparison) { + if _f.matchedRule == nil || _f.matchedRule(opt.NegateComparison) { _group = _f.negateComparison(_f.mem.NormExpr(input).Operator(), left, right) _f.mem.AddAltFingerprint(_notExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.NegateComparison, _group, 0) + } return _group } } @@ -2289,9 +2490,12 @@ func (_f *Factory) ConstructNot( _notExpr2 := _f.mem.NormExpr(input).AsNot() if _notExpr2 != nil { input := _notExpr2.Input() - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.EliminateNot) { + if _f.matchedRule == nil || _f.matchedRule(opt.EliminateNot) { _group = input _f.mem.AddAltFingerprint(_notExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.EliminateNot, _group, 0) + } return _group } } @@ -2302,11 +2506,14 @@ func (_f *Factory) ConstructNot( _andExpr := _f.mem.NormExpr(input).AsAnd() if _andExpr != nil { conditions := _andExpr.Conditions() - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.NegateAnd) { + if _f.matchedRule == nil || _f.matchedRule(opt.NegateAnd) { _group = _f.ConstructOr( _f.negateConditions(conditions), ) _f.mem.AddAltFingerprint(_notExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.NegateAnd, _group, 0) + } return _group } } @@ -2317,11 +2524,14 @@ func (_f *Factory) ConstructNot( _orExpr := _f.mem.NormExpr(input).AsOr() if _orExpr != nil { conditions := _orExpr.Conditions() - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.NegateOr) { + if _f.matchedRule == nil || _f.matchedRule(opt.NegateOr) { _group = _f.ConstructAnd( _f.negateConditions(conditions), ) _f.mem.AddAltFingerprint(_notExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.NegateOr, _group, 0) + } return _group } } @@ -2351,7 +2561,7 @@ func (_f *Factory) ConstructEq( if _f.onlyConstants(leftRight) { if _f.onlyConstants(right) { if _f.canConstructBinary(opt.MinusOp, right, leftRight) { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.NormalizeCmpPlusConst) { + if _f.matchedRule == nil || _f.matchedRule(opt.NormalizeCmpPlusConst) { _group = _f.ConstructEq( leftLeft, _f.ConstructMinus( @@ -2360,6 +2570,9 @@ func (_f *Factory) ConstructEq( ), ) _f.mem.AddAltFingerprint(_eqExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.NormalizeCmpPlusConst, _group, 0) + } return _group } } @@ -2379,7 +2592,7 @@ func (_f *Factory) ConstructEq( if _f.onlyConstants(leftRight) { if _f.onlyConstants(right) { if _f.canConstructBinary(opt.PlusOp, right, leftRight) { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.NormalizeCmpMinusConst) { + if _f.matchedRule == nil || _f.matchedRule(opt.NormalizeCmpMinusConst) { _group = _f.ConstructEq( leftLeft, _f.ConstructPlus( @@ -2388,6 +2601,9 @@ func (_f *Factory) ConstructEq( ), ) _f.mem.AddAltFingerprint(_eqExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.NormalizeCmpMinusConst, _group, 0) + } return _group } } @@ -2407,7 +2623,7 @@ func (_f *Factory) ConstructEq( if !_f.onlyConstants(leftRight) { if _f.onlyConstants(right) { if _f.canConstructBinary(opt.MinusOp, leftLeft, right) { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.NormalizeCmpConstMinus) { + if _f.matchedRule == nil || _f.matchedRule(opt.NormalizeCmpConstMinus) { _group = _f.ConstructEq( _f.ConstructMinus( leftLeft, @@ -2416,6 +2632,9 @@ func (_f *Factory) ConstructEq( leftRight, ) _f.mem.AddAltFingerprint(_eqExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.NormalizeCmpConstMinus, _group, 0) + } return _group } } @@ -2433,9 +2652,12 @@ func (_f *Factory) ConstructEq( _tupleExpr2 := _f.mem.NormExpr(right).AsTuple() if _tupleExpr2 != nil { right := _tupleExpr2.Elems() - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.NormalizeTupleEquality) { + if _f.matchedRule == nil || _f.matchedRule(opt.NormalizeTupleEquality) { _group = _f.normalizeTupleEquality(left, right) _f.mem.AddAltFingerprint(_eqExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.NormalizeTupleEquality, _group, 0) + } return _group } } @@ -2446,11 +2668,14 @@ func (_f *Factory) ConstructEq( { _nullExpr := _f.mem.NormExpr(left).AsNull() if _nullExpr != nil { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.FoldNullComparisonLeft) { + if _f.matchedRule == nil || _f.matchedRule(opt.FoldNullComparisonLeft) { _group = _f.ConstructNull( _f.boolType(), ) _f.mem.AddAltFingerprint(_eqExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.FoldNullComparisonLeft, _group, 0) + } return _group } } @@ -2460,11 +2685,14 @@ func (_f *Factory) ConstructEq( { _nullExpr := _f.mem.NormExpr(right).AsNull() if _nullExpr != nil { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.FoldNullComparisonRight) { + if _f.matchedRule == nil || _f.matchedRule(opt.FoldNullComparisonRight) { _group = _f.ConstructNull( _f.boolType(), ) _f.mem.AddAltFingerprint(_eqExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.FoldNullComparisonRight, _group, 0) + } return _group } } @@ -2476,12 +2704,15 @@ func (_f *Factory) ConstructEq( if _variableExpr == nil { _variableExpr2 := _f.mem.NormExpr(right).AsVariable() if _variableExpr2 != nil { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.CommuteVar) { + if _f.matchedRule == nil || _f.matchedRule(opt.CommuteVar) { _group = _f.ConstructEq( right, left, ) _f.mem.AddAltFingerprint(_eqExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.CommuteVar, _group, 0) + } return _group } } @@ -2492,12 +2723,15 @@ func (_f *Factory) ConstructEq( { if _f.onlyConstants(left) { if !_f.onlyConstants(right) { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.CommuteConst) { + if _f.matchedRule == nil || _f.matchedRule(opt.CommuteConst) { _group = _f.ConstructEq( right, left, ) _f.mem.AddAltFingerprint(_eqExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.CommuteConst, _group, 0) + } return _group } } @@ -2524,9 +2758,12 @@ func (_f *Factory) ConstructLt( if _variableExpr == nil { _variableExpr2 := _f.mem.NormExpr(right).AsVariable() if _variableExpr2 != nil { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.CommuteVarInequality) { + if _f.matchedRule == nil || _f.matchedRule(opt.CommuteVarInequality) { _group = _f.commuteInequality(opt.LtOp, left, right) _f.mem.AddAltFingerprint(_ltExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.CommuteVarInequality, _group, 0) + } return _group } } @@ -2537,9 +2774,12 @@ func (_f *Factory) ConstructLt( { if _f.onlyConstants(left) { if !_f.onlyConstants(right) { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.CommuteConstInequality) { + if _f.matchedRule == nil || _f.matchedRule(opt.CommuteConstInequality) { _group = _f.commuteInequality(opt.LtOp, left, right) _f.mem.AddAltFingerprint(_ltExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.CommuteConstInequality, _group, 0) + } return _group } } @@ -2556,7 +2796,7 @@ func (_f *Factory) ConstructLt( if _f.onlyConstants(leftRight) { if _f.onlyConstants(right) { if _f.canConstructBinary(opt.MinusOp, right, leftRight) { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.NormalizeCmpPlusConst) { + if _f.matchedRule == nil || _f.matchedRule(opt.NormalizeCmpPlusConst) { _group = _f.ConstructLt( leftLeft, _f.ConstructMinus( @@ -2565,6 +2805,9 @@ func (_f *Factory) ConstructLt( ), ) _f.mem.AddAltFingerprint(_ltExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.NormalizeCmpPlusConst, _group, 0) + } return _group } } @@ -2584,7 +2827,7 @@ func (_f *Factory) ConstructLt( if _f.onlyConstants(leftRight) { if _f.onlyConstants(right) { if _f.canConstructBinary(opt.PlusOp, right, leftRight) { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.NormalizeCmpMinusConst) { + if _f.matchedRule == nil || _f.matchedRule(opt.NormalizeCmpMinusConst) { _group = _f.ConstructLt( leftLeft, _f.ConstructPlus( @@ -2593,6 +2836,9 @@ func (_f *Factory) ConstructLt( ), ) _f.mem.AddAltFingerprint(_ltExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.NormalizeCmpMinusConst, _group, 0) + } return _group } } @@ -2612,7 +2858,7 @@ func (_f *Factory) ConstructLt( if !_f.onlyConstants(leftRight) { if _f.onlyConstants(right) { if _f.canConstructBinary(opt.MinusOp, leftLeft, right) { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.NormalizeCmpConstMinus) { + if _f.matchedRule == nil || _f.matchedRule(opt.NormalizeCmpConstMinus) { _group = _f.ConstructLt( _f.ConstructMinus( leftLeft, @@ -2621,6 +2867,9 @@ func (_f *Factory) ConstructLt( leftRight, ) _f.mem.AddAltFingerprint(_ltExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.NormalizeCmpConstMinus, _group, 0) + } return _group } } @@ -2634,11 +2883,14 @@ func (_f *Factory) ConstructLt( { _nullExpr := _f.mem.NormExpr(left).AsNull() if _nullExpr != nil { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.FoldNullComparisonLeft) { + if _f.matchedRule == nil || _f.matchedRule(opt.FoldNullComparisonLeft) { _group = _f.ConstructNull( _f.boolType(), ) _f.mem.AddAltFingerprint(_ltExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.FoldNullComparisonLeft, _group, 0) + } return _group } } @@ -2648,11 +2900,14 @@ func (_f *Factory) ConstructLt( { _nullExpr := _f.mem.NormExpr(right).AsNull() if _nullExpr != nil { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.FoldNullComparisonRight) { + if _f.matchedRule == nil || _f.matchedRule(opt.FoldNullComparisonRight) { _group = _f.ConstructNull( _f.boolType(), ) _f.mem.AddAltFingerprint(_ltExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.FoldNullComparisonRight, _group, 0) + } return _group } } @@ -2678,9 +2933,12 @@ func (_f *Factory) ConstructGt( if _variableExpr == nil { _variableExpr2 := _f.mem.NormExpr(right).AsVariable() if _variableExpr2 != nil { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.CommuteVarInequality) { + if _f.matchedRule == nil || _f.matchedRule(opt.CommuteVarInequality) { _group = _f.commuteInequality(opt.GtOp, left, right) _f.mem.AddAltFingerprint(_gtExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.CommuteVarInequality, _group, 0) + } return _group } } @@ -2691,9 +2949,12 @@ func (_f *Factory) ConstructGt( { if _f.onlyConstants(left) { if !_f.onlyConstants(right) { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.CommuteConstInequality) { + if _f.matchedRule == nil || _f.matchedRule(opt.CommuteConstInequality) { _group = _f.commuteInequality(opt.GtOp, left, right) _f.mem.AddAltFingerprint(_gtExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.CommuteConstInequality, _group, 0) + } return _group } } @@ -2710,7 +2971,7 @@ func (_f *Factory) ConstructGt( if _f.onlyConstants(leftRight) { if _f.onlyConstants(right) { if _f.canConstructBinary(opt.MinusOp, right, leftRight) { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.NormalizeCmpPlusConst) { + if _f.matchedRule == nil || _f.matchedRule(opt.NormalizeCmpPlusConst) { _group = _f.ConstructGt( leftLeft, _f.ConstructMinus( @@ -2719,6 +2980,9 @@ func (_f *Factory) ConstructGt( ), ) _f.mem.AddAltFingerprint(_gtExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.NormalizeCmpPlusConst, _group, 0) + } return _group } } @@ -2738,7 +3002,7 @@ func (_f *Factory) ConstructGt( if _f.onlyConstants(leftRight) { if _f.onlyConstants(right) { if _f.canConstructBinary(opt.PlusOp, right, leftRight) { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.NormalizeCmpMinusConst) { + if _f.matchedRule == nil || _f.matchedRule(opt.NormalizeCmpMinusConst) { _group = _f.ConstructGt( leftLeft, _f.ConstructPlus( @@ -2747,6 +3011,9 @@ func (_f *Factory) ConstructGt( ), ) _f.mem.AddAltFingerprint(_gtExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.NormalizeCmpMinusConst, _group, 0) + } return _group } } @@ -2766,7 +3033,7 @@ func (_f *Factory) ConstructGt( if !_f.onlyConstants(leftRight) { if _f.onlyConstants(right) { if _f.canConstructBinary(opt.MinusOp, leftLeft, right) { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.NormalizeCmpConstMinus) { + if _f.matchedRule == nil || _f.matchedRule(opt.NormalizeCmpConstMinus) { _group = _f.ConstructGt( _f.ConstructMinus( leftLeft, @@ -2775,6 +3042,9 @@ func (_f *Factory) ConstructGt( leftRight, ) _f.mem.AddAltFingerprint(_gtExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.NormalizeCmpConstMinus, _group, 0) + } return _group } } @@ -2788,11 +3058,14 @@ func (_f *Factory) ConstructGt( { _nullExpr := _f.mem.NormExpr(left).AsNull() if _nullExpr != nil { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.FoldNullComparisonLeft) { + if _f.matchedRule == nil || _f.matchedRule(opt.FoldNullComparisonLeft) { _group = _f.ConstructNull( _f.boolType(), ) _f.mem.AddAltFingerprint(_gtExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.FoldNullComparisonLeft, _group, 0) + } return _group } } @@ -2802,11 +3075,14 @@ func (_f *Factory) ConstructGt( { _nullExpr := _f.mem.NormExpr(right).AsNull() if _nullExpr != nil { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.FoldNullComparisonRight) { + if _f.matchedRule == nil || _f.matchedRule(opt.FoldNullComparisonRight) { _group = _f.ConstructNull( _f.boolType(), ) _f.mem.AddAltFingerprint(_gtExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.FoldNullComparisonRight, _group, 0) + } return _group } } @@ -2832,9 +3108,12 @@ func (_f *Factory) ConstructLe( if _variableExpr == nil { _variableExpr2 := _f.mem.NormExpr(right).AsVariable() if _variableExpr2 != nil { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.CommuteVarInequality) { + if _f.matchedRule == nil || _f.matchedRule(opt.CommuteVarInequality) { _group = _f.commuteInequality(opt.LeOp, left, right) _f.mem.AddAltFingerprint(_leExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.CommuteVarInequality, _group, 0) + } return _group } } @@ -2845,9 +3124,12 @@ func (_f *Factory) ConstructLe( { if _f.onlyConstants(left) { if !_f.onlyConstants(right) { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.CommuteConstInequality) { + if _f.matchedRule == nil || _f.matchedRule(opt.CommuteConstInequality) { _group = _f.commuteInequality(opt.LeOp, left, right) _f.mem.AddAltFingerprint(_leExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.CommuteConstInequality, _group, 0) + } return _group } } @@ -2864,7 +3146,7 @@ func (_f *Factory) ConstructLe( if _f.onlyConstants(leftRight) { if _f.onlyConstants(right) { if _f.canConstructBinary(opt.MinusOp, right, leftRight) { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.NormalizeCmpPlusConst) { + if _f.matchedRule == nil || _f.matchedRule(opt.NormalizeCmpPlusConst) { _group = _f.ConstructLe( leftLeft, _f.ConstructMinus( @@ -2873,6 +3155,9 @@ func (_f *Factory) ConstructLe( ), ) _f.mem.AddAltFingerprint(_leExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.NormalizeCmpPlusConst, _group, 0) + } return _group } } @@ -2892,7 +3177,7 @@ func (_f *Factory) ConstructLe( if _f.onlyConstants(leftRight) { if _f.onlyConstants(right) { if _f.canConstructBinary(opt.PlusOp, right, leftRight) { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.NormalizeCmpMinusConst) { + if _f.matchedRule == nil || _f.matchedRule(opt.NormalizeCmpMinusConst) { _group = _f.ConstructLe( leftLeft, _f.ConstructPlus( @@ -2901,6 +3186,9 @@ func (_f *Factory) ConstructLe( ), ) _f.mem.AddAltFingerprint(_leExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.NormalizeCmpMinusConst, _group, 0) + } return _group } } @@ -2920,7 +3208,7 @@ func (_f *Factory) ConstructLe( if !_f.onlyConstants(leftRight) { if _f.onlyConstants(right) { if _f.canConstructBinary(opt.MinusOp, leftLeft, right) { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.NormalizeCmpConstMinus) { + if _f.matchedRule == nil || _f.matchedRule(opt.NormalizeCmpConstMinus) { _group = _f.ConstructLe( _f.ConstructMinus( leftLeft, @@ -2929,6 +3217,9 @@ func (_f *Factory) ConstructLe( leftRight, ) _f.mem.AddAltFingerprint(_leExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.NormalizeCmpConstMinus, _group, 0) + } return _group } } @@ -2942,11 +3233,14 @@ func (_f *Factory) ConstructLe( { _nullExpr := _f.mem.NormExpr(left).AsNull() if _nullExpr != nil { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.FoldNullComparisonLeft) { + if _f.matchedRule == nil || _f.matchedRule(opt.FoldNullComparisonLeft) { _group = _f.ConstructNull( _f.boolType(), ) _f.mem.AddAltFingerprint(_leExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.FoldNullComparisonLeft, _group, 0) + } return _group } } @@ -2956,11 +3250,14 @@ func (_f *Factory) ConstructLe( { _nullExpr := _f.mem.NormExpr(right).AsNull() if _nullExpr != nil { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.FoldNullComparisonRight) { + if _f.matchedRule == nil || _f.matchedRule(opt.FoldNullComparisonRight) { _group = _f.ConstructNull( _f.boolType(), ) _f.mem.AddAltFingerprint(_leExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.FoldNullComparisonRight, _group, 0) + } return _group } } @@ -2986,9 +3283,12 @@ func (_f *Factory) ConstructGe( if _variableExpr == nil { _variableExpr2 := _f.mem.NormExpr(right).AsVariable() if _variableExpr2 != nil { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.CommuteVarInequality) { + if _f.matchedRule == nil || _f.matchedRule(opt.CommuteVarInequality) { _group = _f.commuteInequality(opt.GeOp, left, right) _f.mem.AddAltFingerprint(_geExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.CommuteVarInequality, _group, 0) + } return _group } } @@ -2999,9 +3299,12 @@ func (_f *Factory) ConstructGe( { if _f.onlyConstants(left) { if !_f.onlyConstants(right) { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.CommuteConstInequality) { + if _f.matchedRule == nil || _f.matchedRule(opt.CommuteConstInequality) { _group = _f.commuteInequality(opt.GeOp, left, right) _f.mem.AddAltFingerprint(_geExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.CommuteConstInequality, _group, 0) + } return _group } } @@ -3018,7 +3321,7 @@ func (_f *Factory) ConstructGe( if _f.onlyConstants(leftRight) { if _f.onlyConstants(right) { if _f.canConstructBinary(opt.MinusOp, right, leftRight) { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.NormalizeCmpPlusConst) { + if _f.matchedRule == nil || _f.matchedRule(opt.NormalizeCmpPlusConst) { _group = _f.ConstructGe( leftLeft, _f.ConstructMinus( @@ -3027,6 +3330,9 @@ func (_f *Factory) ConstructGe( ), ) _f.mem.AddAltFingerprint(_geExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.NormalizeCmpPlusConst, _group, 0) + } return _group } } @@ -3046,7 +3352,7 @@ func (_f *Factory) ConstructGe( if _f.onlyConstants(leftRight) { if _f.onlyConstants(right) { if _f.canConstructBinary(opt.PlusOp, right, leftRight) { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.NormalizeCmpMinusConst) { + if _f.matchedRule == nil || _f.matchedRule(opt.NormalizeCmpMinusConst) { _group = _f.ConstructGe( leftLeft, _f.ConstructPlus( @@ -3055,6 +3361,9 @@ func (_f *Factory) ConstructGe( ), ) _f.mem.AddAltFingerprint(_geExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.NormalizeCmpMinusConst, _group, 0) + } return _group } } @@ -3074,7 +3383,7 @@ func (_f *Factory) ConstructGe( if !_f.onlyConstants(leftRight) { if _f.onlyConstants(right) { if _f.canConstructBinary(opt.MinusOp, leftLeft, right) { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.NormalizeCmpConstMinus) { + if _f.matchedRule == nil || _f.matchedRule(opt.NormalizeCmpConstMinus) { _group = _f.ConstructGe( _f.ConstructMinus( leftLeft, @@ -3083,6 +3392,9 @@ func (_f *Factory) ConstructGe( leftRight, ) _f.mem.AddAltFingerprint(_geExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.NormalizeCmpConstMinus, _group, 0) + } return _group } } @@ -3096,11 +3408,14 @@ func (_f *Factory) ConstructGe( { _nullExpr := _f.mem.NormExpr(left).AsNull() if _nullExpr != nil { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.FoldNullComparisonLeft) { + if _f.matchedRule == nil || _f.matchedRule(opt.FoldNullComparisonLeft) { _group = _f.ConstructNull( _f.boolType(), ) _f.mem.AddAltFingerprint(_geExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.FoldNullComparisonLeft, _group, 0) + } return _group } } @@ -3110,11 +3425,14 @@ func (_f *Factory) ConstructGe( { _nullExpr := _f.mem.NormExpr(right).AsNull() if _nullExpr != nil { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.FoldNullComparisonRight) { + if _f.matchedRule == nil || _f.matchedRule(opt.FoldNullComparisonRight) { _group = _f.ConstructNull( _f.boolType(), ) _f.mem.AddAltFingerprint(_geExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.FoldNullComparisonRight, _group, 0) + } return _group } } @@ -3138,11 +3456,14 @@ func (_f *Factory) ConstructNe( { _nullExpr := _f.mem.NormExpr(left).AsNull() if _nullExpr != nil { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.FoldNullComparisonLeft) { + if _f.matchedRule == nil || _f.matchedRule(opt.FoldNullComparisonLeft) { _group = _f.ConstructNull( _f.boolType(), ) _f.mem.AddAltFingerprint(_neExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.FoldNullComparisonLeft, _group, 0) + } return _group } } @@ -3152,11 +3473,14 @@ func (_f *Factory) ConstructNe( { _nullExpr := _f.mem.NormExpr(right).AsNull() if _nullExpr != nil { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.FoldNullComparisonRight) { + if _f.matchedRule == nil || _f.matchedRule(opt.FoldNullComparisonRight) { _group = _f.ConstructNull( _f.boolType(), ) _f.mem.AddAltFingerprint(_neExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.FoldNullComparisonRight, _group, 0) + } return _group } } @@ -3168,12 +3492,15 @@ func (_f *Factory) ConstructNe( if _variableExpr == nil { _variableExpr2 := _f.mem.NormExpr(right).AsVariable() if _variableExpr2 != nil { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.CommuteVar) { + if _f.matchedRule == nil || _f.matchedRule(opt.CommuteVar) { _group = _f.ConstructNe( right, left, ) _f.mem.AddAltFingerprint(_neExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.CommuteVar, _group, 0) + } return _group } } @@ -3184,12 +3511,15 @@ func (_f *Factory) ConstructNe( { if _f.onlyConstants(left) { if !_f.onlyConstants(right) { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.CommuteConst) { + if _f.matchedRule == nil || _f.matchedRule(opt.CommuteConst) { _group = _f.ConstructNe( right, left, ) _f.mem.AddAltFingerprint(_neExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.CommuteConst, _group, 0) + } return _group } } @@ -3217,11 +3547,14 @@ func (_f *Factory) ConstructIn( _tupleExpr := _f.mem.NormExpr(right).AsTuple() if _tupleExpr != nil { if _tupleExpr.Elems().Length != 0 { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.FoldNullInNonEmpty) { + if _f.matchedRule == nil || _f.matchedRule(opt.FoldNullInNonEmpty) { _group = _f.ConstructNull( _f.boolType(), ) _f.mem.AddAltFingerprint(_inExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.FoldNullInNonEmpty, _group, 0) + } return _group } } @@ -3236,9 +3569,12 @@ func (_f *Factory) ConstructIn( _tupleExpr := _f.mem.NormExpr(right).AsTuple() if _tupleExpr != nil { if _tupleExpr.Elems().Length == 0 { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.FoldNullInEmpty) { + if _f.matchedRule == nil || _f.matchedRule(opt.FoldNullInEmpty) { _group = _f.ConstructFalse() _f.mem.AddAltFingerprint(_inExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.FoldNullInEmpty, _group, 0) + } return _group } } @@ -3252,7 +3588,7 @@ func (_f *Factory) ConstructIn( if _tupleExpr != nil { elems := _tupleExpr.Elems() if !_f.isSortedUniqueList(elems) { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.NormalizeInConst) { + if _f.matchedRule == nil || _f.matchedRule(opt.NormalizeInConst) { _group = _f.ConstructIn( left, _f.ConstructTuple( @@ -3260,6 +3596,9 @@ func (_f *Factory) ConstructIn( ), ) _f.mem.AddAltFingerprint(_inExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.NormalizeInConst, _group, 0) + } return _group } } @@ -3274,11 +3613,14 @@ func (_f *Factory) ConstructIn( _item := _f.mem.LookupList(_tupleExpr.Elems())[0] _nullExpr := _f.mem.NormExpr(_item).AsNull() if _nullExpr != nil { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.FoldInNull) { + if _f.matchedRule == nil || _f.matchedRule(opt.FoldInNull) { _group = _f.ConstructNull( _f.boolType(), ) _f.mem.AddAltFingerprint(_inExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.FoldInNull, _group, 0) + } return _group } } @@ -3307,11 +3649,14 @@ func (_f *Factory) ConstructNotIn( _tupleExpr := _f.mem.NormExpr(right).AsTuple() if _tupleExpr != nil { if _tupleExpr.Elems().Length != 0 { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.FoldNullInNonEmpty) { + if _f.matchedRule == nil || _f.matchedRule(opt.FoldNullInNonEmpty) { _group = _f.ConstructNull( _f.boolType(), ) _f.mem.AddAltFingerprint(_notInExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.FoldNullInNonEmpty, _group, 0) + } return _group } } @@ -3326,9 +3671,12 @@ func (_f *Factory) ConstructNotIn( _tupleExpr := _f.mem.NormExpr(right).AsTuple() if _tupleExpr != nil { if _tupleExpr.Elems().Length == 0 { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.FoldNullNotInEmpty) { + if _f.matchedRule == nil || _f.matchedRule(opt.FoldNullNotInEmpty) { _group = _f.ConstructTrue() _f.mem.AddAltFingerprint(_notInExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.FoldNullNotInEmpty, _group, 0) + } return _group } } @@ -3342,7 +3690,7 @@ func (_f *Factory) ConstructNotIn( if _tupleExpr != nil { elems := _tupleExpr.Elems() if !_f.isSortedUniqueList(elems) { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.NormalizeInConst) { + if _f.matchedRule == nil || _f.matchedRule(opt.NormalizeInConst) { _group = _f.ConstructNotIn( left, _f.ConstructTuple( @@ -3350,6 +3698,9 @@ func (_f *Factory) ConstructNotIn( ), ) _f.mem.AddAltFingerprint(_notInExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.NormalizeInConst, _group, 0) + } return _group } } @@ -3364,11 +3715,14 @@ func (_f *Factory) ConstructNotIn( _item := _f.mem.LookupList(_tupleExpr.Elems())[0] _nullExpr := _f.mem.NormExpr(_item).AsNull() if _nullExpr != nil { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.FoldInNull) { + if _f.matchedRule == nil || _f.matchedRule(opt.FoldInNull) { _group = _f.ConstructNull( _f.boolType(), ) _f.mem.AddAltFingerprint(_notInExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.FoldInNull, _group, 0) + } return _group } } @@ -3394,11 +3748,14 @@ func (_f *Factory) ConstructLike( { _nullExpr := _f.mem.NormExpr(left).AsNull() if _nullExpr != nil { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.FoldNullComparisonLeft) { + if _f.matchedRule == nil || _f.matchedRule(opt.FoldNullComparisonLeft) { _group = _f.ConstructNull( _f.boolType(), ) _f.mem.AddAltFingerprint(_likeExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.FoldNullComparisonLeft, _group, 0) + } return _group } } @@ -3408,11 +3765,14 @@ func (_f *Factory) ConstructLike( { _nullExpr := _f.mem.NormExpr(right).AsNull() if _nullExpr != nil { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.FoldNullComparisonRight) { + if _f.matchedRule == nil || _f.matchedRule(opt.FoldNullComparisonRight) { _group = _f.ConstructNull( _f.boolType(), ) _f.mem.AddAltFingerprint(_likeExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.FoldNullComparisonRight, _group, 0) + } return _group } } @@ -3436,11 +3796,14 @@ func (_f *Factory) ConstructNotLike( { _nullExpr := _f.mem.NormExpr(left).AsNull() if _nullExpr != nil { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.FoldNullComparisonLeft) { + if _f.matchedRule == nil || _f.matchedRule(opt.FoldNullComparisonLeft) { _group = _f.ConstructNull( _f.boolType(), ) _f.mem.AddAltFingerprint(_notLikeExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.FoldNullComparisonLeft, _group, 0) + } return _group } } @@ -3450,11 +3813,14 @@ func (_f *Factory) ConstructNotLike( { _nullExpr := _f.mem.NormExpr(right).AsNull() if _nullExpr != nil { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.FoldNullComparisonRight) { + if _f.matchedRule == nil || _f.matchedRule(opt.FoldNullComparisonRight) { _group = _f.ConstructNull( _f.boolType(), ) _f.mem.AddAltFingerprint(_notLikeExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.FoldNullComparisonRight, _group, 0) + } return _group } } @@ -3478,11 +3844,14 @@ func (_f *Factory) ConstructILike( { _nullExpr := _f.mem.NormExpr(left).AsNull() if _nullExpr != nil { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.FoldNullComparisonLeft) { + if _f.matchedRule == nil || _f.matchedRule(opt.FoldNullComparisonLeft) { _group = _f.ConstructNull( _f.boolType(), ) _f.mem.AddAltFingerprint(_iLikeExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.FoldNullComparisonLeft, _group, 0) + } return _group } } @@ -3492,11 +3861,14 @@ func (_f *Factory) ConstructILike( { _nullExpr := _f.mem.NormExpr(right).AsNull() if _nullExpr != nil { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.FoldNullComparisonRight) { + if _f.matchedRule == nil || _f.matchedRule(opt.FoldNullComparisonRight) { _group = _f.ConstructNull( _f.boolType(), ) _f.mem.AddAltFingerprint(_iLikeExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.FoldNullComparisonRight, _group, 0) + } return _group } } @@ -3520,11 +3892,14 @@ func (_f *Factory) ConstructNotILike( { _nullExpr := _f.mem.NormExpr(left).AsNull() if _nullExpr != nil { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.FoldNullComparisonLeft) { + if _f.matchedRule == nil || _f.matchedRule(opt.FoldNullComparisonLeft) { _group = _f.ConstructNull( _f.boolType(), ) _f.mem.AddAltFingerprint(_notILikeExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.FoldNullComparisonLeft, _group, 0) + } return _group } } @@ -3534,11 +3909,14 @@ func (_f *Factory) ConstructNotILike( { _nullExpr := _f.mem.NormExpr(right).AsNull() if _nullExpr != nil { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.FoldNullComparisonRight) { + if _f.matchedRule == nil || _f.matchedRule(opt.FoldNullComparisonRight) { _group = _f.ConstructNull( _f.boolType(), ) _f.mem.AddAltFingerprint(_notILikeExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.FoldNullComparisonRight, _group, 0) + } return _group } } @@ -3562,11 +3940,14 @@ func (_f *Factory) ConstructSimilarTo( { _nullExpr := _f.mem.NormExpr(left).AsNull() if _nullExpr != nil { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.FoldNullComparisonLeft) { + if _f.matchedRule == nil || _f.matchedRule(opt.FoldNullComparisonLeft) { _group = _f.ConstructNull( _f.boolType(), ) _f.mem.AddAltFingerprint(_similarToExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.FoldNullComparisonLeft, _group, 0) + } return _group } } @@ -3576,11 +3957,14 @@ func (_f *Factory) ConstructSimilarTo( { _nullExpr := _f.mem.NormExpr(right).AsNull() if _nullExpr != nil { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.FoldNullComparisonRight) { + if _f.matchedRule == nil || _f.matchedRule(opt.FoldNullComparisonRight) { _group = _f.ConstructNull( _f.boolType(), ) _f.mem.AddAltFingerprint(_similarToExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.FoldNullComparisonRight, _group, 0) + } return _group } } @@ -3604,11 +3988,14 @@ func (_f *Factory) ConstructNotSimilarTo( { _nullExpr := _f.mem.NormExpr(left).AsNull() if _nullExpr != nil { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.FoldNullComparisonLeft) { + if _f.matchedRule == nil || _f.matchedRule(opt.FoldNullComparisonLeft) { _group = _f.ConstructNull( _f.boolType(), ) _f.mem.AddAltFingerprint(_notSimilarToExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.FoldNullComparisonLeft, _group, 0) + } return _group } } @@ -3618,11 +4005,14 @@ func (_f *Factory) ConstructNotSimilarTo( { _nullExpr := _f.mem.NormExpr(right).AsNull() if _nullExpr != nil { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.FoldNullComparisonRight) { + if _f.matchedRule == nil || _f.matchedRule(opt.FoldNullComparisonRight) { _group = _f.ConstructNull( _f.boolType(), ) _f.mem.AddAltFingerprint(_notSimilarToExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.FoldNullComparisonRight, _group, 0) + } return _group } } @@ -3646,11 +4036,14 @@ func (_f *Factory) ConstructRegMatch( { _nullExpr := _f.mem.NormExpr(left).AsNull() if _nullExpr != nil { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.FoldNullComparisonLeft) { + if _f.matchedRule == nil || _f.matchedRule(opt.FoldNullComparisonLeft) { _group = _f.ConstructNull( _f.boolType(), ) _f.mem.AddAltFingerprint(_regMatchExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.FoldNullComparisonLeft, _group, 0) + } return _group } } @@ -3660,11 +4053,14 @@ func (_f *Factory) ConstructRegMatch( { _nullExpr := _f.mem.NormExpr(right).AsNull() if _nullExpr != nil { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.FoldNullComparisonRight) { + if _f.matchedRule == nil || _f.matchedRule(opt.FoldNullComparisonRight) { _group = _f.ConstructNull( _f.boolType(), ) _f.mem.AddAltFingerprint(_regMatchExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.FoldNullComparisonRight, _group, 0) + } return _group } } @@ -3688,11 +4084,14 @@ func (_f *Factory) ConstructNotRegMatch( { _nullExpr := _f.mem.NormExpr(left).AsNull() if _nullExpr != nil { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.FoldNullComparisonLeft) { + if _f.matchedRule == nil || _f.matchedRule(opt.FoldNullComparisonLeft) { _group = _f.ConstructNull( _f.boolType(), ) _f.mem.AddAltFingerprint(_notRegMatchExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.FoldNullComparisonLeft, _group, 0) + } return _group } } @@ -3702,11 +4101,14 @@ func (_f *Factory) ConstructNotRegMatch( { _nullExpr := _f.mem.NormExpr(right).AsNull() if _nullExpr != nil { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.FoldNullComparisonRight) { + if _f.matchedRule == nil || _f.matchedRule(opt.FoldNullComparisonRight) { _group = _f.ConstructNull( _f.boolType(), ) _f.mem.AddAltFingerprint(_notRegMatchExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.FoldNullComparisonRight, _group, 0) + } return _group } } @@ -3730,11 +4132,14 @@ func (_f *Factory) ConstructRegIMatch( { _nullExpr := _f.mem.NormExpr(left).AsNull() if _nullExpr != nil { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.FoldNullComparisonLeft) { + if _f.matchedRule == nil || _f.matchedRule(opt.FoldNullComparisonLeft) { _group = _f.ConstructNull( _f.boolType(), ) _f.mem.AddAltFingerprint(_regIMatchExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.FoldNullComparisonLeft, _group, 0) + } return _group } } @@ -3744,11 +4149,14 @@ func (_f *Factory) ConstructRegIMatch( { _nullExpr := _f.mem.NormExpr(right).AsNull() if _nullExpr != nil { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.FoldNullComparisonRight) { + if _f.matchedRule == nil || _f.matchedRule(opt.FoldNullComparisonRight) { _group = _f.ConstructNull( _f.boolType(), ) _f.mem.AddAltFingerprint(_regIMatchExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.FoldNullComparisonRight, _group, 0) + } return _group } } @@ -3772,11 +4180,14 @@ func (_f *Factory) ConstructNotRegIMatch( { _nullExpr := _f.mem.NormExpr(left).AsNull() if _nullExpr != nil { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.FoldNullComparisonLeft) { + if _f.matchedRule == nil || _f.matchedRule(opt.FoldNullComparisonLeft) { _group = _f.ConstructNull( _f.boolType(), ) _f.mem.AddAltFingerprint(_notRegIMatchExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.FoldNullComparisonLeft, _group, 0) + } return _group } } @@ -3786,11 +4197,14 @@ func (_f *Factory) ConstructNotRegIMatch( { _nullExpr := _f.mem.NormExpr(right).AsNull() if _nullExpr != nil { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.FoldNullComparisonRight) { + if _f.matchedRule == nil || _f.matchedRule(opt.FoldNullComparisonRight) { _group = _f.ConstructNull( _f.boolType(), ) _f.mem.AddAltFingerprint(_notRegIMatchExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.FoldNullComparisonRight, _group, 0) + } return _group } } @@ -3816,9 +4230,12 @@ func (_f *Factory) ConstructIs( if _nullExpr != nil { _nullExpr2 := _f.mem.NormExpr(right).AsNull() if _nullExpr2 != nil { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.FoldIsNull) { + if _f.matchedRule == nil || _f.matchedRule(opt.FoldIsNull) { _group = _f.ConstructTrue() _f.mem.AddAltFingerprint(_isExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.FoldIsNull, _group, 0) + } return _group } } @@ -3833,9 +4250,12 @@ func (_f *Factory) ConstructIs( if _nullExpr == nil { _nullExpr2 := _f.mem.NormExpr(right).AsNull() if _nullExpr2 != nil { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.FoldNonNullIsNull) { + if _f.matchedRule == nil || _f.matchedRule(opt.FoldNonNullIsNull) { _group = _f.ConstructFalse() _f.mem.AddAltFingerprint(_isExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.FoldNonNullIsNull, _group, 0) + } return _group } } @@ -3849,12 +4269,15 @@ func (_f *Factory) ConstructIs( if _nullExpr != nil { _nullExpr2 := _f.mem.NormExpr(right).AsNull() if _nullExpr2 == nil { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.CommuteNullIs) { + if _f.matchedRule == nil || _f.matchedRule(opt.CommuteNullIs) { _group = _f.ConstructIs( right, left, ) _f.mem.AddAltFingerprint(_isExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.CommuteNullIs, _group, 0) + } return _group } } @@ -3867,12 +4290,15 @@ func (_f *Factory) ConstructIs( if _variableExpr == nil { _variableExpr2 := _f.mem.NormExpr(right).AsVariable() if _variableExpr2 != nil { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.CommuteVar) { + if _f.matchedRule == nil || _f.matchedRule(opt.CommuteVar) { _group = _f.ConstructIs( right, left, ) _f.mem.AddAltFingerprint(_isExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.CommuteVar, _group, 0) + } return _group } } @@ -3883,12 +4309,15 @@ func (_f *Factory) ConstructIs( { if _f.onlyConstants(left) { if !_f.onlyConstants(right) { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.CommuteConst) { + if _f.matchedRule == nil || _f.matchedRule(opt.CommuteConst) { _group = _f.ConstructIs( right, left, ) _f.mem.AddAltFingerprint(_isExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.CommuteConst, _group, 0) + } return _group } } @@ -3915,9 +4344,12 @@ func (_f *Factory) ConstructIsNot( if _nullExpr != nil { _nullExpr2 := _f.mem.NormExpr(right).AsNull() if _nullExpr2 != nil { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.FoldIsNotNull) { + if _f.matchedRule == nil || _f.matchedRule(opt.FoldIsNotNull) { _group = _f.ConstructFalse() _f.mem.AddAltFingerprint(_isNotExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.FoldIsNotNull, _group, 0) + } return _group } } @@ -3932,9 +4364,12 @@ func (_f *Factory) ConstructIsNot( if _nullExpr == nil { _nullExpr2 := _f.mem.NormExpr(right).AsNull() if _nullExpr2 != nil { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.FoldNonNullIsNotNull) { + if _f.matchedRule == nil || _f.matchedRule(opt.FoldNonNullIsNotNull) { _group = _f.ConstructTrue() _f.mem.AddAltFingerprint(_isNotExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.FoldNonNullIsNotNull, _group, 0) + } return _group } } @@ -3948,12 +4383,15 @@ func (_f *Factory) ConstructIsNot( if _nullExpr != nil { _nullExpr2 := _f.mem.NormExpr(right).AsNull() if _nullExpr2 == nil { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.CommuteNullIs) { + if _f.matchedRule == nil || _f.matchedRule(opt.CommuteNullIs) { _group = _f.ConstructIsNot( right, left, ) _f.mem.AddAltFingerprint(_isNotExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.CommuteNullIs, _group, 0) + } return _group } } @@ -3966,12 +4404,15 @@ func (_f *Factory) ConstructIsNot( if _variableExpr == nil { _variableExpr2 := _f.mem.NormExpr(right).AsVariable() if _variableExpr2 != nil { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.CommuteVar) { + if _f.matchedRule == nil || _f.matchedRule(opt.CommuteVar) { _group = _f.ConstructIsNot( right, left, ) _f.mem.AddAltFingerprint(_isNotExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.CommuteVar, _group, 0) + } return _group } } @@ -3982,12 +4423,15 @@ func (_f *Factory) ConstructIsNot( { if _f.onlyConstants(left) { if !_f.onlyConstants(right) { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.CommuteConst) { + if _f.matchedRule == nil || _f.matchedRule(opt.CommuteConst) { _group = _f.ConstructIsNot( right, left, ) _f.mem.AddAltFingerprint(_isNotExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.CommuteConst, _group, 0) + } return _group } } @@ -4012,11 +4456,14 @@ func (_f *Factory) ConstructContains( { _nullExpr := _f.mem.NormExpr(left).AsNull() if _nullExpr != nil { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.FoldNullComparisonLeft) { + if _f.matchedRule == nil || _f.matchedRule(opt.FoldNullComparisonLeft) { _group = _f.ConstructNull( _f.boolType(), ) _f.mem.AddAltFingerprint(_containsExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.FoldNullComparisonLeft, _group, 0) + } return _group } } @@ -4026,11 +4473,14 @@ func (_f *Factory) ConstructContains( { _nullExpr := _f.mem.NormExpr(right).AsNull() if _nullExpr != nil { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.FoldNullComparisonRight) { + if _f.matchedRule == nil || _f.matchedRule(opt.FoldNullComparisonRight) { _group = _f.ConstructNull( _f.boolType(), ) _f.mem.AddAltFingerprint(_containsExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.FoldNullComparisonRight, _group, 0) + } return _group } } @@ -4054,11 +4504,14 @@ func (_f *Factory) ConstructJsonExists( { _nullExpr := _f.mem.NormExpr(left).AsNull() if _nullExpr != nil { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.FoldNullComparisonLeft) { + if _f.matchedRule == nil || _f.matchedRule(opt.FoldNullComparisonLeft) { _group = _f.ConstructNull( _f.boolType(), ) _f.mem.AddAltFingerprint(_jsonExistsExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.FoldNullComparisonLeft, _group, 0) + } return _group } } @@ -4068,11 +4521,14 @@ func (_f *Factory) ConstructJsonExists( { _nullExpr := _f.mem.NormExpr(right).AsNull() if _nullExpr != nil { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.FoldNullComparisonRight) { + if _f.matchedRule == nil || _f.matchedRule(opt.FoldNullComparisonRight) { _group = _f.ConstructNull( _f.boolType(), ) _f.mem.AddAltFingerprint(_jsonExistsExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.FoldNullComparisonRight, _group, 0) + } return _group } } @@ -4096,11 +4552,14 @@ func (_f *Factory) ConstructJsonAllExists( { _nullExpr := _f.mem.NormExpr(left).AsNull() if _nullExpr != nil { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.FoldNullComparisonLeft) { + if _f.matchedRule == nil || _f.matchedRule(opt.FoldNullComparisonLeft) { _group = _f.ConstructNull( _f.boolType(), ) _f.mem.AddAltFingerprint(_jsonAllExistsExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.FoldNullComparisonLeft, _group, 0) + } return _group } } @@ -4110,11 +4569,14 @@ func (_f *Factory) ConstructJsonAllExists( { _nullExpr := _f.mem.NormExpr(right).AsNull() if _nullExpr != nil { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.FoldNullComparisonRight) { + if _f.matchedRule == nil || _f.matchedRule(opt.FoldNullComparisonRight) { _group = _f.ConstructNull( _f.boolType(), ) _f.mem.AddAltFingerprint(_jsonAllExistsExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.FoldNullComparisonRight, _group, 0) + } return _group } } @@ -4138,11 +4600,14 @@ func (_f *Factory) ConstructJsonSomeExists( { _nullExpr := _f.mem.NormExpr(left).AsNull() if _nullExpr != nil { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.FoldNullComparisonLeft) { + if _f.matchedRule == nil || _f.matchedRule(opt.FoldNullComparisonLeft) { _group = _f.ConstructNull( _f.boolType(), ) _f.mem.AddAltFingerprint(_jsonSomeExistsExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.FoldNullComparisonLeft, _group, 0) + } return _group } } @@ -4152,11 +4617,14 @@ func (_f *Factory) ConstructJsonSomeExists( { _nullExpr := _f.mem.NormExpr(right).AsNull() if _nullExpr != nil { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.FoldNullComparisonRight) { + if _f.matchedRule == nil || _f.matchedRule(opt.FoldNullComparisonRight) { _group = _f.ConstructNull( _f.boolType(), ) _f.mem.AddAltFingerprint(_jsonSomeExistsExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.FoldNullComparisonRight, _group, 0) + } return _group } } @@ -4182,12 +4650,15 @@ func (_f *Factory) ConstructBitand( if _variableExpr == nil { _variableExpr2 := _f.mem.NormExpr(right).AsVariable() if _variableExpr2 != nil { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.CommuteVar) { + if _f.matchedRule == nil || _f.matchedRule(opt.CommuteVar) { _group = _f.ConstructBitand( right, left, ) _f.mem.AddAltFingerprint(_bitandExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.CommuteVar, _group, 0) + } return _group } } @@ -4198,12 +4669,15 @@ func (_f *Factory) ConstructBitand( { if _f.onlyConstants(left) { if !_f.onlyConstants(right) { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.CommuteConst) { + if _f.matchedRule == nil || _f.matchedRule(opt.CommuteConst) { _group = _f.ConstructBitand( right, left, ) _f.mem.AddAltFingerprint(_bitandExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.CommuteConst, _group, 0) + } return _group } } @@ -4215,9 +4689,12 @@ func (_f *Factory) ConstructBitand( _nullExpr := _f.mem.NormExpr(left).AsNull() if _nullExpr != nil { if !_f.allowNullArgs(opt.BitandOp, left, right) { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.FoldNullBinaryLeft) { + if _f.matchedRule == nil || _f.matchedRule(opt.FoldNullBinaryLeft) { _group = _f.foldNullBinary(opt.BitandOp, left, right) _f.mem.AddAltFingerprint(_bitandExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.FoldNullBinaryLeft, _group, 0) + } return _group } } @@ -4229,9 +4706,12 @@ func (_f *Factory) ConstructBitand( _nullExpr := _f.mem.NormExpr(right).AsNull() if _nullExpr != nil { if !_f.allowNullArgs(opt.BitandOp, left, right) { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.FoldNullBinaryRight) { + if _f.matchedRule == nil || _f.matchedRule(opt.FoldNullBinaryRight) { _group = _f.foldNullBinary(opt.BitandOp, left, right) _f.mem.AddAltFingerprint(_bitandExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.FoldNullBinaryRight, _group, 0) + } return _group } } @@ -4258,12 +4738,15 @@ func (_f *Factory) ConstructBitor( if _variableExpr == nil { _variableExpr2 := _f.mem.NormExpr(right).AsVariable() if _variableExpr2 != nil { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.CommuteVar) { + if _f.matchedRule == nil || _f.matchedRule(opt.CommuteVar) { _group = _f.ConstructBitor( right, left, ) _f.mem.AddAltFingerprint(_bitorExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.CommuteVar, _group, 0) + } return _group } } @@ -4274,12 +4757,15 @@ func (_f *Factory) ConstructBitor( { if _f.onlyConstants(left) { if !_f.onlyConstants(right) { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.CommuteConst) { + if _f.matchedRule == nil || _f.matchedRule(opt.CommuteConst) { _group = _f.ConstructBitor( right, left, ) _f.mem.AddAltFingerprint(_bitorExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.CommuteConst, _group, 0) + } return _group } } @@ -4291,9 +4777,12 @@ func (_f *Factory) ConstructBitor( _nullExpr := _f.mem.NormExpr(left).AsNull() if _nullExpr != nil { if !_f.allowNullArgs(opt.BitorOp, left, right) { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.FoldNullBinaryLeft) { + if _f.matchedRule == nil || _f.matchedRule(opt.FoldNullBinaryLeft) { _group = _f.foldNullBinary(opt.BitorOp, left, right) _f.mem.AddAltFingerprint(_bitorExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.FoldNullBinaryLeft, _group, 0) + } return _group } } @@ -4305,9 +4794,12 @@ func (_f *Factory) ConstructBitor( _nullExpr := _f.mem.NormExpr(right).AsNull() if _nullExpr != nil { if !_f.allowNullArgs(opt.BitorOp, left, right) { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.FoldNullBinaryRight) { + if _f.matchedRule == nil || _f.matchedRule(opt.FoldNullBinaryRight) { _group = _f.foldNullBinary(opt.BitorOp, left, right) _f.mem.AddAltFingerprint(_bitorExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.FoldNullBinaryRight, _group, 0) + } return _group } } @@ -4334,12 +4826,15 @@ func (_f *Factory) ConstructBitxor( if _variableExpr == nil { _variableExpr2 := _f.mem.NormExpr(right).AsVariable() if _variableExpr2 != nil { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.CommuteVar) { + if _f.matchedRule == nil || _f.matchedRule(opt.CommuteVar) { _group = _f.ConstructBitxor( right, left, ) _f.mem.AddAltFingerprint(_bitxorExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.CommuteVar, _group, 0) + } return _group } } @@ -4350,12 +4845,15 @@ func (_f *Factory) ConstructBitxor( { if _f.onlyConstants(left) { if !_f.onlyConstants(right) { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.CommuteConst) { + if _f.matchedRule == nil || _f.matchedRule(opt.CommuteConst) { _group = _f.ConstructBitxor( right, left, ) _f.mem.AddAltFingerprint(_bitxorExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.CommuteConst, _group, 0) + } return _group } } @@ -4367,9 +4865,12 @@ func (_f *Factory) ConstructBitxor( _nullExpr := _f.mem.NormExpr(left).AsNull() if _nullExpr != nil { if !_f.allowNullArgs(opt.BitxorOp, left, right) { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.FoldNullBinaryLeft) { + if _f.matchedRule == nil || _f.matchedRule(opt.FoldNullBinaryLeft) { _group = _f.foldNullBinary(opt.BitxorOp, left, right) _f.mem.AddAltFingerprint(_bitxorExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.FoldNullBinaryLeft, _group, 0) + } return _group } } @@ -4381,9 +4882,12 @@ func (_f *Factory) ConstructBitxor( _nullExpr := _f.mem.NormExpr(right).AsNull() if _nullExpr != nil { if !_f.allowNullArgs(opt.BitxorOp, left, right) { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.FoldNullBinaryRight) { + if _f.matchedRule == nil || _f.matchedRule(opt.FoldNullBinaryRight) { _group = _f.foldNullBinary(opt.BitxorOp, left, right) _f.mem.AddAltFingerprint(_bitxorExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.FoldNullBinaryRight, _group, 0) + } return _group } } @@ -4409,9 +4913,12 @@ func (_f *Factory) ConstructPlus( _constExpr := _f.mem.NormExpr(right).AsConst() if _constExpr != nil { if _f.isZero(right) { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.FoldPlusZero) { + if _f.matchedRule == nil || _f.matchedRule(opt.FoldPlusZero) { _group = left _f.mem.AddAltFingerprint(_plusExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.FoldPlusZero, _group, 0) + } return _group } } @@ -4423,9 +4930,12 @@ func (_f *Factory) ConstructPlus( _constExpr := _f.mem.NormExpr(left).AsConst() if _constExpr != nil { if _f.isZero(left) { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.FoldZeroPlus) { + if _f.matchedRule == nil || _f.matchedRule(opt.FoldZeroPlus) { _group = right _f.mem.AddAltFingerprint(_plusExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.FoldZeroPlus, _group, 0) + } return _group } } @@ -4438,12 +4948,15 @@ func (_f *Factory) ConstructPlus( if _variableExpr == nil { _variableExpr2 := _f.mem.NormExpr(right).AsVariable() if _variableExpr2 != nil { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.CommuteVar) { + if _f.matchedRule == nil || _f.matchedRule(opt.CommuteVar) { _group = _f.ConstructPlus( right, left, ) _f.mem.AddAltFingerprint(_plusExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.CommuteVar, _group, 0) + } return _group } } @@ -4454,12 +4967,15 @@ func (_f *Factory) ConstructPlus( { if _f.onlyConstants(left) { if !_f.onlyConstants(right) { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.CommuteConst) { + if _f.matchedRule == nil || _f.matchedRule(opt.CommuteConst) { _group = _f.ConstructPlus( right, left, ) _f.mem.AddAltFingerprint(_plusExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.CommuteConst, _group, 0) + } return _group } } @@ -4471,9 +4987,12 @@ func (_f *Factory) ConstructPlus( _nullExpr := _f.mem.NormExpr(left).AsNull() if _nullExpr != nil { if !_f.allowNullArgs(opt.PlusOp, left, right) { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.FoldNullBinaryLeft) { + if _f.matchedRule == nil || _f.matchedRule(opt.FoldNullBinaryLeft) { _group = _f.foldNullBinary(opt.PlusOp, left, right) _f.mem.AddAltFingerprint(_plusExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.FoldNullBinaryLeft, _group, 0) + } return _group } } @@ -4485,9 +5004,12 @@ func (_f *Factory) ConstructPlus( _nullExpr := _f.mem.NormExpr(right).AsNull() if _nullExpr != nil { if !_f.allowNullArgs(opt.PlusOp, left, right) { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.FoldNullBinaryRight) { + if _f.matchedRule == nil || _f.matchedRule(opt.FoldNullBinaryRight) { _group = _f.foldNullBinary(opt.PlusOp, left, right) _f.mem.AddAltFingerprint(_plusExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.FoldNullBinaryRight, _group, 0) + } return _group } } @@ -4513,9 +5035,12 @@ func (_f *Factory) ConstructMinus( _constExpr := _f.mem.NormExpr(right).AsConst() if _constExpr != nil { if _f.isZero(right) { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.FoldMinusZero) { + if _f.matchedRule == nil || _f.matchedRule(opt.FoldMinusZero) { _group = left _f.mem.AddAltFingerprint(_minusExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.FoldMinusZero, _group, 0) + } return _group } } @@ -4527,9 +5052,12 @@ func (_f *Factory) ConstructMinus( _nullExpr := _f.mem.NormExpr(left).AsNull() if _nullExpr != nil { if !_f.allowNullArgs(opt.MinusOp, left, right) { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.FoldNullBinaryLeft) { + if _f.matchedRule == nil || _f.matchedRule(opt.FoldNullBinaryLeft) { _group = _f.foldNullBinary(opt.MinusOp, left, right) _f.mem.AddAltFingerprint(_minusExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.FoldNullBinaryLeft, _group, 0) + } return _group } } @@ -4541,9 +5069,12 @@ func (_f *Factory) ConstructMinus( _nullExpr := _f.mem.NormExpr(right).AsNull() if _nullExpr != nil { if !_f.allowNullArgs(opt.MinusOp, left, right) { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.FoldNullBinaryRight) { + if _f.matchedRule == nil || _f.matchedRule(opt.FoldNullBinaryRight) { _group = _f.foldNullBinary(opt.MinusOp, left, right) _f.mem.AddAltFingerprint(_minusExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.FoldNullBinaryRight, _group, 0) + } return _group } } @@ -4569,9 +5100,12 @@ func (_f *Factory) ConstructMult( _constExpr := _f.mem.NormExpr(right).AsConst() if _constExpr != nil { if _f.isOne(right) { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.FoldMultOne) { + if _f.matchedRule == nil || _f.matchedRule(opt.FoldMultOne) { _group = left _f.mem.AddAltFingerprint(_multExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.FoldMultOne, _group, 0) + } return _group } } @@ -4583,9 +5117,12 @@ func (_f *Factory) ConstructMult( _constExpr := _f.mem.NormExpr(left).AsConst() if _constExpr != nil { if _f.isOne(left) { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.FoldOneMult) { + if _f.matchedRule == nil || _f.matchedRule(opt.FoldOneMult) { _group = right _f.mem.AddAltFingerprint(_multExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.FoldOneMult, _group, 0) + } return _group } } @@ -4598,12 +5135,15 @@ func (_f *Factory) ConstructMult( if _variableExpr == nil { _variableExpr2 := _f.mem.NormExpr(right).AsVariable() if _variableExpr2 != nil { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.CommuteVar) { + if _f.matchedRule == nil || _f.matchedRule(opt.CommuteVar) { _group = _f.ConstructMult( right, left, ) _f.mem.AddAltFingerprint(_multExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.CommuteVar, _group, 0) + } return _group } } @@ -4614,12 +5154,15 @@ func (_f *Factory) ConstructMult( { if _f.onlyConstants(left) { if !_f.onlyConstants(right) { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.CommuteConst) { + if _f.matchedRule == nil || _f.matchedRule(opt.CommuteConst) { _group = _f.ConstructMult( right, left, ) _f.mem.AddAltFingerprint(_multExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.CommuteConst, _group, 0) + } return _group } } @@ -4631,9 +5174,12 @@ func (_f *Factory) ConstructMult( _nullExpr := _f.mem.NormExpr(left).AsNull() if _nullExpr != nil { if !_f.allowNullArgs(opt.MultOp, left, right) { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.FoldNullBinaryLeft) { + if _f.matchedRule == nil || _f.matchedRule(opt.FoldNullBinaryLeft) { _group = _f.foldNullBinary(opt.MultOp, left, right) _f.mem.AddAltFingerprint(_multExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.FoldNullBinaryLeft, _group, 0) + } return _group } } @@ -4645,9 +5191,12 @@ func (_f *Factory) ConstructMult( _nullExpr := _f.mem.NormExpr(right).AsNull() if _nullExpr != nil { if !_f.allowNullArgs(opt.MultOp, left, right) { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.FoldNullBinaryRight) { + if _f.matchedRule == nil || _f.matchedRule(opt.FoldNullBinaryRight) { _group = _f.foldNullBinary(opt.MultOp, left, right) _f.mem.AddAltFingerprint(_multExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.FoldNullBinaryRight, _group, 0) + } return _group } } @@ -4673,9 +5222,12 @@ func (_f *Factory) ConstructDiv( _constExpr := _f.mem.NormExpr(right).AsConst() if _constExpr != nil { if _f.isOne(right) { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.FoldDivOne) { + if _f.matchedRule == nil || _f.matchedRule(opt.FoldDivOne) { _group = left _f.mem.AddAltFingerprint(_divExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.FoldDivOne, _group, 0) + } return _group } } @@ -4687,9 +5239,12 @@ func (_f *Factory) ConstructDiv( _nullExpr := _f.mem.NormExpr(left).AsNull() if _nullExpr != nil { if !_f.allowNullArgs(opt.DivOp, left, right) { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.FoldNullBinaryLeft) { + if _f.matchedRule == nil || _f.matchedRule(opt.FoldNullBinaryLeft) { _group = _f.foldNullBinary(opt.DivOp, left, right) _f.mem.AddAltFingerprint(_divExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.FoldNullBinaryLeft, _group, 0) + } return _group } } @@ -4701,9 +5256,12 @@ func (_f *Factory) ConstructDiv( _nullExpr := _f.mem.NormExpr(right).AsNull() if _nullExpr != nil { if !_f.allowNullArgs(opt.DivOp, left, right) { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.FoldNullBinaryRight) { + if _f.matchedRule == nil || _f.matchedRule(opt.FoldNullBinaryRight) { _group = _f.foldNullBinary(opt.DivOp, left, right) _f.mem.AddAltFingerprint(_divExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.FoldNullBinaryRight, _group, 0) + } return _group } } @@ -4729,9 +5287,12 @@ func (_f *Factory) ConstructFloorDiv( _constExpr := _f.mem.NormExpr(right).AsConst() if _constExpr != nil { if _f.isOne(right) { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.FoldDivOne) { + if _f.matchedRule == nil || _f.matchedRule(opt.FoldDivOne) { _group = left _f.mem.AddAltFingerprint(_floorDivExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.FoldDivOne, _group, 0) + } return _group } } @@ -4743,9 +5304,12 @@ func (_f *Factory) ConstructFloorDiv( _nullExpr := _f.mem.NormExpr(left).AsNull() if _nullExpr != nil { if !_f.allowNullArgs(opt.FloorDivOp, left, right) { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.FoldNullBinaryLeft) { + if _f.matchedRule == nil || _f.matchedRule(opt.FoldNullBinaryLeft) { _group = _f.foldNullBinary(opt.FloorDivOp, left, right) _f.mem.AddAltFingerprint(_floorDivExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.FoldNullBinaryLeft, _group, 0) + } return _group } } @@ -4757,9 +5321,12 @@ func (_f *Factory) ConstructFloorDiv( _nullExpr := _f.mem.NormExpr(right).AsNull() if _nullExpr != nil { if !_f.allowNullArgs(opt.FloorDivOp, left, right) { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.FoldNullBinaryRight) { + if _f.matchedRule == nil || _f.matchedRule(opt.FoldNullBinaryRight) { _group = _f.foldNullBinary(opt.FloorDivOp, left, right) _f.mem.AddAltFingerprint(_floorDivExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.FoldNullBinaryRight, _group, 0) + } return _group } } @@ -4785,9 +5352,12 @@ func (_f *Factory) ConstructMod( _nullExpr := _f.mem.NormExpr(left).AsNull() if _nullExpr != nil { if !_f.allowNullArgs(opt.ModOp, left, right) { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.FoldNullBinaryLeft) { + if _f.matchedRule == nil || _f.matchedRule(opt.FoldNullBinaryLeft) { _group = _f.foldNullBinary(opt.ModOp, left, right) _f.mem.AddAltFingerprint(_modExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.FoldNullBinaryLeft, _group, 0) + } return _group } } @@ -4799,9 +5369,12 @@ func (_f *Factory) ConstructMod( _nullExpr := _f.mem.NormExpr(right).AsNull() if _nullExpr != nil { if !_f.allowNullArgs(opt.ModOp, left, right) { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.FoldNullBinaryRight) { + if _f.matchedRule == nil || _f.matchedRule(opt.FoldNullBinaryRight) { _group = _f.foldNullBinary(opt.ModOp, left, right) _f.mem.AddAltFingerprint(_modExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.FoldNullBinaryRight, _group, 0) + } return _group } } @@ -4827,9 +5400,12 @@ func (_f *Factory) ConstructPow( _nullExpr := _f.mem.NormExpr(left).AsNull() if _nullExpr != nil { if !_f.allowNullArgs(opt.PowOp, left, right) { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.FoldNullBinaryLeft) { + if _f.matchedRule == nil || _f.matchedRule(opt.FoldNullBinaryLeft) { _group = _f.foldNullBinary(opt.PowOp, left, right) _f.mem.AddAltFingerprint(_powExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.FoldNullBinaryLeft, _group, 0) + } return _group } } @@ -4841,9 +5417,12 @@ func (_f *Factory) ConstructPow( _nullExpr := _f.mem.NormExpr(right).AsNull() if _nullExpr != nil { if !_f.allowNullArgs(opt.PowOp, left, right) { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.FoldNullBinaryRight) { + if _f.matchedRule == nil || _f.matchedRule(opt.FoldNullBinaryRight) { _group = _f.foldNullBinary(opt.PowOp, left, right) _f.mem.AddAltFingerprint(_powExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.FoldNullBinaryRight, _group, 0) + } return _group } } @@ -4869,9 +5448,12 @@ func (_f *Factory) ConstructConcat( _nullExpr := _f.mem.NormExpr(left).AsNull() if _nullExpr != nil { if !_f.allowNullArgs(opt.ConcatOp, left, right) { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.FoldNullBinaryLeft) { + if _f.matchedRule == nil || _f.matchedRule(opt.FoldNullBinaryLeft) { _group = _f.foldNullBinary(opt.ConcatOp, left, right) _f.mem.AddAltFingerprint(_concatExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.FoldNullBinaryLeft, _group, 0) + } return _group } } @@ -4883,9 +5465,12 @@ func (_f *Factory) ConstructConcat( _nullExpr := _f.mem.NormExpr(right).AsNull() if _nullExpr != nil { if !_f.allowNullArgs(opt.ConcatOp, left, right) { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.FoldNullBinaryRight) { + if _f.matchedRule == nil || _f.matchedRule(opt.FoldNullBinaryRight) { _group = _f.foldNullBinary(opt.ConcatOp, left, right) _f.mem.AddAltFingerprint(_concatExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.FoldNullBinaryRight, _group, 0) + } return _group } } @@ -4911,9 +5496,12 @@ func (_f *Factory) ConstructLShift( _nullExpr := _f.mem.NormExpr(left).AsNull() if _nullExpr != nil { if !_f.allowNullArgs(opt.LShiftOp, left, right) { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.FoldNullBinaryLeft) { + if _f.matchedRule == nil || _f.matchedRule(opt.FoldNullBinaryLeft) { _group = _f.foldNullBinary(opt.LShiftOp, left, right) _f.mem.AddAltFingerprint(_lShiftExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.FoldNullBinaryLeft, _group, 0) + } return _group } } @@ -4925,9 +5513,12 @@ func (_f *Factory) ConstructLShift( _nullExpr := _f.mem.NormExpr(right).AsNull() if _nullExpr != nil { if !_f.allowNullArgs(opt.LShiftOp, left, right) { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.FoldNullBinaryRight) { + if _f.matchedRule == nil || _f.matchedRule(opt.FoldNullBinaryRight) { _group = _f.foldNullBinary(opt.LShiftOp, left, right) _f.mem.AddAltFingerprint(_lShiftExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.FoldNullBinaryRight, _group, 0) + } return _group } } @@ -4953,9 +5544,12 @@ func (_f *Factory) ConstructRShift( _nullExpr := _f.mem.NormExpr(left).AsNull() if _nullExpr != nil { if !_f.allowNullArgs(opt.RShiftOp, left, right) { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.FoldNullBinaryLeft) { + if _f.matchedRule == nil || _f.matchedRule(opt.FoldNullBinaryLeft) { _group = _f.foldNullBinary(opt.RShiftOp, left, right) _f.mem.AddAltFingerprint(_rShiftExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.FoldNullBinaryLeft, _group, 0) + } return _group } } @@ -4967,9 +5561,12 @@ func (_f *Factory) ConstructRShift( _nullExpr := _f.mem.NormExpr(right).AsNull() if _nullExpr != nil { if !_f.allowNullArgs(opt.RShiftOp, left, right) { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.FoldNullBinaryRight) { + if _f.matchedRule == nil || _f.matchedRule(opt.FoldNullBinaryRight) { _group = _f.foldNullBinary(opt.RShiftOp, left, right) _f.mem.AddAltFingerprint(_rShiftExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.FoldNullBinaryRight, _group, 0) + } return _group } } @@ -4997,9 +5594,12 @@ func (_f *Factory) ConstructFetchVal( if _nullExpr != nil { right := index if !_f.allowNullArgs(opt.FetchValOp, left, right) { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.FoldNullBinaryLeft) { + if _f.matchedRule == nil || _f.matchedRule(opt.FoldNullBinaryLeft) { _group = _f.foldNullBinary(opt.FetchValOp, left, right) _f.mem.AddAltFingerprint(_fetchValExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.FoldNullBinaryLeft, _group, 0) + } return _group } } @@ -5013,9 +5613,12 @@ func (_f *Factory) ConstructFetchVal( _nullExpr := _f.mem.NormExpr(index).AsNull() if _nullExpr != nil { if !_f.allowNullArgs(opt.FetchValOp, left, right) { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.FoldNullBinaryRight) { + if _f.matchedRule == nil || _f.matchedRule(opt.FoldNullBinaryRight) { _group = _f.foldNullBinary(opt.FetchValOp, left, right) _f.mem.AddAltFingerprint(_fetchValExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.FoldNullBinaryRight, _group, 0) + } return _group } } @@ -5043,9 +5646,12 @@ func (_f *Factory) ConstructFetchText( if _nullExpr != nil { right := index if !_f.allowNullArgs(opt.FetchTextOp, left, right) { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.FoldNullBinaryLeft) { + if _f.matchedRule == nil || _f.matchedRule(opt.FoldNullBinaryLeft) { _group = _f.foldNullBinary(opt.FetchTextOp, left, right) _f.mem.AddAltFingerprint(_fetchTextExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.FoldNullBinaryLeft, _group, 0) + } return _group } } @@ -5059,9 +5665,12 @@ func (_f *Factory) ConstructFetchText( _nullExpr := _f.mem.NormExpr(index).AsNull() if _nullExpr != nil { if !_f.allowNullArgs(opt.FetchTextOp, left, right) { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.FoldNullBinaryRight) { + if _f.matchedRule == nil || _f.matchedRule(opt.FoldNullBinaryRight) { _group = _f.foldNullBinary(opt.FetchTextOp, left, right) _f.mem.AddAltFingerprint(_fetchTextExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.FoldNullBinaryRight, _group, 0) + } return _group } } @@ -5089,9 +5698,12 @@ func (_f *Factory) ConstructFetchValPath( if _nullExpr != nil { right := path if !_f.allowNullArgs(opt.FetchValPathOp, left, right) { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.FoldNullBinaryLeft) { + if _f.matchedRule == nil || _f.matchedRule(opt.FoldNullBinaryLeft) { _group = _f.foldNullBinary(opt.FetchValPathOp, left, right) _f.mem.AddAltFingerprint(_fetchValPathExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.FoldNullBinaryLeft, _group, 0) + } return _group } } @@ -5105,9 +5717,12 @@ func (_f *Factory) ConstructFetchValPath( _nullExpr := _f.mem.NormExpr(path).AsNull() if _nullExpr != nil { if !_f.allowNullArgs(opt.FetchValPathOp, left, right) { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.FoldNullBinaryRight) { + if _f.matchedRule == nil || _f.matchedRule(opt.FoldNullBinaryRight) { _group = _f.foldNullBinary(opt.FetchValPathOp, left, right) _f.mem.AddAltFingerprint(_fetchValPathExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.FoldNullBinaryRight, _group, 0) + } return _group } } @@ -5135,9 +5750,12 @@ func (_f *Factory) ConstructFetchTextPath( if _nullExpr != nil { right := path if !_f.allowNullArgs(opt.FetchTextPathOp, left, right) { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.FoldNullBinaryLeft) { + if _f.matchedRule == nil || _f.matchedRule(opt.FoldNullBinaryLeft) { _group = _f.foldNullBinary(opt.FetchTextPathOp, left, right) _f.mem.AddAltFingerprint(_fetchTextPathExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.FoldNullBinaryLeft, _group, 0) + } return _group } } @@ -5151,9 +5769,12 @@ func (_f *Factory) ConstructFetchTextPath( _nullExpr := _f.mem.NormExpr(path).AsNull() if _nullExpr != nil { if !_f.allowNullArgs(opt.FetchTextPathOp, left, right) { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.FoldNullBinaryRight) { + if _f.matchedRule == nil || _f.matchedRule(opt.FoldNullBinaryRight) { _group = _f.foldNullBinary(opt.FetchTextPathOp, left, right) _f.mem.AddAltFingerprint(_fetchTextPathExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.FoldNullBinaryRight, _group, 0) + } return _group } } @@ -5180,12 +5801,15 @@ func (_f *Factory) ConstructUnaryMinus( left := _minusExpr.Left() right := _minusExpr.Right() if _f.canConstructBinary(opt.MinusOp, right, left) { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.InvertMinus) { + if _f.matchedRule == nil || _f.matchedRule(opt.InvertMinus) { _group = _f.ConstructMinus( right, left, ) _f.mem.AddAltFingerprint(_unaryMinusExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.InvertMinus, _group, 0) + } return _group } } @@ -5197,9 +5821,12 @@ func (_f *Factory) ConstructUnaryMinus( _unaryMinusExpr2 := _f.mem.NormExpr(input).AsUnaryMinus() if _unaryMinusExpr2 != nil { input := _unaryMinusExpr2.Input() - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.EliminateUnaryMinus) { + if _f.matchedRule == nil || _f.matchedRule(opt.EliminateUnaryMinus) { _group = input _f.mem.AddAltFingerprint(_unaryMinusExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.EliminateUnaryMinus, _group, 0) + } return _group } } @@ -5209,9 +5836,12 @@ func (_f *Factory) ConstructUnaryMinus( { _nullExpr := _f.mem.NormExpr(input).AsNull() if _nullExpr != nil { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.FoldNullUnary) { + if _f.matchedRule == nil || _f.matchedRule(opt.FoldNullUnary) { _group = _f.foldNullUnary(opt.UnaryMinusOp, input) _f.mem.AddAltFingerprint(_unaryMinusExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.FoldNullUnary, _group, 0) + } return _group } } @@ -5234,9 +5864,12 @@ func (_f *Factory) ConstructUnaryComplement( { _nullExpr := _f.mem.NormExpr(input).AsNull() if _nullExpr != nil { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.FoldNullUnary) { + if _f.matchedRule == nil || _f.matchedRule(opt.FoldNullUnary) { _group = _f.foldNullUnary(opt.UnaryComplementOp, input) _f.mem.AddAltFingerprint(_unaryComplementExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.FoldNullUnary, _group, 0) + } return _group } } @@ -5259,9 +5892,12 @@ func (_f *Factory) ConstructCast( // [EliminateCast] { if _f.hasType(input, typ) { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.EliminateCast) { + if _f.matchedRule == nil || _f.matchedRule(opt.EliminateCast) { _group = input _f.mem.AddAltFingerprint(_castExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.EliminateCast, _group, 0) + } return _group } } @@ -5271,11 +5907,14 @@ func (_f *Factory) ConstructCast( { _nullExpr := _f.mem.NormExpr(input).AsNull() if _nullExpr != nil { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.FoldNullCast) { + if _f.matchedRule == nil || _f.matchedRule(opt.FoldNullCast) { _group = _f.ConstructNull( typ, ) _f.mem.AddAltFingerprint(_castExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.FoldNullCast, _group, 0) + } return _group } } @@ -5378,9 +6017,12 @@ func (_f *Factory) ConstructCoalesce( if args.Length == 1 { _item := _f.mem.LookupList(args)[0] item := _item - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.EliminateCoalesce) { + if _f.matchedRule == nil || _f.matchedRule(opt.EliminateCoalesce) { _group = item _f.mem.AddAltFingerprint(_coalesceExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.EliminateCoalesce, _group, 0) + } return _group } } @@ -5392,9 +6034,12 @@ func (_f *Factory) ConstructCoalesce( _item := _f.mem.LookupList(args)[0] _expr := _f.mem.NormExpr(_item) if _expr.IsConstValue() { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.SimplifyCoalesce) { + if _f.matchedRule == nil || _f.matchedRule(opt.SimplifyCoalesce) { _group = _f.simplifyCoalesce(args) _f.mem.AddAltFingerprint(_coalesceExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.SimplifyCoalesce, _group, 0) + } return _group } } diff --git a/pkg/sql/opt/norm/testdata/combo b/pkg/sql/opt/norm/testdata/combo index ea0c904b2224..c568772b029a 100644 --- a/pkg/sql/opt/norm/testdata/combo +++ b/pkg/sql/opt/norm/testdata/combo @@ -1,5 +1,12 @@ exec-ddl -CREATE TABLE a (x INT PRIMARY KEY, i INT, f FLOAT, s STRING, j JSON) +CREATE TABLE a ( + x INT PRIMARY KEY, + i INT, + f FLOAT, + s STRING, + j JSON, + UNIQUE INDEX (s DESC, f) STORING (j) +) ---- TABLE a ├── x int not null @@ -7,8 +14,13 @@ TABLE a ├── f float ├── s string ├── j jsonb - └── INDEX primary - └── x int not null + ├── INDEX primary + │ └── x int not null + └── INDEX secondary + ├── s string desc + ├── f float + ├── x int not null (storing) + └── j jsonb (storing) exec-ddl CREATE TABLE t.b (x INT PRIMARY KEY, z INT) @@ -25,15 +37,17 @@ TABLE b optsteps SELECT s FROM a INNER JOIN b ON a.x=b.x AND i+1=10 ---- ----- -*** Initial expr: +================================================================================ +Initial expression + Cost: 2000.00 +================================================================================ project ├── columns: s:4(string) ├── inner-join │ ├── columns: a.x:1(int!null) a.i:2(int) a.f:3(float) a.s:4(string) a.j:5(jsonb) b.x:6(int!null) b.z:7(int) │ ├── scan a │ │ ├── columns: a.x:1(int!null) a.i:2(int) a.f:3(float) a.s:4(string) a.j:5(jsonb) - │ │ └── keys: (1) + │ │ └── keys: (1) weak(3,4) │ ├── scan b │ │ ├── columns: b.x:6(int!null) b.z:7(int) │ │ └── keys: (6) @@ -48,15 +62,17 @@ SELECT s FROM a INNER JOIN b ON a.x=b.x AND i+1=10 │ └── const: 10 [type=int] └── projections [outer=(4)] └── variable: a.s [type=string, outer=(4)] - -*** NormalizeCmpPlusConst applied; best expr changed: +================================================================================ +NormalizeCmpPlusConst + Cost: 2000.00 +================================================================================ project ├── columns: s:4(string) ├── inner-join │ ├── columns: a.x:1(int!null) a.i:2(int) a.f:3(float) a.s:4(string) a.j:5(jsonb) b.x:6(int!null) b.z:7(int) │ ├── scan a │ │ ├── columns: a.x:1(int!null) a.i:2(int) a.f:3(float) a.s:4(string) a.j:5(jsonb) - │ │ └── keys: (1) + │ │ └── keys: (1) weak(3,4) │ ├── scan b │ │ ├── columns: b.x:6(int!null) b.z:7(int) │ │ └── keys: (6) @@ -75,15 +91,17 @@ SELECT s FROM a INNER JOIN b ON a.x=b.x AND i+1=10 + │ └── const: 1 [type=int] └── projections [outer=(4)] └── variable: a.s [type=string, outer=(4)] - -*** EnsureJoinFiltersAnd applied; best expr changed: +================================================================================ +EnsureJoinFiltersAnd + Cost: 2000.00 +================================================================================ project ├── columns: s:4(string) ├── inner-join │ ├── columns: a.x:1(int!null) a.i:2(int) a.f:3(float) a.s:4(string) a.j:5(jsonb) b.x:6(int!null) b.z:7(int) │ ├── scan a │ │ ├── columns: a.x:1(int!null) a.i:2(int) a.f:3(float) a.s:4(string) a.j:5(jsonb) - │ │ └── keys: (1) + │ │ └── keys: (1) weak(3,4) │ ├── scan b │ │ ├── columns: b.x:6(int!null) b.z:7(int) │ │ └── keys: (6) @@ -99,8 +117,10 @@ SELECT s FROM a INNER JOIN b ON a.x=b.x AND i+1=10 │ └── const: 1 [type=int] └── projections [outer=(4)] └── variable: a.s [type=string, outer=(4)] - -*** PushFilterIntoJoinLeft applied; best expr changed: +================================================================================ +PushFilterIntoJoinLeft + Cost: 2100.00 +================================================================================ project ├── columns: s:4(string) ├── inner-join @@ -108,11 +128,11 @@ SELECT s FROM a INNER JOIN b ON a.x=b.x AND i+1=10 - │ ├── scan a + │ ├── select │ │ ├── columns: a.x:1(int!null) a.i:2(int) a.f:3(float) a.s:4(string) a.j:5(jsonb) - - │ │ └── keys: (1) - + │ │ ├── keys: (1) + - │ │ └── keys: (1) weak(3,4) + + │ │ ├── keys: (1) weak(3,4) + │ │ ├── scan a + │ │ │ ├── columns: a.x:1(int!null) a.i:2(int) a.f:3(float) a.s:4(string) a.j:5(jsonb) - + │ │ │ └── keys: (1) + + │ │ │ └── keys: (1) weak(3,4) + │ │ └── filters [type=bool, outer=(2)] + │ │ └── eq [type=bool, outer=(2)] + │ │ ├── variable: a.i [type=int, outer=(2)] @@ -137,32 +157,35 @@ SELECT s FROM a INNER JOIN b ON a.x=b.x AND i+1=10 + │ └── variable: b.x [type=int, outer=(6)] └── projections [outer=(4)] └── variable: a.s [type=string, outer=(4)] - -*** FilterUnusedJoinLeftCols applied; best expr changed: +================================================================================ +FilterUnusedJoinLeftCols + Cost: 2100.00 +================================================================================ project ├── columns: s:4(string) ├── inner-join - │ ├── columns: a.x:1(int!null) a.i:2(int) a.f:3(float) a.s:4(string) a.j:5(jsonb) b.x:6(int!null) b.z:7(int) - │ ├── select - │ │ ├── columns: a.x:1(int!null) a.i:2(int) a.f:3(float) a.s:4(string) a.j:5(jsonb) + - │ │ ├── keys: (1) weak(3,4) + - │ │ ├── scan a + │ ├── columns: a.x:1(int!null) a.s:4(string) b.x:6(int!null) b.z:7(int) + │ ├── project + │ │ ├── columns: a.x:1(int!null) a.s:4(string) - │ │ ├── keys: (1) - - │ │ ├── scan a + + │ │ ├── keys: (1) + │ │ ├── select │ │ │ ├── columns: a.x:1(int!null) a.i:2(int) a.f:3(float) a.s:4(string) a.j:5(jsonb) - - │ │ │ └── keys: (1) + - │ │ │ └── keys: (1) weak(3,4) - │ │ └── filters [type=bool, outer=(2)] - │ │ └── eq [type=bool, outer=(2)] - │ │ ├── variable: a.i [type=int, outer=(2)] - │ │ └── minus [type=int] - │ │ ├── const: 10 [type=int] - │ │ └── const: 1 [type=int] - + │ │ │ ├── keys: (1) + + │ │ │ ├── keys: (1) weak(3,4) + │ │ │ ├── scan a + │ │ │ │ ├── columns: a.x:1(int!null) a.i:2(int) a.f:3(float) a.s:4(string) a.j:5(jsonb) - + │ │ │ │ └── keys: (1) + + │ │ │ │ └── keys: (1) weak(3,4) + │ │ │ └── filters [type=bool, outer=(2)] + │ │ │ └── eq [type=bool, outer=(2)] + │ │ │ ├── variable: a.i [type=int, outer=(2)] @@ -181,8 +204,10 @@ SELECT s FROM a INNER JOIN b ON a.x=b.x AND i+1=10 │ └── variable: b.x [type=int, outer=(6)] └── projections [outer=(4)] └── variable: a.s [type=string, outer=(4)] - -*** FilterUnusedSelectCols applied; best expr changed: +================================================================================ +FilterUnusedSelectCols + Cost: 2100.00 +================================================================================ project ├── columns: s:4(string) ├── inner-join @@ -192,12 +217,14 @@ SELECT s FROM a INNER JOIN b ON a.x=b.x AND i+1=10 │ │ ├── keys: (1) │ │ ├── select - │ │ │ ├── columns: a.x:1(int!null) a.i:2(int) a.f:3(float) a.s:4(string) a.j:5(jsonb) + - │ │ │ ├── keys: (1) weak(3,4) + │ │ │ ├── columns: a.x:1(int!null) a.i:2(int) a.s:4(string) - │ │ │ ├── keys: (1) + + │ │ │ ├── keys: (1) │ │ │ ├── scan a - │ │ │ │ ├── columns: a.x:1(int!null) a.i:2(int) a.f:3(float) a.s:4(string) a.j:5(jsonb) + - │ │ │ │ └── keys: (1) weak(3,4) + │ │ │ │ ├── columns: a.x:1(int!null) a.i:2(int) a.s:4(string) - │ │ │ │ └── keys: (1) + + │ │ │ │ └── keys: (1) │ │ │ └── filters [type=bool, outer=(2)] │ │ │ └── eq [type=bool, outer=(2)] │ │ │ ├── variable: a.i [type=int, outer=(2)] @@ -216,8 +243,10 @@ SELECT s FROM a INNER JOIN b ON a.x=b.x AND i+1=10 │ └── variable: b.x [type=int, outer=(6)] └── projections [outer=(4)] └── variable: a.s [type=string, outer=(4)] - -*** FilterUnusedJoinRightCols applied; best expr changed: +================================================================================ +FilterUnusedJoinRightCols + Cost: 2100.00 +================================================================================ project ├── columns: s:4(string) ├── inner-join @@ -251,14 +280,19 @@ SELECT s FROM a INNER JOIN b ON a.x=b.x AND i+1=10 │ └── variable: b.x [type=int, outer=(6)] └── projections [outer=(4)] └── variable: a.s [type=string, outer=(4)] - -*** GenerateIndexScans applied; best expr unchanged. - -*** ConstrainScan applied; best expr unchanged. - -*** GenerateIndexScans applied; best expr unchanged. - -*** Final best expr: +-------------------------------------------------------------------------------- +GenerateIndexScans (no changes) +-------------------------------------------------------------------------------- +-------------------------------------------------------------------------------- +ConstrainScan (no changes) +-------------------------------------------------------------------------------- +-------------------------------------------------------------------------------- +GenerateIndexScans (no changes) +-------------------------------------------------------------------------------- +================================================================================ +Final best expression + Cost: 2100.00 +================================================================================ project ├── columns: s:4(string) ├── inner-join @@ -290,15 +324,16 @@ SELECT s FROM a INNER JOIN b ON a.x=b.x AND i+1=10 │ └── variable: b.x [type=int, outer=(6)] └── projections [outer=(4)] └── variable: a.s [type=string, outer=(4)] ----- ----- + # Select/Project/Limit/Offset rules have cyclical dependencies. optsteps SELECT i, i+1 FROM a FULL JOIN b ON a.x=b.x WHERE i=10 ORDER BY i LIMIT 5 OFFSET 1 ---- ----- -*** Initial expr: +================================================================================ +Initial expression + Cost: 14500.00 +================================================================================ limit ├── columns: i:2(int) column8:8(int) ├── ordering: +2 @@ -316,7 +351,7 @@ SELECT i, i+1 FROM a FULL JOIN b ON a.x=b.x WHERE i=10 ORDER BY i LIMIT 5 OFFSET │ │ │ │ ├── columns: a.x:1(int) a.i:2(int) a.f:3(float) a.s:4(string) a.j:5(jsonb) b.x:6(int) b.z:7(int) │ │ │ │ ├── scan a │ │ │ │ │ ├── columns: a.x:1(int!null) a.i:2(int) a.f:3(float) a.s:4(string) a.j:5(jsonb) - │ │ │ │ │ └── keys: (1) + │ │ │ │ │ └── keys: (1) weak(3,4) │ │ │ │ ├── scan b │ │ │ │ │ ├── columns: b.x:6(int!null) b.z:7(int) │ │ │ │ │ └── keys: (6) @@ -333,8 +368,10 @@ SELECT i, i+1 FROM a FULL JOIN b ON a.x=b.x WHERE i=10 ORDER BY i LIMIT 5 OFFSET │ │ └── const: 1 [type=int] │ └── const: 1 [type=int] └── const: 5 [type=int] - -*** EnsureJoinFilters applied; best expr changed: +================================================================================ +EnsureJoinFilters + Cost: 14500.00 +================================================================================ limit ├── columns: i:2(int) column8:8(int) ├── ordering: +2 @@ -352,7 +389,7 @@ SELECT i, i+1 FROM a FULL JOIN b ON a.x=b.x WHERE i=10 ORDER BY i LIMIT 5 OFFSET │ │ │ │ ├── columns: a.x:1(int) a.i:2(int) a.f:3(float) a.s:4(string) a.j:5(jsonb) b.x:6(int) b.z:7(int) │ │ │ │ ├── scan a │ │ │ │ │ ├── columns: a.x:1(int!null) a.i:2(int) a.f:3(float) a.s:4(string) a.j:5(jsonb) - │ │ │ │ │ └── keys: (1) + │ │ │ │ │ └── keys: (1) weak(3,4) │ │ │ │ ├── scan b │ │ │ │ │ ├── columns: b.x:6(int!null) b.z:7(int) │ │ │ │ │ └── keys: (6) @@ -373,8 +410,10 @@ SELECT i, i+1 FROM a FULL JOIN b ON a.x=b.x WHERE i=10 ORDER BY i LIMIT 5 OFFSET │ │ └── const: 1 [type=int] │ └── const: 1 [type=int] └── const: 5 [type=int] - -*** EnsureSelectFilters applied; best expr changed: +================================================================================ +EnsureSelectFilters + Cost: 14500.00 +================================================================================ limit ├── columns: i:2(int) column8:8(int) ├── ordering: +2 @@ -392,7 +431,7 @@ SELECT i, i+1 FROM a FULL JOIN b ON a.x=b.x WHERE i=10 ORDER BY i LIMIT 5 OFFSET │ │ │ │ ├── columns: a.x:1(int) a.i:2(int) a.f:3(float) a.s:4(string) a.j:5(jsonb) b.x:6(int) b.z:7(int) │ │ │ │ ├── scan a │ │ │ │ │ ├── columns: a.x:1(int!null) a.i:2(int) a.f:3(float) a.s:4(string) a.j:5(jsonb) - │ │ │ │ │ └── keys: (1) + │ │ │ │ │ └── keys: (1) weak(3,4) │ │ │ │ ├── scan b │ │ │ │ │ ├── columns: b.x:6(int!null) b.z:7(int) │ │ │ │ │ └── keys: (6) @@ -414,8 +453,10 @@ SELECT i, i+1 FROM a FULL JOIN b ON a.x=b.x WHERE i=10 ORDER BY i LIMIT 5 OFFSET │ │ └── const: 1 [type=int] │ └── const: 1 [type=int] └── const: 5 [type=int] - -*** FilterUnusedSelectCols applied; best expr changed: +================================================================================ +FilterUnusedSelectCols + Cost: 14500.00 +================================================================================ limit ├── columns: i:2(int) column8:8(int) ├── ordering: +2 @@ -433,7 +474,7 @@ SELECT i, i+1 FROM a FULL JOIN b ON a.x=b.x WHERE i=10 ORDER BY i LIMIT 5 OFFSET - │ │ │ │ ├── columns: a.x:1(int) a.i:2(int) a.f:3(float) a.s:4(string) a.j:5(jsonb) b.x:6(int) b.z:7(int) - │ │ │ │ ├── scan a - │ │ │ │ │ ├── columns: a.x:1(int!null) a.i:2(int) a.f:3(float) a.s:4(string) a.j:5(jsonb) - - │ │ │ │ │ └── keys: (1) + - │ │ │ │ │ └── keys: (1) weak(3,4) - │ │ │ │ ├── scan b - │ │ │ │ │ ├── columns: b.x:6(int!null) b.z:7(int) - │ │ │ │ │ └── keys: (6) @@ -448,7 +489,7 @@ SELECT i, i+1 FROM a FULL JOIN b ON a.x=b.x WHERE i=10 ORDER BY i LIMIT 5 OFFSET + │ │ │ │ │ ├── columns: a.x:1(int) a.i:2(int) a.f:3(float) a.s:4(string) a.j:5(jsonb) b.x:6(int) b.z:7(int) + │ │ │ │ │ ├── scan a + │ │ │ │ │ │ ├── columns: a.x:1(int!null) a.i:2(int) a.f:3(float) a.s:4(string) a.j:5(jsonb) - + │ │ │ │ │ │ └── keys: (1) + + │ │ │ │ │ │ └── keys: (1) weak(3,4) + │ │ │ │ │ ├── scan b + │ │ │ │ │ │ ├── columns: b.x:6(int!null) b.z:7(int) + │ │ │ │ │ │ └── keys: (6) @@ -469,8 +510,10 @@ SELECT i, i+1 FROM a FULL JOIN b ON a.x=b.x WHERE i=10 ORDER BY i LIMIT 5 OFFSET │ │ └── const: 1 [type=int] │ └── const: 1 [type=int] └── const: 5 [type=int] - -*** FilterUnusedJoinLeftCols applied; best expr changed: +================================================================================ +FilterUnusedJoinLeftCols + Cost: 14500.00 +================================================================================ limit ├── columns: i:2(int) column8:8(int) ├── ordering: +2 @@ -491,8 +534,9 @@ SELECT i, i+1 FROM a FULL JOIN b ON a.x=b.x WHERE i=10 ORDER BY i LIMIT 5 OFFSET + │ │ │ │ │ ├── columns: a.x:1(int) a.i:2(int) b.x:6(int) b.z:7(int) │ │ │ │ │ ├── scan a - │ │ │ │ │ │ ├── columns: a.x:1(int!null) a.i:2(int) a.f:3(float) a.s:4(string) a.j:5(jsonb) + - │ │ │ │ │ │ └── keys: (1) weak(3,4) + │ │ │ │ │ │ ├── columns: a.x:1(int!null) a.i:2(int) - │ │ │ │ │ │ └── keys: (1) + + │ │ │ │ │ │ └── keys: (1) │ │ │ │ │ ├── scan b │ │ │ │ │ │ ├── columns: b.x:6(int!null) b.z:7(int) │ │ │ │ │ │ └── keys: (6) @@ -513,8 +557,10 @@ SELECT i, i+1 FROM a FULL JOIN b ON a.x=b.x WHERE i=10 ORDER BY i LIMIT 5 OFFSET │ │ └── const: 1 [type=int] │ └── const: 1 [type=int] └── const: 5 [type=int] - -*** FilterUnusedJoinRightCols applied; best expr changed: +================================================================================ +FilterUnusedJoinRightCols + Cost: 14500.00 +================================================================================ limit ├── columns: i:2(int) column8:8(int) ├── ordering: +2 @@ -557,8 +603,10 @@ SELECT i, i+1 FROM a FULL JOIN b ON a.x=b.x WHERE i=10 ORDER BY i LIMIT 5 OFFSET │ │ └── const: 1 [type=int] │ └── const: 1 [type=int] └── const: 5 [type=int] - -*** PushSelectIntoProject applied; best expr changed: +================================================================================ +PushSelectIntoProject + Cost: 13250.00 +================================================================================ limit ├── columns: i:2(int) column8:8(int) ├── ordering: +2 @@ -617,8 +665,10 @@ SELECT i, i+1 FROM a FULL JOIN b ON a.x=b.x WHERE i=10 ORDER BY i LIMIT 5 OFFSET │ │ └── const: 1 [type=int] │ └── const: 1 [type=int] └── const: 5 [type=int] - -*** EliminateEmptyAnd applied; best expr changed: +================================================================================ +EliminateEmptyAnd + Cost: 13250.00 +================================================================================ limit ├── columns: i:2(int) column8:8(int) ├── ordering: +2 @@ -663,8 +713,10 @@ SELECT i, i+1 FROM a FULL JOIN b ON a.x=b.x WHERE i=10 ORDER BY i LIMIT 5 OFFSET │ │ └── const: 1 [type=int] │ └── const: 1 [type=int] └── const: 5 [type=int] - -*** EliminateSelect applied; best expr changed: +================================================================================ +EliminateSelect + Cost: 14500.00 +================================================================================ limit ├── columns: i:2(int) column8:8(int) ├── ordering: +2 @@ -728,8 +780,10 @@ SELECT i, i+1 FROM a FULL JOIN b ON a.x=b.x WHERE i=10 ORDER BY i LIMIT 5 OFFSET │ │ └── const: 1 [type=int] │ └── const: 1 [type=int] └── const: 5 [type=int] - -*** EliminateProjectProject applied; best expr changed: +================================================================================ +EliminateProjectProject + Cost: 14500.00 +================================================================================ limit ├── columns: i:2(int) column8:8(int) ├── ordering: +2 @@ -787,8 +841,10 @@ SELECT i, i+1 FROM a FULL JOIN b ON a.x=b.x WHERE i=10 ORDER BY i LIMIT 5 OFFSET │ │ └── const: 1 [type=int] │ └── const: 1 [type=int] └── const: 5 [type=int] - -*** FilterUnusedSelectCols applied; best expr changed: +================================================================================ +FilterUnusedSelectCols + Cost: 14500.00 +================================================================================ limit ├── columns: i:2(int) column8:8(int) ├── ordering: +2 @@ -846,8 +902,10 @@ SELECT i, i+1 FROM a FULL JOIN b ON a.x=b.x WHERE i=10 ORDER BY i LIMIT 5 OFFSET │ │ └── const: 1 [type=int] │ └── const: 1 [type=int] └── const: 5 [type=int] - -*** EliminateProjectProject applied; best expr changed: +================================================================================ +EliminateProjectProject + Cost: 14500.00 +================================================================================ limit ├── columns: i:2(int) column8:8(int) ├── ordering: +2 @@ -905,8 +963,10 @@ SELECT i, i+1 FROM a FULL JOIN b ON a.x=b.x WHERE i=10 ORDER BY i LIMIT 5 OFFSET │ │ └── const: 1 [type=int] │ └── const: 1 [type=int] └── const: 5 [type=int] - -*** PushOffsetIntoProject applied; best expr changed: +================================================================================ +PushOffsetIntoProject + Cost: 14500.00 +================================================================================ limit ├── columns: i:2(int) column8:8(int) ├── ordering: +2 @@ -973,8 +1033,10 @@ SELECT i, i+1 FROM a FULL JOIN b ON a.x=b.x WHERE i=10 ORDER BY i LIMIT 5 OFFSET + │ ├── variable: a.i [type=int, outer=(2)] + │ └── const: 1 [type=int] └── const: 5 [type=int] - -*** FilterUnusedOffsetCols applied; best expr changed: +================================================================================ +FilterUnusedOffsetCols + Cost: 14500.00 +================================================================================ limit ├── columns: i:2(int) column8:8(int) ├── ordering: +2 @@ -1035,8 +1097,10 @@ SELECT i, i+1 FROM a FULL JOIN b ON a.x=b.x WHERE i=10 ORDER BY i LIMIT 5 OFFSET │ ├── variable: a.i [type=int, outer=(2)] │ └── const: 1 [type=int] └── const: 5 [type=int] - -*** PushLimitIntoProject applied; best expr changed: +================================================================================ +PushLimitIntoProject + Cost: 14500.00 +================================================================================ -limit +project ├── columns: i:2(int) column8:8(int) @@ -1087,12 +1151,16 @@ SELECT i, i+1 FROM a FULL JOIN b ON a.x=b.x WHERE i=10 ORDER BY i LIMIT 5 OFFSET + └── plus [type=int, outer=(2)] + ├── variable: a.i [type=int, outer=(2)] + └── const: 1 [type=int] - -*** GenerateIndexScans applied; best expr unchanged. - -*** GenerateIndexScans applied; best expr unchanged. - -*** Final best expr: +-------------------------------------------------------------------------------- +GenerateIndexScans (no changes) +-------------------------------------------------------------------------------- +-------------------------------------------------------------------------------- +GenerateIndexScans (no changes) +-------------------------------------------------------------------------------- +================================================================================ +Final best expression + Cost: 14500.00 +================================================================================ project ├── columns: i:2(int) column8:8(int) ├── ordering: +2 @@ -1134,16 +1202,17 @@ SELECT i, i+1 FROM a FULL JOIN b ON a.x=b.x WHERE i=10 ORDER BY i LIMIT 5 OFFSET └── plus [type=int, outer=(2)] ├── variable: a.i [type=int, outer=(2)] └── const: 1 [type=int] ----- ----- + # Cyclical rules that trigger assert in AddAltFingerprint without extra code to # check whether nested rule has already called AddAltFingerprint. optsteps SELECT SUM(x) FROM a GROUP BY s, x HAVING SUM(x)=1 ---- ----- -*** Initial expr: +================================================================================ +Initial expression + Cost: 1010.00 +================================================================================ project ├── columns: column6:6(decimal) ├── select @@ -1158,7 +1227,7 @@ SELECT SUM(x) FROM a GROUP BY s, x HAVING SUM(x)=1 │ │ │ ├── keys: (1) │ │ │ ├── scan a │ │ │ │ ├── columns: a.x:1(int!null) a.i:2(int) a.f:3(float) a.s:4(string) a.j:5(jsonb) - │ │ │ │ └── keys: (1) + │ │ │ │ └── keys: (1) weak(3,4) │ │ │ └── projections [outer=(1,4)] │ │ │ ├── variable: a.s [type=string, outer=(4)] │ │ │ └── variable: a.x [type=int, outer=(1)] @@ -1170,8 +1239,10 @@ SELECT SUM(x) FROM a GROUP BY s, x HAVING SUM(x)=1 │ └── const: 1 [type=decimal] └── projections [outer=(6)] └── variable: column6 [type=decimal, outer=(6)] - -*** FilterUnusedScanCols applied; best expr changed: +================================================================================ +FilterUnusedScanCols + Cost: 1010.00 +================================================================================ project ├── columns: column6:6(decimal) ├── select @@ -1186,8 +1257,9 @@ SELECT SUM(x) FROM a GROUP BY s, x HAVING SUM(x)=1 │ │ │ ├── keys: (1) │ │ │ ├── scan a - │ │ │ │ ├── columns: a.x:1(int!null) a.i:2(int) a.f:3(float) a.s:4(string) a.j:5(jsonb) + - │ │ │ │ └── keys: (1) weak(3,4) + │ │ │ │ ├── columns: a.x:1(int!null) a.s:4(string) - │ │ │ │ └── keys: (1) + + │ │ │ │ └── keys: (1) │ │ │ └── projections [outer=(1,4)] │ │ │ ├── variable: a.s [type=string, outer=(4)] │ │ │ └── variable: a.x [type=int, outer=(1)] @@ -1199,8 +1271,10 @@ SELECT SUM(x) FROM a GROUP BY s, x HAVING SUM(x)=1 │ └── const: 1 [type=decimal] └── projections [outer=(6)] └── variable: column6 [type=decimal, outer=(6)] - -*** EliminateProject applied; best expr changed: +================================================================================ +EliminateProject + Cost: 1010.00 +================================================================================ project ├── columns: column6:6(decimal) ├── select @@ -1230,8 +1304,10 @@ SELECT SUM(x) FROM a GROUP BY s, x HAVING SUM(x)=1 │ └── const: 1 [type=decimal] └── projections [outer=(6)] └── variable: column6 [type=decimal, outer=(6)] - -*** EnsureSelectFilters applied; best expr changed: +================================================================================ +EnsureSelectFilters + Cost: 1010.00 +================================================================================ project ├── columns: column6:6(decimal) ├── select @@ -1256,8 +1332,10 @@ SELECT SUM(x) FROM a GROUP BY s, x HAVING SUM(x)=1 + │ └── const: 1 [type=decimal] └── projections [outer=(6)] └── variable: column6 [type=decimal, outer=(6)] - -*** FilterUnusedSelectCols applied; best expr changed: +================================================================================ +FilterUnusedSelectCols + Cost: 1010.00 +================================================================================ project ├── columns: column6:6(decimal) ├── select @@ -1294,8 +1372,10 @@ SELECT SUM(x) FROM a GROUP BY s, x HAVING SUM(x)=1 │ └── const: 1 [type=decimal] └── projections [outer=(6)] └── variable: column6 [type=decimal, outer=(6)] - -*** PushSelectIntoProject applied; best expr changed: +================================================================================ +PushSelectIntoProject + Cost: 1011.00 +================================================================================ project ├── columns: column6:6(decimal) ├── select @@ -1336,8 +1416,10 @@ SELECT SUM(x) FROM a GROUP BY s, x HAVING SUM(x)=1 + │ └── filters [type=bool] └── projections [outer=(6)] └── variable: column6 [type=decimal, outer=(6)] - -*** EliminateEmptyAnd applied; best expr changed: +================================================================================ +EliminateEmptyAnd + Cost: 1011.00 +================================================================================ project ├── columns: column6:6(decimal) ├── select @@ -1367,8 +1449,10 @@ SELECT SUM(x) FROM a GROUP BY s, x HAVING SUM(x)=1 + │ └── true [type=bool] └── projections [outer=(6)] └── variable: column6 [type=decimal, outer=(6)] - -*** EliminateSelect applied; best expr changed: +================================================================================ +EliminateSelect + Cost: 1010.00 +================================================================================ project ├── columns: column6:6(decimal) - ├── select @@ -1415,8 +1499,10 @@ SELECT SUM(x) FROM a GROUP BY s, x HAVING SUM(x)=1 + │ └── variable: column6 [type=decimal, outer=(6)] └── projections [outer=(6)] └── variable: column6 [type=decimal, outer=(6)] - -*** EliminateProject applied; best expr changed: +================================================================================ +EliminateProject + Cost: 1010.00 +================================================================================ project ├── columns: column6:6(decimal) - ├── project @@ -1457,10 +1543,35 @@ SELECT SUM(x) FROM a GROUP BY s, x HAVING SUM(x)=1 + │ └── const: 1 [type=decimal] └── projections [outer=(6)] └── variable: column6 [type=decimal, outer=(6)] - -*** GenerateIndexScans applied; best expr unchanged. - -*** Final best expr: +-------------------------------------------------------------------------------- +GenerateIndexScans (higher cost) +-------------------------------------------------------------------------------- + project + ├── columns: column6:6(decimal) + ├── select + │ ├── columns: a.x:1(int!null) a.s:4(string) column6:6(decimal) + │ ├── keys: (1) + │ ├── group-by + │ │ ├── columns: a.x:1(int!null) a.s:4(string) column6:6(decimal) + │ │ ├── grouping columns: a.x:1(int!null) a.s:4(string) + │ │ ├── keys: (1) + - │ │ ├── scan a + + │ │ ├── scan a@secondary + │ │ │ ├── columns: a.x:1(int!null) a.s:4(string) + │ │ │ └── keys: (1) + │ │ └── aggregations [outer=(1)] + │ │ └── function: sum [type=decimal, outer=(1)] + │ │ └── variable: a.x [type=int, outer=(1)] + │ └── filters [type=bool, outer=(6), constraints=(/6: [/1 - /1]; tight)] + │ └── eq [type=bool, outer=(6), constraints=(/6: [/1 - /1]; tight)] + │ ├── variable: column6 [type=decimal, outer=(6)] + │ └── const: 1 [type=decimal] + └── projections [outer=(6)] + └── variable: column6 [type=decimal, outer=(6)] +================================================================================ +Final best expression + Cost: 1010.00 +================================================================================ project ├── columns: column6:6(decimal) ├── select @@ -1482,5 +1593,190 @@ SELECT SUM(x) FROM a GROUP BY s, x HAVING SUM(x)=1 │ └── const: 1 [type=decimal] └── projections [outer=(6)] └── variable: column6 [type=decimal, outer=(6)] + +optsteps +SELECT s, x FROM a WHERE s='foo' AND f>100 ---- ----- +================================================================================ +Initial expression + Cost: 1100.00 +================================================================================ + project + ├── columns: s:4(string) x:1(int!null) + ├── keys: (1) + ├── select + │ ├── columns: a.x:1(int!null) a.i:2(int) a.f:3(float) a.s:4(string) a.j:5(jsonb) + │ ├── keys: (1) weak(3,4) + │ ├── scan a + │ │ ├── columns: a.x:1(int!null) a.i:2(int) a.f:3(float) a.s:4(string) a.j:5(jsonb) + │ │ └── keys: (1) weak(3,4) + │ └── and [type=bool, outer=(3,4), constraints=(/3: [/100.00000000000001 - ]; /4: [/'foo' - /'foo']; tight)] + │ ├── eq [type=bool, outer=(4), constraints=(/4: [/'foo' - /'foo']; tight)] + │ │ ├── variable: a.s [type=string, outer=(4)] + │ │ └── const: 'foo' [type=string] + │ └── gt [type=bool, outer=(3), constraints=(/3: [/100.00000000000001 - ]; tight)] + │ ├── variable: a.f [type=float, outer=(3)] + │ └── const: 100.0 [type=float] + └── projections [outer=(1,4)] + ├── variable: a.s [type=string, outer=(4)] + └── variable: a.x [type=int, outer=(1)] +================================================================================ +EnsureSelectFiltersAnd + Cost: 1100.00 +================================================================================ + project + ├── columns: s:4(string) x:1(int!null) + ├── keys: (1) + ├── select + │ ├── columns: a.x:1(int!null) a.i:2(int) a.f:3(float) a.s:4(string) a.j:5(jsonb) + │ ├── keys: (1) weak(3,4) + │ ├── scan a + │ │ ├── columns: a.x:1(int!null) a.i:2(int) a.f:3(float) a.s:4(string) a.j:5(jsonb) + │ │ └── keys: (1) weak(3,4) + - │ └── and [type=bool, outer=(3,4), constraints=(/3: [/100.00000000000001 - ]; /4: [/'foo' - /'foo']; tight)] + + │ └── filters [type=bool, outer=(3,4), constraints=(/3: [/100.00000000000001 - ]; /4: [/'foo' - /'foo']; tight)] + │ ├── eq [type=bool, outer=(4), constraints=(/4: [/'foo' - /'foo']; tight)] + │ │ ├── variable: a.s [type=string, outer=(4)] + │ │ └── const: 'foo' [type=string] + │ └── gt [type=bool, outer=(3), constraints=(/3: [/100.00000000000001 - ]; tight)] + │ ├── variable: a.f [type=float, outer=(3)] + │ └── const: 100.0 [type=float] + └── projections [outer=(1,4)] + ├── variable: a.s [type=string, outer=(4)] + └── variable: a.x [type=int, outer=(1)] +================================================================================ +FilterUnusedSelectCols + Cost: 1100.00 +================================================================================ + project + ├── columns: s:4(string) x:1(int!null) + ├── keys: (1) + ├── select + - │ ├── columns: a.x:1(int!null) a.i:2(int) a.f:3(float) a.s:4(string) a.j:5(jsonb) + + │ ├── columns: a.x:1(int!null) a.f:3(float) a.s:4(string) + │ ├── keys: (1) weak(3,4) + │ ├── scan a + - │ │ ├── columns: a.x:1(int!null) a.i:2(int) a.f:3(float) a.s:4(string) a.j:5(jsonb) + + │ │ ├── columns: a.x:1(int!null) a.f:3(float) a.s:4(string) + │ │ └── keys: (1) weak(3,4) + │ └── filters [type=bool, outer=(3,4), constraints=(/3: [/100.00000000000001 - ]; /4: [/'foo' - /'foo']; tight)] + │ ├── eq [type=bool, outer=(4), constraints=(/4: [/'foo' - /'foo']; tight)] + │ │ ├── variable: a.s [type=string, outer=(4)] + │ │ └── const: 'foo' [type=string] + │ └── gt [type=bool, outer=(3), constraints=(/3: [/100.00000000000001 - ]; tight)] + │ ├── variable: a.f [type=float, outer=(3)] + │ └── const: 100.0 [type=float] + └── projections [outer=(1,4)] + ├── variable: a.s [type=string, outer=(4)] + └── variable: a.x [type=int, outer=(1)] +-------------------------------------------------------------------------------- +GenerateIndexScans (higher cost) +-------------------------------------------------------------------------------- + project + ├── columns: s:4(string) x:1(int!null) + ├── keys: (1) + ├── select + │ ├── columns: a.x:1(int!null) a.f:3(float) a.s:4(string) + │ ├── keys: (1) weak(3,4) + - │ ├── scan a + + │ ├── scan a@secondary + │ │ ├── columns: a.x:1(int!null) a.f:3(float) a.s:4(string) + │ │ └── keys: (1) weak(3,4) + │ └── filters [type=bool, outer=(3,4), constraints=(/3: [/100.00000000000001 - ]; /4: [/'foo' - /'foo']; tight)] + │ ├── eq [type=bool, outer=(4), constraints=(/4: [/'foo' - /'foo']; tight)] + │ │ ├── variable: a.s [type=string, outer=(4)] + │ │ └── const: 'foo' [type=string] + │ └── gt [type=bool, outer=(3), constraints=(/3: [/100.00000000000001 - ]; tight)] + │ ├── variable: a.f [type=float, outer=(3)] + │ └── const: 100.0 [type=float] + └── projections [outer=(1,4)] + ├── variable: a.s [type=string, outer=(4)] + └── variable: a.x [type=int, outer=(1)] +-------------------------------------------------------------------------------- +ConstrainScan (higher cost) +-------------------------------------------------------------------------------- + project + ├── columns: s:4(string) x:1(int!null) + ├── keys: (1) + ├── select + │ ├── columns: a.x:1(int!null) a.f:3(float) a.s:4(string) + │ ├── keys: (1) weak(3,4) + - │ ├── scan a@secondary + + │ ├── scan a + │ │ ├── columns: a.x:1(int!null) a.f:3(float) a.s:4(string) + │ │ └── keys: (1) weak(3,4) + │ └── filters [type=bool, outer=(3,4), constraints=(/3: [/100.00000000000001 - ]; /4: [/'foo' - /'foo']; tight)] + │ ├── eq [type=bool, outer=(4), constraints=(/4: [/'foo' - /'foo']; tight)] + │ │ ├── variable: a.s [type=string, outer=(4)] + │ │ └── const: 'foo' [type=string] + │ └── gt [type=bool, outer=(3), constraints=(/3: [/100.00000000000001 - ]; tight)] + │ ├── variable: a.f [type=float, outer=(3)] + │ └── const: 100.0 [type=float] + └── projections [outer=(1,4)] + ├── variable: a.s [type=string, outer=(4)] + └── variable: a.x [type=int, outer=(1)] +================================================================================ +ConstrainScan + Cost: 110.00 +================================================================================ + project + ├── columns: s:4(string) x:1(int!null) + ├── keys: (1) + ├── select + │ ├── columns: a.x:1(int!null) a.f:3(float) a.s:4(string) + │ ├── keys: (1) weak(3,4) + - │ ├── scan a + + │ ├── scan a@secondary + │ │ ├── columns: a.x:1(int!null) a.f:3(float) a.s:4(string) + + │ │ ├── constraint: /-4/3: [/'foo'/100.00000000000001 - /'foo'] + │ │ └── keys: (1) weak(3,4) + - │ └── filters [type=bool, outer=(3,4), constraints=(/3: [/100.00000000000001 - ]; /4: [/'foo' - /'foo']; tight)] + - │ ├── eq [type=bool, outer=(4), constraints=(/4: [/'foo' - /'foo']; tight)] + - │ │ ├── variable: a.s [type=string, outer=(4)] + - │ │ └── const: 'foo' [type=string] + - │ └── gt [type=bool, outer=(3), constraints=(/3: [/100.00000000000001 - ]; tight)] + - │ ├── variable: a.f [type=float, outer=(3)] + - │ └── const: 100.0 [type=float] + + │ └── filters [type=bool] + + │ ├── true [type=bool] + + │ └── true [type=bool] + └── projections [outer=(1,4)] + ├── variable: a.s [type=string, outer=(4)] + └── variable: a.x [type=int, outer=(1)] +================================================================================ +SimplifyFilters + Cost: 100.00 +================================================================================ + project + ├── columns: s:4(string) x:1(int!null) + ├── keys: (1) + - ├── select + + ├── scan a@secondary + │ ├── columns: a.x:1(int!null) a.f:3(float) a.s:4(string) + - │ ├── keys: (1) weak(3,4) + - │ ├── scan a@secondary + - │ │ ├── columns: a.x:1(int!null) a.f:3(float) a.s:4(string) + - │ │ ├── constraint: /-4/3: [/'foo'/100.00000000000001 - /'foo'] + - │ │ └── keys: (1) weak(3,4) + - │ └── filters [type=bool] + - │ ├── true [type=bool] + - │ └── true [type=bool] + + │ ├── constraint: /-4/3: [/'foo'/100.00000000000001 - /'foo'] + + │ └── keys: (1) weak(3,4) + └── projections [outer=(1,4)] + ├── variable: a.s [type=string, outer=(4)] + └── variable: a.x [type=int, outer=(1)] +================================================================================ +Final best expression + Cost: 100.00 +================================================================================ + project + ├── columns: s:4(string) x:1(int!null) + ├── keys: (1) + ├── scan a@secondary + │ ├── columns: a.x:1(int!null) a.f:3(float) a.s:4(string) + │ ├── constraint: /-4/3: [/'foo'/100.00000000000001 - /'foo'] + │ └── keys: (1) weak(3,4) + └── projections [outer=(1,4)] + ├── variable: a.s [type=string, outer=(4)] + └── variable: a.x [type=int, outer=(1)] diff --git a/pkg/sql/opt/norm/testdata/comp b/pkg/sql/opt/norm/testdata/comp index a15fceb2ae2e..05321dac9b27 100644 --- a/pkg/sql/opt/norm/testdata/comp +++ b/pkg/sql/opt/norm/testdata/comp @@ -451,4 +451,3 @@ project └── projections ├── true [type=bool] └── false [type=bool] - diff --git a/pkg/sql/opt/optgen/cmd/optgen/rule_gen.go b/pkg/sql/opt/optgen/cmd/optgen/rule_gen.go index d6d11d22da98..70a6b0a2bbaa 100644 --- a/pkg/sql/opt/optgen/cmd/optgen/rule_gen.go +++ b/pkg/sql/opt/optgen/cmd/optgen/rule_gen.go @@ -571,7 +571,7 @@ func (g *ruleGen) genNormalizeReplace(define *lang.DefineExpr, rule *lang.RuleEx g.w.nest("if !_f.ruleCycles[%s.Fingerprint()] {\n", exprName) } - g.w.nestIndent("if _f.onRuleMatch == nil || _f.onRuleMatch(opt.%s) {\n", rule.Name) + g.w.nestIndent("if _f.matchedRule == nil || _f.matchedRule(opt.%s) {\n", rule.Name) if detectCycle { g.w.writeIndent("_f.ruleCycles[%s.Fingerprint()] = true\n", exprName) @@ -607,6 +607,11 @@ func (g *ruleGen) genNormalizeReplace(define *lang.DefineExpr, rule *lang.RuleEx g.w.writeIndent("_f.mem.AddAltFingerprint(%s.Fingerprint(), _group)\n", exprName) } + // Notify listeners that rule was applied. + g.w.nestIndent("if _f.appliedRule != nil {\n") + g.w.writeIndent("_f.appliedRule(opt.%s, _group, 0)\n", rule.Name) + g.w.unnest("}\n") + g.w.writeIndent("return _group\n") } @@ -616,7 +621,7 @@ func (g *ruleGen) genNormalizeReplace(define *lang.DefineExpr, rule *lang.RuleEx // raw MakeXXXExpr method, and passes it to Memo.MemoizeDenormExpr, which adds // the expression to an existing group. func (g *ruleGen) genExploreReplace(define *lang.DefineExpr, rule *lang.RuleExpr) { - g.w.nestIndent("if _e.o.onRuleMatch == nil || _e.o.onRuleMatch(opt.%s) {\n", rule.Name) + g.w.nestIndent("if _e.o.matchedRule == nil || _e.o.matchedRule(opt.%s) {\n", rule.Name) switch t := rule.Replace.(type) { case *lang.ConstructExpr: @@ -631,22 +636,30 @@ func (g *ruleGen) genExploreReplace(define *lang.DefineExpr, rule *lang.RuleExpr g.w.write(",\n") } g.w.unnest(")\n") + g.w.writeIndent("_before := _e.mem.ExprCount(_root.Group)\n") g.w.writeIndent("_e.mem.MemoizeDenormExpr(_root.Group, memo.Expr(_expr))\n") case *lang.CustomFuncExpr: // Top-level custom function returns a memo.Expr slice, so iterate // through that and memoize each expression. - g.w.writeIndent("exprs := ") + g.w.writeIndent("_exprs := ") g.genNestedExpr(rule.Replace) g.w.newline() - g.w.nestIndent("for i := range exprs {\n") - g.w.writeIndent("_e.mem.MemoizeDenormExpr(_root.Group, exprs[i])\n") + g.w.writeIndent("_before := _e.mem.ExprCount(_root.Group)\n") + g.w.nestIndent("for i := range _exprs {\n") + g.w.writeIndent("_e.mem.MemoizeDenormExpr(_root.Group, _exprs[i])\n") g.w.unnest("}\n") default: panic(fmt.Sprintf("unsupported replace expression in explore rule: %s", rule.Replace)) } + // Notify listeners that rule was applied. + g.w.nestIndent("if _e.o.appliedRule != nil {\n") + g.w.writeIndent("_after := _e.mem.ExprCount(_root.Group)\n") + g.w.writeIndent("_e.o.appliedRule(opt.%s, _root.Group, _after-_before)\n", rule.Name) + g.w.unnest("}\n") + g.w.unnest("}\n") } diff --git a/pkg/sql/opt/optgen/cmd/optgen/testdata/explorer b/pkg/sql/opt/optgen/cmd/optgen/testdata/explorer index 5ce577ef6e2e..c7f175e1ae23 100644 --- a/pkg/sql/opt/optgen/cmd/optgen/testdata/explorer +++ b/pkg/sql/opt/optgen/cmd/optgen/testdata/explorer @@ -130,12 +130,17 @@ func (_e *explorer) exploreInnerJoin(_rootState *exploreState, _root memo.ExprID if _root.Expr >= _rootState.start { r := _rootExpr.Left() s := _rootExpr.Right() - if _e.o.onRuleMatch == nil || _e.o.onRuleMatch(opt.CommuteJoin) { + if _e.o.matchedRule == nil || _e.o.matchedRule(opt.CommuteJoin) { _expr := memo.MakeInnerJoinExpr( s, r, ) + _before := _e.mem.ExprCount(_root.Group) _e.mem.MemoizeDenormExpr(_root.Group, memo.Expr(_expr)) + if _e.o.appliedRule != nil { + _after := _e.mem.ExprCount(_root.Group) + _e.o.appliedRule(opt.CommuteJoin, _root.Group, _after-_before) + } } } } @@ -169,7 +174,7 @@ func (_e *explorer) exploreInnerJoin(_rootState *exploreState, _root memo.ExprID upperConditions := _filtersExpr.Conditions() if _e.isCorrelated(filters, r) { if _e.isCorrelated(filters, t) { - if _e.o.onRuleMatch == nil || _e.o.onRuleMatch(opt.AssociateJoin) { + if _e.o.matchedRule == nil || _e.o.matchedRule(opt.AssociateJoin) { _expr := memo.MakeInnerJoinExpr( _e.f.ConstructInnerJoin( r, @@ -183,7 +188,12 @@ func (_e *explorer) exploreInnerJoin(_rootState *exploreState, _root memo.ExprID _e.constructConditionsUsing(s, lowerConditions, upperConditions), ), ) + _before := _e.mem.ExprCount(_root.Group) _e.mem.MemoizeDenormExpr(_root.Group, memo.Expr(_expr)) + if _e.o.appliedRule != nil { + _after := _e.mem.ExprCount(_root.Group) + _e.o.appliedRule(opt.AssociateJoin, _root.Group, _after-_before) + } } } } @@ -232,7 +242,7 @@ func (_e *explorer) exploreSelect(_rootState *exploreState, _root memo.ExprID) ( groupingCols := _groupByExpr.GroupingCols() filter := _rootExpr.Filter() if !_e.isCorrelated(filter, right) { - if _e.o.onRuleMatch == nil || _e.o.onRuleMatch(opt.PushDownGroupBy) { + if _e.o.matchedRule == nil || _e.o.matchedRule(opt.PushDownGroupBy) { _expr := memo.MakeInnerJoinExpr( _e.f.ConstructSelect( _e.f.ConstructGroupBy( @@ -245,7 +255,12 @@ func (_e *explorer) exploreSelect(_rootState *exploreState, _root memo.ExprID) ( right, on, ) + _before := _e.mem.ExprCount(_root.Group) _e.mem.MemoizeDenormExpr(_root.Group, memo.Expr(_expr)) + if _e.o.appliedRule != nil { + _after := _e.mem.ExprCount(_root.Group) + _e.o.appliedRule(opt.PushDownGroupBy, _root.Group, _after-_before) + } } } } @@ -266,10 +281,15 @@ func (_e *explorer) exploreScan(_rootState *exploreState, _root memo.ExprID) (_f if _root.Expr >= _rootState.start { def := _rootExpr.Def() if _e.isPrimaryScan(def) { - if _e.o.onRuleMatch == nil || _e.o.onRuleMatch(opt.GenerateIndexScans) { - exprs := _e.generateIndexScans(def) - for i := range exprs { - _e.mem.MemoizeDenormExpr(_root.Group, exprs[i]) + if _e.o.matchedRule == nil || _e.o.matchedRule(opt.GenerateIndexScans) { + _exprs := _e.generateIndexScans(def) + _before := _e.mem.ExprCount(_root.Group) + for i := range _exprs { + _e.mem.MemoizeDenormExpr(_root.Group, _exprs[i]) + } + if _e.o.appliedRule != nil { + _after := _e.mem.ExprCount(_root.Group) + _e.o.appliedRule(opt.GenerateIndexScans, _root.Group, _after-_before) } } } diff --git a/pkg/sql/opt/optgen/cmd/optgen/testdata/factory b/pkg/sql/opt/optgen/cmd/optgen/testdata/factory index b268fa699faf..116e5c645e6e 100644 --- a/pkg/sql/opt/optgen/cmd/optgen/testdata/factory +++ b/pkg/sql/opt/optgen/cmd/optgen/testdata/factory @@ -135,12 +135,15 @@ func (_f *Factory) ConstructInnerJoin( { r := left s := right - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.CommuteJoin) { + if _f.matchedRule == nil || _f.matchedRule(opt.CommuteJoin) { _group = _f.ConstructInnerJoin( s, r, ) _f.mem.AddAltFingerprint(_innerJoinExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.CommuteJoin, _group, 0) + } return _group } } @@ -257,12 +260,15 @@ func (_f *Factory) ConstructEq( _constExpr3 := _f.mem.NormExpr(right).AsConst() if _constExpr3 != nil { if !_f.isInvalidBinary(opt.MinusOp, right, leftRight) { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.NormalizeVarPlus) { + if _f.matchedRule == nil || _f.matchedRule(opt.NormalizeVarPlus) { _group = _f.ConstructEq( leftLeft, _f.constructBinary(opt.MinusOp, right, leftRight), ) _f.mem.AddAltFingerprint(_eqExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.NormalizeVarPlus, _group, 0) + } return _group } } @@ -299,12 +305,15 @@ func (_f *Factory) ConstructLt( _constExpr3 := _f.mem.NormExpr(right).AsConst() if _constExpr3 != nil { if !_f.isInvalidBinary(opt.MinusOp, right, leftRight) { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.NormalizeVarPlus) { + if _f.matchedRule == nil || _f.matchedRule(opt.NormalizeVarPlus) { _group = _f.ConstructLt( leftLeft, _f.constructBinary(opt.MinusOp, right, leftRight), ) _f.mem.AddAltFingerprint(_ltExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.NormalizeVarPlus, _group, 0) + } return _group } } @@ -467,12 +476,15 @@ func (_f *Factory) ConstructFunc( _variableExpr := _f.mem.NormExpr(_item).AsVariable() if _variableExpr == nil { if _f.isEmpty(item) { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.Concat) { + if _f.matchedRule == nil || _f.matchedRule(opt.Concat) { _group = _f.ConstructFunc( m.mem.InternPrivate(tree.NewDString("concat")), _f.removeListItem(args, item), ) _f.mem.AddAltFingerprint(_funcExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.Concat, _group, 0) + } return _group } } @@ -607,7 +619,7 @@ func (_f *Factory) ConstructSelect( if _expr2.IsJoin() { t := _expr2.ChildGroup(_f.mem, 0) u := _expr2.ChildGroup(_f.mem, 1) - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.Test) { + if _f.matchedRule == nil || _f.matchedRule(opt.Test) { _group = _f.DynamicConstruct( _f.mem.NormExpr(item).Operator(), memo.DynamicOperands{ @@ -619,6 +631,9 @@ func (_f *Factory) ConstructSelect( }, ) _f.mem.AddAltFingerprint(_selectExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.Test, _group, 0) + } return _group } } @@ -798,9 +813,12 @@ func (_f *Factory) ConstructList( if _listExpr5 != nil { empty := _listExpr5.Items() if _listExpr5.Items().Length == 0 { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.List) { + if _f.matchedRule == nil || _f.matchedRule(opt.List) { _group = _f.construct(any, first, last, single, empty) _f.mem.AddAltFingerprint(_listExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.List, _group, 0) + } return _group } } @@ -881,9 +899,12 @@ func (_f *Factory) ConstructJoin( // [ConstructList] { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.ConstructList) { + if _f.matchedRule == nil || _f.matchedRule(opt.ConstructList) { _group = _f.construct(_f.mem.InternList([]memo.GroupID{}), _f.mem.InternList([]memo.GroupID{left}), _f.mem.InternList([]memo.GroupID{left, right}), _f.mem.InternList([]memo.GroupID{_f.mem.InternList([]memo.GroupID{on})})) _f.mem.AddAltFingerprint(_joinExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.ConstructList, _group, 0) + } return _group } } @@ -978,12 +999,15 @@ func (_f *Factory) ConstructOp( _listExpr2 := _f.mem.NormExpr(single).AsList() if _listExpr2 != nil { if _listExpr2.Items().Length != 1 { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.ListNot) { + if _f.matchedRule == nil || _f.matchedRule(opt.ListNot) { _group = _f.ConstructOp( empty, single, ) _f.mem.AddAltFingerprint(_opExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.ListNot, _group, 0) + } return _group } } @@ -1098,9 +1122,12 @@ func (_f *Factory) ConstructEq( } if !_match { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.Constant) { + if _f.matchedRule == nil || _f.matchedRule(opt.Constant) { _group = m.mem.InternPrivate(tree.NewDString("foo")) _f.mem.AddAltFingerprint(_eqExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.Constant, _group, 0) + } return _group } } @@ -1144,9 +1171,12 @@ func (_f *Factory) ConstructNe( } if !_match { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.Dynamic) { + if _f.matchedRule == nil || _f.matchedRule(opt.Dynamic) { _group = m.mem.InternPrivate(tree.NewDString("foo")) _f.mem.AddAltFingerprint(_neExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.Dynamic, _group, 0) + } return _group } } @@ -1240,9 +1270,12 @@ func (_f *Factory) ConstructPlus( _nullExpr := _f.mem.NormExpr(right).AsNull() if _nullExpr != nil { if !_f.hasNullableArgs(opt.PlusOp, _f.anotherFunc(_f.mem.NormExpr(left).Operator())) { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.Fold) { + if _f.matchedRule == nil || _f.matchedRule(opt.Fold) { _group = _f.ConstructNull() _f.mem.AddAltFingerprint(_plusExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.Fold, _group, 0) + } return _group } } @@ -1268,9 +1301,12 @@ func (_f *Factory) ConstructMinus( _nullExpr := _f.mem.NormExpr(right).AsNull() if _nullExpr != nil { if !_f.hasNullableArgs(opt.MinusOp, _f.anotherFunc(_f.mem.NormExpr(left).Operator())) { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.Fold) { + if _f.matchedRule == nil || _f.matchedRule(opt.Fold) { _group = _f.ConstructNull() _f.mem.AddAltFingerprint(_minusExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.Fold, _group, 0) + } return _group } } @@ -1428,9 +1464,12 @@ func (_f *Factory) ConstructNot( _containsExpr := _f.mem.NormExpr(input).AsContains() if _containsExpr == nil { if _f.someOtherCondition(input) { - if _f.onRuleMatch == nil || _f.onRuleMatch(opt.Invert) { + if _f.matchedRule == nil || _f.matchedRule(opt.Invert) { _group = _f.invert(_f.mem.NormExpr(input).Operator(), left, right) _f.mem.AddAltFingerprint(_notExpr.Fingerprint(), _group) + if _f.appliedRule != nil { + _f.appliedRule(opt.Invert, _group, 0) + } return _group } } diff --git a/pkg/sql/opt/testutils/opt_steps.go b/pkg/sql/opt/testutils/opt_steps.go new file mode 100644 index 000000000000..35b73299c4e8 --- /dev/null +++ b/pkg/sql/opt/testutils/opt_steps.go @@ -0,0 +1,283 @@ +// Copyright 2018 The Cockroach Authors. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or +// implied. See the License for the specific language governing +// permissions and limitations under the License. + +package testutils + +import ( + "github.com/cockroachdb/cockroach/pkg/sql/opt" + "github.com/cockroachdb/cockroach/pkg/sql/opt/memo" + "github.com/cockroachdb/cockroach/pkg/sql/opt/xform" +) + +// optSteps implements the stepping algorithm used by the OptTester's OptSteps +// command. See the OptTester.OptSteps comment for more details on the command. +// +// The algorithm works as follows: +// 1. The first time optSteps.next() is called, optSteps returns the starting +// expression tree, with no transformations applied to it. +// +// 2. Each optSteps.next() call after that will perform N+1 transformations, +// where N is the the number of steps performed during the previous call +// (starting at 0 with the first call). +// +// 3. Each optSteps.next() call will build the expression tree from scratch +// and re-run all transformations that were run in the previous call, plus +// one additional transformation (N+1). Therefore, the output expression +// tree from each call will differ from the previous call only by the last +// transformation's changes. +// +// 4. optSteps hooks the optimizer's MatchedRule event in order to limit the +// number of transformations that can be applied, as well as to record the +// name of the last rule that was applied, for later output. +// +// 5. While this works well for normalization rules, exploration rules are +// more difficult. This is because exploration rules are not guaranteed to +// produce a lower cost tree. Unless extra measures are taken, the returned +// ExprView would not include the changed portion of the Memo, since +// ExprView only shows the lowest cost path through the Memo. +// +// 6. To address this issue, optSteps hooks the optimizer's AppliedRule event +// and records the expression(s) that the last transformation has affected. +// It then re-runs the optimizer, but this time using a special Coster +// implementation that fools the optimizer into thinking that the new +// expression(s) have the lowest cost. The coster does this by assigning an +// infinite cost to all other expressions in the same group as the new +// expression(s), as well as in all ancestor groups. +// +type optSteps struct { + tester *OptTester + + // steps is the maximum number of rules that can be applied by the optimizer + // during the current iteration. + steps int + + // remaining is the number of "unused" steps remaining in this iteration. + remaining int + + // lastMatched records the name of the rule that was most recently matched + // by the optimizer. + lastMatched opt.RuleName + + // lastApplied records the id of the expression that marks the portion of the + // tree affected by the most recent rule application. All expressions in the + // same memo group that are < lastApplied.Expr will assigned an infinite cost + // by the forcingCoster. Therefore, only expressions >= lastApplied.Expr can + // be in the output expression tree. + lastApplied memo.ExprID + + // ev is the expression tree produced by the most recent optSteps iteration. + ev memo.ExprView + + // better is true if ev is lower cost than the expression tree produced by + // the previous iteration of optSteps. + better bool + + // best is the textual representation of the most recent expression tree that + // was an improvement over the previous best tree. + best string +} + +func newOptSteps(tester *OptTester) *optSteps { + return &optSteps{tester: tester} +} + +// exprView returns the expression tree produced by the most recent optSteps +// iteration. +func (os *optSteps) exprView() memo.ExprView { + return os.ev +} + +// lastRuleName returns the name of the rule that was most recently matched by +// the optimizer. +func (os *optSteps) lastRuleName() opt.RuleName { + return os.lastMatched +} + +// isBetter returns true if exprView is lower cost than the expression tree +// produced by the previous iteration of optSteps. +func (os *optSteps) isBetter() bool { + return os.better +} + +// done returns true if there are no more rules to apply. Further calls to the +// next method will result in a panic. +func (os *optSteps) done() bool { + // remaining starts out equal to steps, and is decremented each time a rule + // is applied. If it never reaches zero, then all possible rules were + // already applied, and optimization is complete. + return os.remaining != 0 +} + +// next triggers the next iteration of optSteps. If there is no error, then +// results of the iteration can be accessed via the exprView, lastRuleName, and +// isBetter methods. +func (os *optSteps) next() error { + if os.done() { + panic("iteration already complete") + } + + // Create optimizer that will run for a fixed number of steps. + o := os.createOptimizer(os.steps) + root, required, err := os.tester.buildExpr(o.Factory()) + if err != nil { + return err + } + + // Hook the AppliedRule notification in order to track the portion of the + // expression tree affected by each transformation rule. + os.lastApplied = memo.InvalidExprID + o.NotifyOnAppliedRule(func(ruleName opt.RuleName, group memo.GroupID, added int) { + if added > 0 { + // This was an exploration rule that added one or more expressions to + // an existing group. Record the id of the first of those expressions. + // Previous expressions will be suppressed. + ord := memo.ExprOrdinal(o.Memo().ExprCount(group) - added) + os.lastApplied = memo.ExprID{Group: group, Expr: ord} + } else { + // This was a normalization that created a new memo group, or it was + // an exploration rule that didn't add any expressions to the group. + // Either way, none of the expressions in the group need to be + // suppressed. + os.lastApplied = memo.MakeNormExprID(group) + } + }) + + os.ev = o.Optimize(root, required) + text := os.ev.String() + + // If the expression text changes, then it must have gotten better. + os.better = text != os.best + if os.better { + os.best = text + } else if !os.done() { + // The expression is not better, so suppress the lowest cost expressions + // so that the changed portions of the tree will be part of the output. + o2 := os.createOptimizer(os.steps) + root, required, err := os.tester.buildExpr(o2.Factory()) + if err != nil { + return err + } + + // Set up the coster that will assign infinite costs to the expressions + // that need to be suppressed. + coster := newForcingCoster(o2.Coster()) + os.suppressExprs(coster, o.Memo(), root) + + o2.SetCoster(coster) + os.ev = o2.Optimize(root, required) + } + + os.steps++ + return nil +} + +func (os *optSteps) createOptimizer(steps int) *xform.Optimizer { + o := xform.NewOptimizer(&os.tester.evalCtx) + + // Override NotifyOnMatchedRule to stop optimizing after "steps" rule matches. + os.remaining = steps + o.NotifyOnMatchedRule(func(ruleName opt.RuleName) bool { + if os.remaining == 0 { + return false + } + os.remaining-- + os.lastMatched = ruleName + return true + }) + + return o +} + +// suppressExprs walks the tree and adds expressions which need to be suppressed +// to the forcingCoster. The expressions which need to suppressed are: +// 1. Expressions in the same group as the lastApplied expression, but with +// a lower ordinal position in the group. +// 2. Expressions in ancestor groups of the lastApplied expression that are +// not themselves ancestors of the lastApplied group. +// +// suppressExprs does this by recursively traversing the memo, starting at the +// root group. If a group expression is not an ancestor of the last applied +// group, then it is suppressed. If it is an ancestor, then suppressExprs +// recurses on any child group that is an ancestor. +func (os *optSteps) suppressExprs(coster *forcingCoster, mem *memo.Memo, group memo.GroupID) { + for e := 0; e < mem.ExprCount(group); e++ { + eid := memo.ExprID{Group: group, Expr: memo.ExprOrdinal(e)} + if eid.Group == os.lastApplied.Group { + if eid.Expr < os.lastApplied.Expr { + coster.SuppressExpr(eid) + } + continue + } + + found := false + expr := mem.Expr(eid) + for g := 0; g < expr.ChildCount(); g++ { + child := expr.ChildGroup(mem, g) + if os.findLastApplied(mem, child) { + os.suppressExprs(coster, mem, child) + found = true + break + } + } + + if !found { + coster.SuppressExpr(eid) + } + } +} + +// findLastApplied returns true if the given group is the last applied group or +// one of its ancestor groups. +func (os *optSteps) findLastApplied(mem *memo.Memo, group memo.GroupID) bool { + if group == os.lastApplied.Group { + return true + } + + for e := 0; e < mem.ExprCount(group); e++ { + eid := memo.ExprID{Group: group, Expr: memo.ExprOrdinal(e)} + expr := mem.Expr(eid) + for g := 0; g < expr.ChildCount(); g++ { + if os.findLastApplied(mem, expr.ChildGroup(mem, g)) { + return true + } + } + } + + return false +} + +// forcingCoster implements the xform.Coster interface so that it can suppress +// expressions in the memo that can't be part of the output tree. +type forcingCoster struct { + inner xform.Coster + suppressed map[memo.ExprID]bool +} + +func newForcingCoster(inner xform.Coster) *forcingCoster { + return &forcingCoster{inner: inner, suppressed: make(map[memo.ExprID]bool)} +} + +func (fc *forcingCoster) SuppressExpr(eid memo.ExprID) { + fc.suppressed[eid] = true +} + +// ComputeCost is part of the xform.Coster interface. +func (fc *forcingCoster) ComputeCost(candidate *memo.BestExpr, props *memo.LogicalProps) memo.Cost { + if fc.suppressed[candidate.Expr()] { + // Suppressed expressions get assigned MaxCost so that they never have + // the lowest cost. + return memo.MaxCost + } + return fc.inner.ComputeCost(candidate, props) +} diff --git a/pkg/sql/opt/testutils/opt_tester.go b/pkg/sql/opt/testutils/opt_tester.go index 9e8989ded973..94dcde4412d6 100644 --- a/pkg/sql/opt/testutils/opt_tester.go +++ b/pkg/sql/opt/testutils/opt_tester.go @@ -72,6 +72,10 @@ type OptTesterFlags struct { // an UnsupportedExpr node. This is temporary; it is used for interfacing with // the old planning code. AllowUnsupportedExpr bool + + // Verbose indicates whether verbose test debugging information will be + // output to stdout when commands run. Only certain commands support this. + Verbose bool } // NewOptTester constructs a new instance of the OptTester for the given SQL @@ -123,45 +127,47 @@ func NewOptTester(catalog opt.Catalog, sql string) *OptTester { // // - allow-unsupported: wrap unsupported expressions in UnsupportedOp. // -func (e *OptTester) RunCommand(tb testing.TB, d *datadriven.TestData) string { +func (ot *OptTester) RunCommand(tb testing.TB, d *datadriven.TestData) string { // Allow testcases to override the flags. for _, a := range d.CmdArgs { - if err := e.Flags.Set(a); err != nil { + if err := ot.Flags.Set(a); err != nil { d.Fatalf(tb, "%s", err) } } + ot.Flags.Verbose = testing.Verbose() + switch d.Cmd { case "exec-ddl": - testCatalog, ok := e.catalog.(*TestCatalog) + testCatalog, ok := ot.catalog.(*TestCatalog) if !ok { tb.Fatal("exec-ddl can only be used with TestCatalog") } return ExecuteTestDDL(tb, d.Input, testCatalog) case "build": - ev, err := e.OptBuild() + ev, err := ot.OptBuild() if err != nil { return fmt.Sprintf("error: %s\n", strings.TrimSpace(err.Error())) } - return ev.FormatString(e.Flags.ExprFormat) + return ev.FormatString(ot.Flags.ExprFormat) case "opt": - ev, err := e.Optimize() + ev, err := ot.Optimize() if err != nil { d.Fatalf(tb, "%v", err) } - return ev.FormatString(e.Flags.ExprFormat) + return ev.FormatString(ot.Flags.ExprFormat) case "optsteps": - result, err := e.OptSteps(testing.Verbose()) + result, err := ot.OptSteps() if err != nil { d.Fatalf(tb, "%v", err) } return result case "memo": - result, err := e.Memo() + result, err := ot.Memo() if err != nil { d.Fatalf(tb, "%v", err) } @@ -212,41 +218,63 @@ func (f *OptTesterFlags) Set(arg datadriven.CmdArg) error { // OptBuild constructs an opt expression tree for the SQL query, with no // transformations applied to it. The untouched output of the optbuilder is the // final expression tree. -func (e *OptTester) OptBuild() (memo.ExprView, error) { - return e.optimizeExpr(false /* allowOptimizations */) +func (ot *OptTester) OptBuild() (memo.ExprView, error) { + return ot.optimizeExpr(false /* allowOptimizations */) } // Optimize constructs an opt expression tree for the SQL query, with all // transformations applied to it. The result is the memo expression tree with // the lowest estimated cost. -func (e *OptTester) Optimize() (memo.ExprView, error) { - return e.optimizeExpr(true /* allowOptimizations */) +func (ot *OptTester) Optimize() (memo.ExprView, error) { + return ot.optimizeExpr(true /* allowOptimizations */) } // Memo returns a string that shows the memo data structure that is constructed // by the optimizer. -func (e *OptTester) Memo() (string, error) { - o := xform.NewOptimizer(&e.evalCtx) - root, required, err := e.buildExpr(o.Factory()) +func (ot *OptTester) Memo() (string, error) { + o := xform.NewOptimizer(&ot.evalCtx) + root, required, err := ot.buildExpr(o.Factory()) if err != nil { return "", err } o.Optimize(root, required) - return o.Memo().FormatString(e.Flags.MemoFormat), nil + return o.Memo().FormatString(ot.Flags.MemoFormat), nil } -// OptSteps returns a string that shows each optimization step using the -// standard unified diff format. It is used for debugging the optimizer. -// If verbose is true, each step is also printed on stdout. -func (e *OptTester) OptSteps(verbose bool) (string, error) { +// OptSteps steps through the transformations performed by the optimizer on the +// memo, one-by-one. The output of each step is the lowest cost expression tree +// that also contains the expressions that were changed or added by the +// transformation. The output of each step is diff'd against the output of a +// previous step, using the standard unified diff format. +// +// CREATE TABLE a (x INT PRIMARY KEY, y INT, UNIQUE INDEX (y)) +// +// SELECT x FROM a WHERE x=1 +// +// At the time of this writing, this query triggers 6 rule applications: +// EnsureSelectFilters Wrap Select predicate with Filters operator +// FilterUnusedSelectCols Do not return unused "y" column from Scan +// EliminateProject Remove unneeded Project operator +// GenerateIndexScans Explore scanning "y" index to get "x" values +// ConstrainScan Explore pushing "x=1" into "x" index Scan +// ConstrainScan Explore pushing "x=1" into "y" index Scan +// +// Some steps produce better plans that have a lower execution cost. Other steps +// don't. However, it's useful to see both kinds of steps. The optsteps output +// distinguishes these two cases by using stronger "====" header delimiters when +// a better plan has been found, and weaker "----" header delimiters when not. +// In both cases, the output shows the expressions that were changed or added by +// the rule, even if the total expression tree cost worsened. +// +func (ot *OptTester) OptSteps() (string, error) { var buf bytes.Buffer - var prev, next string - if verbose { + var prevBest, prev, next string + if ot.Flags.Verbose { fmt.Print("------ optsteps verbose output starts ------\n") } output := func(format string, args ...interface{}) { fmt.Fprintf(&buf, format, args...) - if verbose { + if ot.Flags.Verbose { fmt.Printf(format, args...) } } @@ -257,66 +285,90 @@ func (e *OptTester) OptSteps(verbose bool) (string, error) { output(" %s\n", line) } } - for i := 0; ; i++ { - o := xform.NewOptimizer(&e.evalCtx) - - // Override SetOnRuleMatch to stop optimizing after the ith rule matches. - steps := i - lastRuleName := opt.InvalidRuleName - o.SetOnRuleMatch(func(ruleName opt.RuleName) bool { - if steps == 0 { - return false - } - steps-- - lastRuleName = ruleName - return true - }) - root, required, err := e.buildExpr(o.Factory()) + // bestHeader is used when the expression is an improvement over the previous + // expression. + bestHeader := func(ev memo.ExprView, format string, args ...interface{}) { + output("%s\n", strings.Repeat("=", 80)) + output(format, args...) + output(" Cost: %.2f\n", ev.Cost()) + output("%s\n", strings.Repeat("=", 80)) + } + + // altHeader is used when the expression doesn't improve over the previous + // expression, but it's still desirable to see what changed. + altHeader := func(format string, args ...interface{}) { + output("%s\n", strings.Repeat("-", 80)) + output(format, args...) + output("%s\n", strings.Repeat("-", 80)) + } + + os := newOptSteps(ot) + for { + err := os.next() if err != nil { return "", err } - next = o.Optimize(root, required).FormatString(e.Flags.ExprFormat) - if steps != 0 { - // All steps were not used, so must be done. + next = os.exprView().FormatString(ot.Flags.ExprFormat) + + // This call comes after setting "next", because we want to output the + // final expression, even though there were no diffs from the previous + // iteration. + if os.done() { break } - if i == 0 { - output("*** Initial expr:\n") + if prev == "" { // Output starting tree. + bestHeader(os.exprView(), "Initial expression\n") indent(next) + prevBest = next + } else if next == prev { + altHeader("%s (no changes)\n", os.lastRuleName()) } else { - output("\n*** %s applied; ", lastRuleName.String()) + var diff difflib.UnifiedDiff + if os.isBetter() { + // New expression is better than the previous expression. Diff + // it against the previous *best* expression (might not be the + // previous expression). + bestHeader(os.exprView(), "%s\n", os.lastRuleName()) + + diff = difflib.UnifiedDiff{ + A: difflib.SplitLines(prevBest), + B: difflib.SplitLines(next), + Context: 100, + } - if prev == next { - // The expression can be unchanged if a part of the memo changed that - // does not affect the final best expression. - output("best expr unchanged.\n") + prevBest = next } else { - output("best expr changed:\n") - diff := difflib.UnifiedDiff{ + // New expression is not better than the previous expression, but + // still show the change. Diff it against the previous expression, + // regardless if it was a "best" expression or not. + altHeader("%s (higher cost)\n", os.lastRuleName()) + + next = os.exprView().FormatString(ot.Flags.ExprFormat) + diff = difflib.UnifiedDiff{ A: difflib.SplitLines(prev), B: difflib.SplitLines(next), Context: 100, } - - text, _ := difflib.GetUnifiedDiffString(diff) - // Skip the "@@ ... @@" header (first line). - text = strings.SplitN(text, "\n", 2)[1] - indent(text) } + + text, _ := difflib.GetUnifiedDiffString(diff) + // Skip the "@@ ... @@" header (first line). + text = strings.SplitN(text, "\n", 2)[1] + indent(text) } prev = next } // Output ending tree. - output("\n*** Final best expr:\n") + bestHeader(os.exprView(), "Final best expression\n") indent(next) - if verbose { + if ot.Flags.Verbose { fmt.Print("------ optsteps verbose output ends ------\n") } @@ -325,8 +377,8 @@ func (e *OptTester) OptSteps(verbose bool) (string, error) { // ExecBuild builds the exec node tree for the SQL query. This can be executed // by the exec engine. -func (e *OptTester) ExecBuild(eng exec.TestEngine) (exec.Node, error) { - ev, err := e.Optimize() +func (ot *OptTester) ExecBuild(eng exec.TestEngine) (exec.Node, error) { + ev, err := ot.Optimize() if err != nil { return nil, err } @@ -335,8 +387,8 @@ func (e *OptTester) ExecBuild(eng exec.TestEngine) (exec.Node, error) { // Explain builds the exec node tree for the SQL query and then runs the // explain command that describes the physical execution plan. -func (e *OptTester) Explain(eng exec.TestEngine) ([]tree.Datums, error) { - node, err := e.ExecBuild(eng) +func (ot *OptTester) Explain(eng exec.TestEngine) ([]tree.Datums, error) { + node, err := ot.ExecBuild(eng) if err != nil { return nil, err } @@ -344,8 +396,8 @@ func (e *OptTester) Explain(eng exec.TestEngine) ([]tree.Datums, error) { } // Exec builds the exec node tree for the SQL query and then executes it. -func (e *OptTester) Exec(eng exec.TestEngine) (sqlbase.ResultColumns, []tree.Datums, error) { - node, err := e.ExecBuild(eng) +func (ot *OptTester) Exec(eng exec.TestEngine) (sqlbase.ResultColumns, []tree.Datums, error) { + node, err := ot.ExecBuild(eng) if err != nil { return nil, nil, err } @@ -360,25 +412,25 @@ func (e *OptTester) Exec(eng exec.TestEngine) (sqlbase.ResultColumns, []tree.Dat return columns, datums, err } -func (e *OptTester) buildExpr( +func (ot *OptTester) buildExpr( factory *norm.Factory, ) (root memo.GroupID, required *memo.PhysicalProps, _ error) { - stmt, err := parser.ParseOne(e.sql) + stmt, err := parser.ParseOne(ot.sql) if err != nil { return 0, nil, err } - b := optbuilder.New(e.ctx, &e.semaCtx, &e.evalCtx, e.catalog, factory, stmt) - b.AllowUnsupportedExpr = e.Flags.AllowUnsupportedExpr + b := optbuilder.New(ot.ctx, &ot.semaCtx, &ot.evalCtx, ot.catalog, factory, stmt) + b.AllowUnsupportedExpr = ot.Flags.AllowUnsupportedExpr return b.Build() } -func (e *OptTester) optimizeExpr(allowOptimizations bool) (memo.ExprView, error) { - o := xform.NewOptimizer(&e.evalCtx) +func (ot *OptTester) optimizeExpr(allowOptimizations bool) (memo.ExprView, error) { + o := xform.NewOptimizer(&ot.evalCtx) if !allowOptimizations { o.DisableOptimizations() } - root, required, err := e.buildExpr(o.Factory()) + root, required, err := ot.buildExpr(o.Factory()) if err != nil { return memo.ExprView{}, err } diff --git a/pkg/sql/opt/xform/coster.go b/pkg/sql/opt/xform/coster.go index a1ca14050d7f..a6f37701743b 100644 --- a/pkg/sql/opt/xform/coster.go +++ b/pkg/sql/opt/xform/coster.go @@ -19,17 +19,35 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/opt/memo" ) -// coster encapsulates the cost model for the optimizer. The coster assigns an -// estimated cost to each expression in the memo so that the optimizer can -// choose the lowest cost expression tree. The estimated cost is a best-effort -// approximation of the actual cost of execution, based on table and index -// statistics that are propagated throughout the logical expression tree. +// Coster is used by the optimizer to assign a cost to a candidate expression +// that can provide a set of required physical properties. If a candidate +// expression has a lower cost than any other expression in the memo group, then +// it becomes the new best expression for the group. +// +// Coster is an interface so that different costing algorithms can be used by +// the optimizer. For example, the OptSteps command uses a custom coster that +// assigns infinite costs to some expressions in order to prevent them from +// being part of the lowest cost tree (for debugging purposes). +type Coster interface { + // ComputeCost returns the estimated cost of executing the candidate + // expression. The optimizer does not expect the cost to correspond to any + // real-world metric, but does expect costs to be comparable to one another, + // as well as summable. + ComputeCost(candidate *memo.BestExpr, props *memo.LogicalProps) memo.Cost +} + +// coster encapsulates the default cost model for the optimizer. The coster +// assigns an estimated cost to each expression in the memo so that the +// optimizer can choose the lowest cost expression tree. The estimated cost is +// a best-effort approximation of the actual cost of execution, based on table +// and index statistics that are propagated throughout the logical expression +// tree. type coster struct { mem *memo.Memo } -func (c *coster) init(mem *memo.Memo) { - c.mem = mem +func newCoster(mem *memo.Memo) *coster { + return &coster{mem: mem} } // computeCost calculates the estimated cost of the candidate best expression, @@ -38,27 +56,24 @@ func (c *coster) init(mem *memo.Memo) { // branch-and-bound pruning will work properly. // // TODO: This is just a skeleton, and needs to compute real costs. -func (c *coster) computeCost(candidate *memo.BestExpr, props *memo.LogicalProps) { - var cost memo.Cost +func (c *coster) ComputeCost(candidate *memo.BestExpr, props *memo.LogicalProps) memo.Cost { switch candidate.Operator() { case opt.SortOp: - cost = c.computeSortCost(candidate, props) + return c.computeSortCost(candidate, props) case opt.ScanOp: - cost = c.computeScanCost(candidate, props) + return c.computeScanCost(candidate, props) case opt.SelectOp: - cost = c.computeSelectCost(candidate, props) + return c.computeSelectCost(candidate, props) case opt.ValuesOp: - cost = c.computeValuesCost(candidate, props) + return c.computeValuesCost(candidate, props) default: // By default, cost of parent is sum of child costs. - cost = c.computeChildrenCost(candidate) + return c.computeChildrenCost(candidate) } - - candidate.SetCost(cost) } func (c *coster) computeSortCost(candidate *memo.BestExpr, props *memo.LogicalProps) memo.Cost { diff --git a/pkg/sql/opt/xform/explorer.og.go b/pkg/sql/opt/xform/explorer.og.go index b55d06f2f160..165af648426d 100644 --- a/pkg/sql/opt/xform/explorer.og.go +++ b/pkg/sql/opt/xform/explorer.og.go @@ -31,10 +31,15 @@ func (_e *explorer) exploreScan(_rootState *exploreState, _root memo.ExprID) (_f if _root.Expr >= _rootState.start { def := _rootExpr.Def() if _e.canGenerateIndexScans(def) { - if _e.o.onRuleMatch == nil || _e.o.onRuleMatch(opt.GenerateIndexScans) { - exprs := _e.generateIndexScans(def) - for i := range exprs { - _e.mem.MemoizeDenormExpr(_root.Group, exprs[i]) + if _e.o.matchedRule == nil || _e.o.matchedRule(opt.GenerateIndexScans) { + _exprs := _e.generateIndexScans(def) + _before := _e.mem.ExprCount(_root.Group) + for i := range _exprs { + _e.mem.MemoizeDenormExpr(_root.Group, _exprs[i]) + } + if _e.o.appliedRule != nil { + _after := _e.mem.ExprCount(_root.Group) + _e.o.appliedRule(opt.GenerateIndexScans, _root.Group, _after-_before) } } } @@ -66,10 +71,15 @@ func (_e *explorer) exploreSelect(_rootState *exploreState, _root memo.ExprID) ( def := _scanExpr.Def() if _e.canConstrainScan(def) { filter := _rootExpr.Filter() - if _e.o.onRuleMatch == nil || _e.o.onRuleMatch(opt.ConstrainScan) { - exprs := _e.constrainScan(filter, def) - for i := range exprs { - _e.mem.MemoizeDenormExpr(_root.Group, exprs[i]) + if _e.o.matchedRule == nil || _e.o.matchedRule(opt.ConstrainScan) { + _exprs := _e.constrainScan(filter, def) + _before := _e.mem.ExprCount(_root.Group) + for i := range _exprs { + _e.mem.MemoizeDenormExpr(_root.Group, _exprs[i]) + } + if _e.o.appliedRule != nil { + _after := _e.mem.ExprCount(_root.Group) + _e.o.appliedRule(opt.ConstrainScan, _root.Group, _after-_before) } } } @@ -106,11 +116,16 @@ func (_e *explorer) exploreLimit(_rootState *exploreState, _root memo.ExprID) (_ limit := _constExpr.Value() ordering := _rootExpr.Ordering() if _e.canLimitScan(def, ordering) { - if _e.o.onRuleMatch == nil || _e.o.onRuleMatch(opt.PushLimitIntoScan) { + if _e.o.matchedRule == nil || _e.o.matchedRule(opt.PushLimitIntoScan) { _expr := memo.MakeScanExpr( _e.limitScanDef(def, limit), ) + _before := _e.mem.ExprCount(_root.Group) _e.mem.MemoizeDenormExpr(_root.Group, memo.Expr(_expr)) + if _e.o.appliedRule != nil { + _after := _e.mem.ExprCount(_root.Group) + _e.o.appliedRule(opt.PushLimitIntoScan, _root.Group, _after-_before) + } } } } diff --git a/pkg/sql/opt/xform/optimizer.go b/pkg/sql/opt/xform/optimizer.go index 336af679707b..c21ce29628d9 100644 --- a/pkg/sql/opt/xform/optimizer.go +++ b/pkg/sql/opt/xform/optimizer.go @@ -24,6 +24,16 @@ import ( "github.com/cockroachdb/cockroach/pkg/util" ) +// MatchedRuleFunc defines the callback function for the NotifyOnMatchedRule +// event supported by the optimizer. See the comment in factory.go for more +// details. +type MatchedRuleFunc = norm.MatchedRuleFunc + +// AppliedRuleFunc defines the callback function for the NotifyOnAppliedRule +// event supported by the optimizer. See the comment in factory.go for more +// details. +type AppliedRuleFunc = norm.AppliedRuleFunc + // Optimizer transforms an input expression tree into the logically equivalent // output expression tree with the lowest possible execution cost. // @@ -38,7 +48,7 @@ type Optimizer struct { evalCtx *tree.EvalContext f *norm.Factory mem *memo.Memo - coster coster + coster Coster explorer explorer // stateMap allocates temporary storage that's used to speed up optimization. @@ -46,10 +56,15 @@ type Optimizer struct { stateMap map[optStateKey]*optState stateAlloc optStateAlloc - // onRuleMatch is the callback function that is invoked each time a normalize - // rule has been matched by the factory. It can be set via a call to the - // SetOnRuleMatch method. - onRuleMatch func(ruleName opt.RuleName) bool + // matchedRule is the callback function that is invoked each time an + // optimization rule (Normalize or Explore) has been matched by the optimizer. + // It can be set via a call to the NotifyOnMatchedRule method. + matchedRule MatchedRuleFunc + + // appliedRule is the callback function which is invoked each time an + // optimization rule (Normalize or Explore) has been applied by the optimizer. + // It can be set via a call to the NotifyOnAppliedRule method. + appliedRule AppliedRuleFunc } // NewOptimizer constructs an instance of the optimizer. @@ -59,9 +74,9 @@ func NewOptimizer(evalCtx *tree.EvalContext) *Optimizer { evalCtx: evalCtx, f: f, mem: f.Memo(), + coster: newCoster(f.Memo()), stateMap: make(map[optStateKey]*optState), } - o.coster.init(o.mem) o.explorer.init(o) return o } @@ -73,25 +88,49 @@ func (o *Optimizer) Factory() *norm.Factory { return o.f } +// Coster returns the coster instance that the optimizer is currently using to +// estimate the cost of executing portions of the expression tree. When a new +// optimizer is constructed, it creates a default coster that will be used +// unless it is overridden with a call to SetCoster. +func (o *Optimizer) Coster() Coster { + return o.coster +} + +// SetCoster overrides the default coster. The optimizer will now use the given +// coster to estimate the cost of expression execution. +func (o *Optimizer) SetCoster(coster Coster) { + o.coster = coster +} + // DisableOptimizations disables all transformation rules, including normalize // and explore rules. The unaltered input expression tree becomes the output // expression tree (because no transforms are applied). func (o *Optimizer) DisableOptimizations() { - o.SetOnRuleMatch(func(opt.RuleName) bool { return false }) + o.NotifyOnMatchedRule(func(opt.RuleName) bool { return false }) +} + +// NotifyOnMatchedRule sets a callback function which is invoked each time an +// optimization rule (Normalize or Explore) has been matched by the optimizer. +// If matchedRule is nil, then no notifications are sent, and all rules are +// applied by default. In addition, callers can invoke the DisableOptimizations +// convenience method to disable all rules. +func (o *Optimizer) NotifyOnMatchedRule(matchedRule MatchedRuleFunc) { + o.matchedRule = matchedRule + + // Also pass through the call to the factory so that normalization rules + // make same callback. + o.f.NotifyOnMatchedRule(matchedRule) } -// SetOnRuleMatch sets a callback function which is invoked each time an -// optimization rule (Normalize or Explore rule) has been matched by the -// optimizer. If the function returns false, then the rule is not applied. By -// default, all rules are applied, but callers can set the callback function to -// override the default behavior. In addition, callers can invoke the -// DisableOptimizations convenience method to disable all rules. -func (o *Optimizer) SetOnRuleMatch(onRuleMatch func(ruleName opt.RuleName) bool) { - o.onRuleMatch = onRuleMatch +// NotifyOnAppliedRule sets a callback function which is invoked each time an +// optimization rule (Normalize or Explore) has been applied by the optimizer. +// If appliedRule is nil, then no further notifications are sent. +func (o *Optimizer) NotifyOnAppliedRule(appliedRule AppliedRuleFunc) { + o.appliedRule = appliedRule // Also pass through the call to the factory so that normalization rules // make same callback. - o.f.SetOnRuleMatch(onRuleMatch) + o.f.NotifyOnAppliedRule(appliedRule) } // Memo returns the memo structure that the optimizer is using to optimize. @@ -424,7 +463,8 @@ func (o *Optimizer) enforceProps( // group. If so, then the candidate becomes the new lowest cost expression. func (o *Optimizer) ratchetCost(candidate *memo.BestExpr) { group := candidate.Group() - o.coster.computeCost(candidate, o.mem.GroupProperties(group)) + cost := o.coster.ComputeCost(candidate, o.mem.GroupProperties(group)) + candidate.SetCost(cost) state := o.lookupOptState(group, candidate.Required()) if state.best == memo.UnknownBestExprID { // Lazily allocate the best expression only when it's needed. diff --git a/pkg/testutils/error.go b/pkg/testutils/error.go index 33b9b4d25f5a..f5bc4371f42e 100644 --- a/pkg/testutils/error.go +++ b/pkg/testutils/error.go @@ -60,10 +60,16 @@ func IsPError(pErr *roachpb.Error, re string) bool { // itself. This can occur when a node is restarting or is unstable in // some other way. Note that retryable errors may occur event in cases // where the SQL execution ran to completion. +// +// TODO(bdarnell): Why are RPC errors in this list? These should +// generally be retried on the server side or transformed into +// ambiguous result errors ("connection reset/refused" are needed for +// the pgwire connection, but anything RPC-related should be handled +// within the cluster). func IsSQLRetryableError(err error) bool { // Don't forget to update the corresponding test when making adjustments // here. - return IsError(err, "(connection reset by peer|connection refused|failed to send RPC|EOF|result is ambiguous)") + return IsError(err, "(connection reset by peer|connection refused|failed to send RPC|rpc error: code = Unavailable|EOF|result is ambiguous)") } // Caller returns filename and line number info for the specified stack