From 01c425ea1d3f985ef07ca9d913d8ee520b90d757 Mon Sep 17 00:00:00 2001 From: DrewKimball Date: Mon, 22 Aug 2022 12:10:09 -0700 Subject: [PATCH] opt: don't propagate rule props when corresponding rule is disabled This commit decreases the likelihood of rule cycles occuring during optimizer tests with randomly disabled rules. `DerivePruneCols` and `DeriveRejectNullCols` now check whether propagating their corresponding properties would trigger a disabled rule (if it wasn't disabled), and avoid propagating in that case. This is necessary to avoid cases where a `Select` or `Project` gets repeatedly pushed down and eliminated. Addresses #86308 Release note: None Release justification: testing-only change --- pkg/sql/opt/norm/factory.go | 13 +++ pkg/sql/opt/norm/prune_cols_funcs.go | 83 ++++++++++++++++--- pkg/sql/opt/norm/reject_nulls_funcs.go | 59 ++++++++++--- pkg/sql/opt/norm/testdata/rules/prune_cols | 75 +++++++++++++++++ .../opt/testutils/opttester/forcing_opt.go | 1 + pkg/sql/opt/testutils/opttester/opt_tester.go | 6 +- pkg/sql/opt/xform/optimizer.go | 17 ++-- 7 files changed, 218 insertions(+), 36 deletions(-) diff --git a/pkg/sql/opt/norm/factory.go b/pkg/sql/opt/norm/factory.go index b9c3029c70b8..dfd4487bc221 100644 --- a/pkg/sql/opt/norm/factory.go +++ b/pkg/sql/opt/norm/factory.go @@ -20,6 +20,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/sem/eval" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/types" + "github.com/cockroachdb/cockroach/pkg/util" "github.com/cockroachdb/cockroach/pkg/util/buildutil" "github.com/cockroachdb/cockroach/pkg/util/errorutil" "github.com/cockroachdb/errors" @@ -95,6 +96,10 @@ type Factory struct { // methods. It is incremented when a constructor function is called, and // decremented when a constructor function returns. constructorStackDepth int + + // disabledRules is a set of rules that are not allowed to run, used when + // rules are disabled during testing to prevent rule cycles. + disabledRules util.FastIntSet } // maxConstructorStackDepth is the maximum allowed depth of a constructor call @@ -195,6 +200,14 @@ func (f *Factory) NotifyOnAppliedRule(appliedRule AppliedRuleFunc) { f.appliedRule = appliedRule } +// SetDisabledRules is used to prevent normalization rule cycles when rules are +// disabled during testing. SetDisabledRules does not prevent rules from +// matching - rather, it notifies the Factory that rules have been prevented +// from matching using NotifyOnMatchedRule. +func (f *Factory) SetDisabledRules(disabledRules util.FastIntSet) { + f.disabledRules = disabledRules +} + // Memo returns the memo structure that the factory is operating upon. func (f *Factory) Memo() *memo.Memo { return f.mem diff --git a/pkg/sql/opt/norm/prune_cols_funcs.go b/pkg/sql/opt/norm/prune_cols_funcs.go index 78100c03c286..a75617b8311c 100644 --- a/pkg/sql/opt/norm/prune_cols_funcs.go +++ b/pkg/sql/opt/norm/prune_cols_funcs.go @@ -16,6 +16,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/opt/memo" "github.com/cockroachdb/cockroach/pkg/sql/opt/props" "github.com/cockroachdb/cockroach/pkg/sql/types" + "github.com/cockroachdb/cockroach/pkg/util" ) // NeededGroupingCols returns the columns needed by a grouping operator's @@ -251,7 +252,7 @@ func (c *CustomFuncs) NeededMutationFetchCols( // needed columns from that. See the props.Relational.Rule.PruneCols comment for // more details. func (c *CustomFuncs) CanPruneCols(target memo.RelExpr, neededCols opt.ColSet) bool { - return !DerivePruneCols(target).SubsetOf(neededCols) + return !DerivePruneCols(target, c.f.disabledRules).SubsetOf(neededCols) } // CanPruneAggCols returns true if one or more of the target aggregations is not @@ -309,7 +310,7 @@ func (c *CustomFuncs) PruneCols(target memo.RelExpr, neededCols opt.ColSet) memo // Get the subset of the target expression's output columns that should // not be pruned. Don't prune if the target output column is needed by a // higher-level expression, or if it's not part of the PruneCols set. - pruneCols := DerivePruneCols(target).Difference(neededCols) + pruneCols := DerivePruneCols(target, c.f.disabledRules).Difference(neededCols) colSet := c.OutputCols(target).Difference(pruneCols) return c.f.ConstructProject(target, memo.EmptyProjectionsExpr, colSet) } @@ -499,7 +500,12 @@ func (c *CustomFuncs) PruneWindows(needed opt.ColSet, windows memo.WindowsExpr) // what columns it allows to be pruned. Note that if an operator allows columns // to be pruned, then there must be logic in the PruneCols method to actually // prune those columns when requested. -func DerivePruneCols(e memo.RelExpr) opt.ColSet { +// +// disabledRules is the set of rules currently disabled, only used when rules +// are randomly disabled for testing. It is used to prevent propagating the +// PruneCols property when the corresponding column-pruning normalization rule +// is disabled. This prevents rule cycles during testing. +func DerivePruneCols(e memo.RelExpr, disabledRules util.FastIntSet) opt.ColSet { relProps := e.Relational() if relProps.IsAvailable(props.PruneCols) { return relProps.Rule.PruneCols @@ -508,19 +514,33 @@ func DerivePruneCols(e memo.RelExpr) opt.ColSet { switch e.Op() { case opt.ScanOp, opt.ValuesOp, opt.WithScanOp: + if disabledRules.Contains(int(opt.PruneScanCols)) || + disabledRules.Contains(int(opt.PruneValuesCols)) || + disabledRules.Contains(int(opt.PruneWithScanCols)) { + // Avoid rule cycles. + break + } // All columns can potentially be pruned from the Scan, Values, and WithScan // operators. relProps.Rule.PruneCols = relProps.OutputCols.Copy() case opt.SelectOp: + if disabledRules.Contains(int(opt.PruneSelectCols)) { + // Avoid rule cycles. + break + } // Any pruneable input columns can potentially be pruned, as long as they're // not used by the filter. sel := e.(*memo.SelectExpr) - relProps.Rule.PruneCols = DerivePruneCols(sel.Input).Copy() + relProps.Rule.PruneCols = DerivePruneCols(sel.Input, disabledRules).Copy() usedCols := sel.Filters.OuterCols() relProps.Rule.PruneCols.DifferenceWith(usedCols) case opt.ProjectOp: + if disabledRules.Contains(int(opt.PruneProjectCols)) { + // Avoid rule cycles. + break + } // All columns can potentially be pruned from the Project, if they're never // used in a higher-level expression. relProps.Rule.PruneCols = relProps.OutputCols.Copy() @@ -528,13 +548,19 @@ func DerivePruneCols(e memo.RelExpr) opt.ColSet { case opt.InnerJoinOp, opt.LeftJoinOp, opt.RightJoinOp, opt.FullJoinOp, opt.SemiJoinOp, opt.AntiJoinOp, opt.InnerJoinApplyOp, opt.LeftJoinApplyOp, opt.SemiJoinApplyOp, opt.AntiJoinApplyOp: + if disabledRules.Contains(int(opt.PruneJoinLeftCols)) || + disabledRules.Contains(int(opt.PruneJoinRightCols)) || + disabledRules.Contains(int(opt.PruneSemiAntiJoinRightCols)) { + // Avoid rule cycles. + break + } // Any pruneable columns from projected inputs can potentially be pruned, as // long as they're not used by the right input (i.e. in Apply case) or by // the join filter. left := e.Child(0).(memo.RelExpr) - leftPruneCols := DerivePruneCols(left) + leftPruneCols := DerivePruneCols(left, disabledRules) right := e.Child(1).(memo.RelExpr) - rightPruneCols := DerivePruneCols(right) + rightPruneCols := DerivePruneCols(right, disabledRules) switch e.Op() { case opt.SemiJoinOp, opt.SemiJoinApplyOp, opt.AntiJoinOp, opt.AntiJoinApplyOp: @@ -548,6 +574,11 @@ func DerivePruneCols(e memo.RelExpr) opt.ColSet { relProps.Rule.PruneCols.DifferenceWith(onCols) case opt.GroupByOp, opt.ScalarGroupByOp, opt.DistinctOnOp, opt.EnsureDistinctOnOp: + if disabledRules.Contains(int(opt.PruneGroupByCols)) || + disabledRules.Contains(int(opt.PruneAggCols)) { + // Avoid rule cycles. + break + } // Grouping columns can't be pruned, because they were used to group rows. // However, aggregation columns can potentially be pruned. groupingColSet := e.Private().(*memo.GroupingPrivate).GroupingCols @@ -558,19 +589,28 @@ func DerivePruneCols(e memo.RelExpr) opt.ColSet { } case opt.LimitOp, opt.OffsetOp: + if disabledRules.Contains(int(opt.PruneLimitCols)) || + disabledRules.Contains(int(opt.PruneOffsetCols)) { + // Avoid rule cycles. + break + } // Any pruneable input columns can potentially be pruned, as long as // they're not used as an ordering column. - inputPruneCols := DerivePruneCols(e.Child(0).(memo.RelExpr)) + inputPruneCols := DerivePruneCols(e.Child(0).(memo.RelExpr), disabledRules) ordering := e.Private().(*props.OrderingChoice).ColSet() relProps.Rule.PruneCols = inputPruneCols.Difference(ordering) case opt.OrdinalityOp: + if disabledRules.Contains(int(opt.PruneOrdinalityCols)) { + // Avoid rule cycles. + break + } // Any pruneable input columns can potentially be pruned, as long as // they're not used as an ordering column. The new row number column // cannot be pruned without adding an additional Project operator, so // don't add it to the set. ord := e.(*memo.OrdinalityExpr) - inputPruneCols := DerivePruneCols(ord.Input) + inputPruneCols := DerivePruneCols(ord.Input, disabledRules) relProps.Rule.PruneCols = inputPruneCols.Difference(ord.Ordering.ColSet()) case opt.IndexJoinOp, opt.LookupJoinOp, opt.MergeJoinOp: @@ -581,26 +621,39 @@ func DerivePruneCols(e memo.RelExpr) opt.ColSet { // currently a PruneCols rule for these operators. case opt.ProjectSetOp: + if disabledRules.Contains(int(opt.PruneProjectSetCols)) { + // Avoid rule cycles. + break + } // Any pruneable input columns can potentially be pruned, as long as // they're not used in the Zip. // TODO(rytaft): It may be possible to prune Zip columns, but we need to // make sure that we still get the correct number of rows in the output. projectSet := e.(*memo.ProjectSetExpr) - relProps.Rule.PruneCols = DerivePruneCols(projectSet.Input).Copy() + relProps.Rule.PruneCols = DerivePruneCols(projectSet.Input, disabledRules).Copy() usedCols := projectSet.Zip.OuterCols() relProps.Rule.PruneCols.DifferenceWith(usedCols) case opt.UnionAllOp: + if disabledRules.Contains(int(opt.PruneUnionAllCols)) { + // Avoid rule cycles. + break + } // Pruning can be beneficial as long as one of our inputs has advertised pruning, // so that we can push down the project and eliminate the advertisement. u := e.(*memo.UnionAllExpr) - pruneFromLeft := opt.TranslateColSet(DerivePruneCols(u.Left), u.LeftCols, u.OutCols) - pruneFromRight := opt.TranslateColSet(DerivePruneCols(u.Right), u.RightCols, u.OutCols) + pruneFromLeft := opt.TranslateColSet(DerivePruneCols(u.Left, disabledRules), u.LeftCols, u.OutCols) + pruneFromRight := opt.TranslateColSet(DerivePruneCols(u.Right, disabledRules), u.RightCols, u.OutCols) relProps.Rule.PruneCols = pruneFromLeft.Union(pruneFromRight) case opt.WindowOp: + if disabledRules.Contains(int(opt.PruneWindowInputCols)) || + disabledRules.Contains(int(opt.PruneWindowOutputCols)) { + // Avoid rule cycles. + break + } win := e.(*memo.WindowExpr) - relProps.Rule.PruneCols = DerivePruneCols(win.Input).Copy() + relProps.Rule.PruneCols = DerivePruneCols(win.Input, disabledRules).Copy() relProps.Rule.PruneCols.DifferenceWith(win.Partition) relProps.Rule.PruneCols.DifferenceWith(win.Ordering.ColSet()) for _, w := range win.Windows { @@ -609,9 +662,13 @@ func DerivePruneCols(e memo.RelExpr) opt.ColSet { } case opt.WithOp: + if disabledRules.Contains(int(opt.PruneWithCols)) { + // Avoid rule cycles. + break + } // WithOp passes through its input unchanged, so it has the same pruning // characteristics as its input. - relProps.Rule.PruneCols = DerivePruneCols(e.(*memo.WithExpr).Main) + relProps.Rule.PruneCols = DerivePruneCols(e.(*memo.WithExpr).Main, disabledRules) default: // Don't allow any columns to be pruned, since that would trigger the diff --git a/pkg/sql/opt/norm/reject_nulls_funcs.go b/pkg/sql/opt/norm/reject_nulls_funcs.go index 89201737308c..45309bed9c2c 100644 --- a/pkg/sql/opt/norm/reject_nulls_funcs.go +++ b/pkg/sql/opt/norm/reject_nulls_funcs.go @@ -14,6 +14,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/opt" "github.com/cockroachdb/cockroach/pkg/sql/opt/memo" "github.com/cockroachdb/cockroach/pkg/sql/opt/props" + "github.com/cockroachdb/cockroach/pkg/util" "github.com/cockroachdb/errors" ) @@ -21,7 +22,7 @@ import ( // rejection filter pushdown. See the Relational.Rule.RejectNullCols comment for // more details. func (c *CustomFuncs) RejectNullCols(in memo.RelExpr) opt.ColSet { - return DeriveRejectNullCols(in) + return DeriveRejectNullCols(in, c.f.disabledRules) } // HasNullRejectingFilter returns true if the filter causes some of the columns @@ -118,7 +119,12 @@ func (c *CustomFuncs) NullRejectProjections( // DeriveRejectNullCols returns the set of columns that are candidates for NULL // rejection filter pushdown. See the Relational.Rule.RejectNullCols comment for // more details. -func DeriveRejectNullCols(in memo.RelExpr) opt.ColSet { +// +// disabledRules is the set of rules currently disabled, only used when rules +// are randomly disabled for testing. It is used to prevent propagating the +// RejectNullCols property when the corresponding column-pruning normalization +// rule is disabled. This prevents rule cycles during testing. +func DeriveRejectNullCols(in memo.RelExpr, disabledRules util.FastIntSet) opt.ColSet { // Lazily calculate and store the RejectNullCols value. relProps := in.Relational() if relProps.IsAvailable(props.RejectNullCols) { @@ -131,37 +137,66 @@ func DeriveRejectNullCols(in memo.RelExpr) opt.ColSet { case opt.InnerJoinOp, opt.InnerJoinApplyOp: // Pass through null-rejecting columns from both inputs. if in.Child(0).(memo.RelExpr).Relational().OuterCols.Empty() { - relProps.Rule.RejectNullCols.UnionWith(DeriveRejectNullCols(in.Child(0).(memo.RelExpr))) + relProps.Rule.RejectNullCols.UnionWith( + DeriveRejectNullCols(in.Child(0).(memo.RelExpr), disabledRules), + ) } if in.Child(1).(memo.RelExpr).Relational().OuterCols.Empty() { - relProps.Rule.RejectNullCols.UnionWith(DeriveRejectNullCols(in.Child(1).(memo.RelExpr))) + relProps.Rule.RejectNullCols.UnionWith( + DeriveRejectNullCols(in.Child(1).(memo.RelExpr), disabledRules), + ) } case opt.LeftJoinOp, opt.LeftJoinApplyOp: + if disabledRules.Contains(int(opt.RejectNullsLeftJoin)) { + // Avoid rule cycles. + break + } // Pass through null-rejection columns from left input, and request null- // rejection on right columns. if in.Child(0).(memo.RelExpr).Relational().OuterCols.Empty() { - relProps.Rule.RejectNullCols.UnionWith(DeriveRejectNullCols(in.Child(0).(memo.RelExpr))) + relProps.Rule.RejectNullCols.UnionWith( + DeriveRejectNullCols(in.Child(0).(memo.RelExpr), disabledRules), + ) } relProps.Rule.RejectNullCols.UnionWith(in.Child(1).(memo.RelExpr).Relational().OutputCols) case opt.RightJoinOp: + if disabledRules.Contains(int(opt.RejectNullsRightJoin)) { + // Avoid rule cycles. + break + } // Pass through null-rejection columns from right input, and request null- // rejection on left columns. relProps.Rule.RejectNullCols.UnionWith(in.Child(0).(memo.RelExpr).Relational().OutputCols) if in.Child(1).(memo.RelExpr).Relational().OuterCols.Empty() { - relProps.Rule.RejectNullCols.UnionWith(DeriveRejectNullCols(in.Child(1).(memo.RelExpr))) + relProps.Rule.RejectNullCols.UnionWith( + DeriveRejectNullCols(in.Child(1).(memo.RelExpr), disabledRules), + ) } case opt.FullJoinOp: + if disabledRules.Contains(int(opt.RejectNullsLeftJoin)) || + disabledRules.Contains(int(opt.RejectNullsRightJoin)) { + // Avoid rule cycles. + break + } // Request null-rejection on all output columns. relProps.Rule.RejectNullCols.UnionWith(relProps.OutputCols) case opt.GroupByOp, opt.ScalarGroupByOp: - relProps.Rule.RejectNullCols.UnionWith(deriveGroupByRejectNullCols(in)) + if disabledRules.Contains(int(opt.RejectNullsGroupBy)) { + // Avoid rule cycles. + break + } + relProps.Rule.RejectNullCols.UnionWith(deriveGroupByRejectNullCols(in, disabledRules)) case opt.ProjectOp: - relProps.Rule.RejectNullCols.UnionWith(deriveProjectRejectNullCols(in)) + if disabledRules.Contains(int(opt.RejectNullsProject)) { + // Avoid rule cycles. + break + } + relProps.Rule.RejectNullCols.UnionWith(deriveProjectRejectNullCols(in, disabledRules)) case opt.ScanOp: relProps.Rule.RejectNullCols.UnionWith(deriveScanRejectNullCols(in)) @@ -191,7 +226,7 @@ func DeriveRejectNullCols(in memo.RelExpr) opt.ColSet { // ignored because all rows in each group must have the same value for this // column, so it doesn't matter which rows are filtered. // -func deriveGroupByRejectNullCols(in memo.RelExpr) opt.ColSet { +func deriveGroupByRejectNullCols(in memo.RelExpr, disabledRules util.FastIntSet) opt.ColSet { input := in.Child(0).(memo.RelExpr) aggs := *in.Child(1).(*memo.AggregationsExpr) @@ -222,7 +257,7 @@ func deriveGroupByRejectNullCols(in memo.RelExpr) opt.ColSet { } savedInColID = inColID - if !DeriveRejectNullCols(input).Contains(inColID) { + if !DeriveRejectNullCols(input, disabledRules).Contains(inColID) { // Input has not requested null rejection on the input column. return opt.ColSet{} } @@ -278,8 +313,8 @@ func (c *CustomFuncs) MakeNullRejectFilters(nullRejectCols opt.ColSet) memo.Filt // child operator (for example, an outer join that may be simplified). This // prevents filters from getting in the way of other rules. // -func deriveProjectRejectNullCols(in memo.RelExpr) opt.ColSet { - rejectNullCols := DeriveRejectNullCols(in.Child(0).(memo.RelExpr)) +func deriveProjectRejectNullCols(in memo.RelExpr, disabledRules util.FastIntSet) opt.ColSet { + rejectNullCols := DeriveRejectNullCols(in.Child(0).(memo.RelExpr), disabledRules) projections := *in.Child(1).(*memo.ProjectionsExpr) var projectionsRejectCols opt.ColSet diff --git a/pkg/sql/opt/norm/testdata/rules/prune_cols b/pkg/sql/opt/norm/testdata/rules/prune_cols index 23ff4069ef25..ae3b393f0031 100644 --- a/pkg/sql/opt/norm/testdata/rules/prune_cols +++ b/pkg/sql/opt/norm/testdata/rules/prune_cols @@ -5164,3 +5164,78 @@ project │ └── filters (true) └── projections └── NULL [as="?column?":18] + +# Regression test for #86308. +exec-ddl +CREATE TABLE table86308 ( + col3_0 INET NOT NULL, + col3_1 INT4 NOT NULL, + col3_2 TIMETZ NOT NULL, + col3_3 FLOAT4 NOT NULL, + col3_4 BIT(17) NOT NULL, + col3_5 GEOGRAPHY NOT NULL, + col3_6 BIT(22) NOT NULL, + col3_7 STRING NOT NULL, + col3_8 UUID NOT NULL, + col3_9 FLOAT8 NOT NULL AS (col3_3 + 0.7530620098114014:::FLOAT8) STORED, + col3_10 STRING NOT NULL AS (lower(CAST(col3_0 AS STRING))) STORED, + col3_11 FLOAT8 NOT NULL AS (col3_3 + 0.8790965676307678:::FLOAT8) VIRTUAL, + col3_12 STRING NOT NULL AS (lower(CAST(col3_0 AS STRING))) VIRTUAL, + col3_13 STRING NOT NULL AS (lower(CAST(col3_6 AS STRING))) STORED, + col3_14 FLOAT8 NOT NULL AS (col3_3 + (-0.27059364318847656):::FLOAT8) STORED, + col3_15 STRING NOT NULL AS (lower(CAST(col3_5 AS STRING))) STORED, + rowid INT8 NOT VISIBLE NOT NULL DEFAULT unique_rowid(), + CONSTRAINT table3_pkey PRIMARY KEY (rowid ASC), + UNIQUE INDEX table3_col3_4_col3_11_col3_1_col3_0_col3_3_key (col3_4 DESC, col3_11 ASC, col3_1 DESC, col3_0 ASC, col3_3 DESC) STORING (col3_5, col3_8, col3_9, col3_10, col3_13) WHERE ((((col3_9 = 5e-324:::FLOAT8) OR (col3_1 = (-32768):::INT8)) OR (col3_15 = '':::STRING)) AND (col3_3 > 3.4028234663852886e+38:::FLOAT8)) AND (col3_12 != '""':::STRING) +); +---- + +# Before the fix, PruneGroupByCols would fire to push a Project below the +# GroupBy. Then, since PruneWindowInputCols was disabled, +# EliminateGroupByProject would remove the Project and the cycle would repeat. +norm disable=PruneWindowInputCols format=hide-all +SELECT concat_agg(tab_1739.col3_7::STRING ORDER BY tab_1739.col3_7 DESC)::STRING AS col_4664 +FROM table86308@[0] AS tab_1739 GROUP BY tab_1739.col3_7 HAVING bool_and(false::BOOL)::BOOL +---- +project + └── group-by (hash) + ├── select + │ ├── window partition=(8) + │ │ ├── window partition=(8) + │ │ │ ├── project + │ │ │ │ ├── scan table86308 + │ │ │ │ │ ├── computed column expressions + │ │ │ │ │ │ ├── col3_9 + │ │ │ │ │ │ │ └── col3_3 + 0.7530620098114014 + │ │ │ │ │ │ ├── col3_10 + │ │ │ │ │ │ │ └── lower(col3_0::STRING) + │ │ │ │ │ │ ├── col3_11 + │ │ │ │ │ │ │ └── col3_3 + 0.8790965676307678 + │ │ │ │ │ │ ├── col3_12 + │ │ │ │ │ │ │ └── lower(col3_0::STRING) + │ │ │ │ │ │ ├── col3_13 + │ │ │ │ │ │ │ └── lower(col3_6::STRING) + │ │ │ │ │ │ ├── col3_14 + │ │ │ │ │ │ │ └── col3_3 + -0.27059364318847656 + │ │ │ │ │ │ └── col3_15 + │ │ │ │ │ │ └── lower(col3_5::STRING) + │ │ │ │ │ └── partial index predicates + │ │ │ │ │ └── table3_col3_4_col3_11_col3_1_col3_0_col3_3_key: filters + │ │ │ │ │ ├── ((col3_9 = 5e-324) OR (col3_1 = -32768)) OR (col3_15 = '') + │ │ │ │ │ ├── col3_3 > 3.4028234663852886e+38 + │ │ │ │ │ └── lower(col3_0::STRING) != '""' + │ │ │ │ └── projections + │ │ │ │ ├── false + │ │ │ │ ├── col3_3 + 0.8790965676307678 + │ │ │ │ └── lower(col3_0::STRING) + │ │ │ └── windows + │ │ │ └── concat-agg [frame="range from unbounded to unbounded"] + │ │ │ └── col3_7 + │ │ └── windows + │ │ └── bool-and [frame="range from unbounded to unbounded"] + │ │ └── column21 + │ └── filters + │ └── bool_and + └── aggregations + └── const-agg + └── concat_agg diff --git a/pkg/sql/opt/testutils/opttester/forcing_opt.go b/pkg/sql/opt/testutils/opttester/forcing_opt.go index 6769ab104f07..3866ea7d8edf 100644 --- a/pkg/sql/opt/testutils/opttester/forcing_opt.go +++ b/pkg/sql/opt/testutils/opttester/forcing_opt.go @@ -70,6 +70,7 @@ func newForcingOptimizer( fo.o.Factory().FoldingControl().AllowStableFolds() fo.coster.Init(&fo.o, &fo.groups) fo.o.SetCoster(&fo.coster) + fo.o.Factory().SetDisabledRules(tester.Flags.DisableRules) fo.o.NotifyOnMatchedRule(func(ruleName opt.RuleName) bool { if ignoreNormRules && ruleName.IsNormalize() { diff --git a/pkg/sql/opt/testutils/opttester/opt_tester.go b/pkg/sql/opt/testutils/opttester/opt_tester.go index 46b2c1fe4122..6e8ac7e9ea8d 100644 --- a/pkg/sql/opt/testutils/opttester/opt_tester.go +++ b/pkg/sql/opt/testutils/opttester/opt_tester.go @@ -839,10 +839,10 @@ func fillInLazyProps(e opt.Expr) { rel = rel.FirstExpr() // Derive columns that are candidates for pruning. - norm.DerivePruneCols(rel) + norm.DerivePruneCols(rel, util.FastIntSet{} /* disabledRules */) // Derive columns that are candidates for null rejection. - norm.DeriveRejectNullCols(rel) + norm.DeriveRejectNullCols(rel, util.FastIntSet{} /* disabledRules */) // Make sure the interesting orderings are calculated. ordering.DeriveInterestingOrderings(rel) @@ -1252,6 +1252,7 @@ func (ot *OptTester) Expr() (opt.Expr, error) { func (ot *OptTester) ExprNorm() (opt.Expr, error) { var f norm.Factory f.Init(&ot.evalCtx, ot.catalog) + f.SetDisabledRules(ot.Flags.DisableRules) if !ot.Flags.NoStableFolds { f.FoldingControl().AllowStableFolds() @@ -2187,6 +2188,7 @@ func (ot *OptTester) buildExpr(factory *norm.Factory) error { func (ot *OptTester) makeOptimizer() *xform.Optimizer { var o xform.Optimizer o.Init(ot.ctx, &ot.evalCtx, ot.catalog) + o.Factory().SetDisabledRules(ot.Flags.DisableRules) o.NotifyOnAppliedRule(func(ruleName opt.RuleName, source, target opt.Expr) { // Exploration rules are marked as "applied" if they generate one or // more new expressions. diff --git a/pkg/sql/opt/xform/optimizer.go b/pkg/sql/opt/xform/optimizer.go index 2f2c47d84326..c10404c0f80a 100644 --- a/pkg/sql/opt/xform/optimizer.go +++ b/pkg/sql/opt/xform/optimizer.go @@ -95,10 +95,6 @@ type Optimizer struct { // It can be set via a call to the NotifyOnAppliedRule method. appliedRule AppliedRuleFunc - // disabledRules is a set of rules that are not allowed to run, used for - // testing. - disabledRules RuleSet - // JoinOrderBuilder adds new join orderings to the memo. jb JoinOrderBuilder @@ -152,7 +148,7 @@ func (o *Optimizer) Init(ctx context.Context, evalCtx *eval.Context, catalog cat o.defaultCoster.Init(evalCtx, o.mem, costPerturbation, o.rng) o.coster = &o.defaultCoster if disableRuleProbability > 0 { - o.disableRules(disableRuleProbability) + o.disableRulesRandom(disableRuleProbability) } } @@ -966,8 +962,8 @@ func (a *groupStateAlloc) allocate() *groupState { return state } -// disableRules disables rules with the given probability for testing. -func (o *Optimizer) disableRules(probability float64) { +// disableRulesRandom disables rules with the given probability for testing. +func (o *Optimizer) disableRulesRandom(probability float64) { essentialRules := util.MakeFastIntSet( // Needed to prevent constraint building from failing. int(opt.NormalizeInConst), @@ -1005,6 +1001,7 @@ func (o *Optimizer) disableRules(probability float64) { int(opt.EliminateSelect), ) + var disabledRules RuleSet for i := opt.RuleName(1); i < opt.NumRuleNames; i++ { var r float64 if o.rng == nil { @@ -1013,12 +1010,14 @@ func (o *Optimizer) disableRules(probability float64) { r = o.rng.Float64() } if r < probability && !essentialRules.Contains(int(i)) { - o.disabledRules.Add(int(i)) + disabledRules.Add(int(i)) } } + o.f.SetDisabledRules(disabledRules) + o.NotifyOnMatchedRule(func(ruleName opt.RuleName) bool { - if o.disabledRules.Contains(int(ruleName)) { + if disabledRules.Contains(int(ruleName)) { log.Infof(o.evalCtx.Context, "disabled rule matched: %s", ruleName.String()) return false }