Skip to content

Commit

Permalink
sql: add optimizer_use_limit_ordering_for_streaming_group_by session …
Browse files Browse the repository at this point in the history
…setting

Fixes #93410

This commit adds the `optimizer_use_limit_ordering_for_streaming_group_by`
session setting that defaults to `true`. When `true`, an exploration
rule which uses the ordering specified in a
`SELECT ... GROUP BY ... ORDER BY ... LIMIT n;` statement as an
interesting ordering to require from the input to the group-by
expression, possibly eliminating a top-k operation.  When `false`, the
exploration rule is disabled.

Release note: None
  • Loading branch information
Mark Sirek committed Jan 5, 2023
1 parent c533c2b commit 63d9aed
Show file tree
Hide file tree
Showing 12 changed files with 164 additions and 6 deletions.
4 changes: 4 additions & 0 deletions pkg/sql/exec_util.go
Original file line number Diff line number Diff line change
Expand Up @@ -3450,6 +3450,10 @@ func (m *sessionDataMutator) SetOptimizerUseImprovedDisjunctionStats(val bool) {
m.data.OptimizerUseImprovedDisjunctionStats = val
}

func (m *sessionDataMutator) SetOptimizerUseLimitOrderingForStreamingGroupBy(val bool) {
m.data.OptimizerUseLimitOrderingForStreamingGroupBy = val
}

// Utility functions related to scrubbing sensitive information on SQL Stats.

// quantizeCounts ensures that the Count field in the
Expand Down
1 change: 1 addition & 0 deletions pkg/sql/logictest/testdata/logic_test/information_schema
Original file line number Diff line number Diff line change
Expand Up @@ -4846,6 +4846,7 @@ optimizer on
optimizer_use_forecasts on
optimizer_use_histograms on
optimizer_use_improved_disjunction_stats on
optimizer_use_limit_ordering_for_streaming_group_by on
optimizer_use_multicol_stats on
optimizer_use_not_visible_indexes off
override_multi_region_zone_config off
Expand Down
3 changes: 3 additions & 0 deletions pkg/sql/logictest/testdata/logic_test/pg_catalog
Original file line number Diff line number Diff line change
Expand Up @@ -2851,6 +2851,7 @@ opt_split_scan_limit 2048 NULL
optimizer_use_forecasts on NULL NULL NULL string
optimizer_use_histograms on NULL NULL NULL string
optimizer_use_improved_disjunction_stats on NULL NULL NULL string
optimizer_use_limit_ordering_for_streaming_group_by on NULL NULL NULL string
optimizer_use_multicol_stats on NULL NULL NULL string
optimizer_use_not_visible_indexes off NULL NULL NULL string
override_multi_region_zone_config off NULL NULL NULL string
Expand Down Expand Up @@ -2993,6 +2994,7 @@ opt_split_scan_limit 2048 NULL
optimizer_use_forecasts on NULL user NULL on on
optimizer_use_histograms on NULL user NULL on on
optimizer_use_improved_disjunction_stats on NULL user NULL on on
optimizer_use_limit_ordering_for_streaming_group_by on NULL user NULL on on
optimizer_use_multicol_stats on NULL user NULL on on
optimizer_use_not_visible_indexes off NULL user NULL off off
override_multi_region_zone_config off NULL user NULL off off
Expand Down Expand Up @@ -3133,6 +3135,7 @@ optimizer NULL NULL NULL
optimizer_use_forecasts NULL NULL NULL NULL NULL
optimizer_use_histograms NULL NULL NULL NULL NULL
optimizer_use_improved_disjunction_stats NULL NULL NULL NULL NULL
optimizer_use_limit_ordering_for_streaming_group_by NULL NULL NULL NULL NULL
optimizer_use_multicol_stats NULL NULL NULL NULL NULL
optimizer_use_not_visible_indexes NULL NULL NULL NULL NULL
override_multi_region_zone_config NULL NULL NULL NULL NULL
Expand Down
1 change: 1 addition & 0 deletions pkg/sql/logictest/testdata/logic_test/show_source
Original file line number Diff line number Diff line change
Expand Up @@ -109,6 +109,7 @@ opt_split_scan_limit 2048
optimizer_use_forecasts on
optimizer_use_histograms on
optimizer_use_improved_disjunction_stats on
optimizer_use_limit_ordering_for_streaming_group_by on
optimizer_use_multicol_stats on
optimizer_use_not_visible_indexes off
override_multi_region_zone_config off
Expand Down
5 changes: 4 additions & 1 deletion pkg/sql/opt/memo/memo.go
Original file line number Diff line number Diff line change
Expand Up @@ -160,6 +160,7 @@ type Memo struct {
variableInequalityLookupJoinEnabled bool
allowOrdinalColumnReferences bool
useImprovedDisjunctionStats bool
useLimitOrderingForStreamingGroupBy bool

// curRank is the highest currently in-use scalar expression rank.
curRank opt.ScalarRank
Expand Down Expand Up @@ -215,6 +216,7 @@ func (m *Memo) Init(ctx context.Context, evalCtx *eval.Context) {
variableInequalityLookupJoinEnabled: evalCtx.SessionData().VariableInequalityLookupJoinEnabled,
allowOrdinalColumnReferences: evalCtx.SessionData().AllowOrdinalColumnReferences,
useImprovedDisjunctionStats: evalCtx.SessionData().OptimizerUseImprovedDisjunctionStats,
useLimitOrderingForStreamingGroupBy: evalCtx.SessionData().OptimizerUseLimitOrderingForStreamingGroupBy,
}
m.metadata.Init()
m.logPropsBuilder.init(ctx, evalCtx, m)
Expand Down Expand Up @@ -353,7 +355,8 @@ func (m *Memo) IsStale(
m.enforceHomeRegion != evalCtx.SessionData().EnforceHomeRegion ||
m.variableInequalityLookupJoinEnabled != evalCtx.SessionData().VariableInequalityLookupJoinEnabled ||
m.allowOrdinalColumnReferences != evalCtx.SessionData().AllowOrdinalColumnReferences ||
m.useImprovedDisjunctionStats != evalCtx.SessionData().OptimizerUseImprovedDisjunctionStats {
m.useImprovedDisjunctionStats != evalCtx.SessionData().OptimizerUseImprovedDisjunctionStats ||
m.useLimitOrderingForStreamingGroupBy != evalCtx.SessionData().OptimizerUseLimitOrderingForStreamingGroupBy {
return true, nil
}

Expand Down
6 changes: 6 additions & 0 deletions pkg/sql/opt/memo/memo_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -306,6 +306,12 @@ func TestMemoIsStale(t *testing.T) {
evalCtx.SessionData().VariableInequalityLookupJoinEnabled = false
notStale()

// Stale use limit ordering for streaming group by.
evalCtx.SessionData().OptimizerUseLimitOrderingForStreamingGroupBy = true
stale()
evalCtx.SessionData().OptimizerUseLimitOrderingForStreamingGroupBy = false
notStale()

// Stale testing_optimizer_random_seed.
evalCtx.SessionData().TestingOptimizerRandomSeed = 100
stale()
Expand Down
6 changes: 1 addition & 5 deletions pkg/sql/opt/norm/factory.go
Original file line number Diff line number Diff line change
Expand Up @@ -189,11 +189,7 @@ func (f *Factory) DisableOptimizations() {
// DisableOptimizationRules disables a specific set of transformation rules.
func (f *Factory) DisableOptimizationRules(disabledRules intsets.Fast) {
f.NotifyOnMatchedRule(func(rule opt.RuleName) bool {
if disabledRules.Contains(int(rule)) {
return false
} else {
return true
}
return !disabledRules.Contains(int(rule))
})
}

Expand Down
1 change: 1 addition & 0 deletions pkg/sql/opt/testutils/opttester/opt_tester.go
Original file line number Diff line number Diff line change
Expand Up @@ -293,6 +293,7 @@ func New(catalog cat.Catalog, sql string) *OptTester {
ot.evalCtx.SessionData().OptSplitScanLimit = tabledesc.MaxBucketAllowed
ot.evalCtx.SessionData().VariableInequalityLookupJoinEnabled = true
ot.evalCtx.SessionData().OptimizerUseImprovedDisjunctionStats = true
ot.evalCtx.SessionData().OptimizerUseLimitOrderingForStreamingGroupBy = true

return ot
}
Expand Down
4 changes: 4 additions & 0 deletions pkg/sql/opt/xform/groupby_funcs.go
Original file line number Diff line number Diff line change
Expand Up @@ -228,6 +228,10 @@ func (c *CustomFuncs) GenerateStreamingGroupByLimitOrderingHint(
groupingCols opt.ColSet,
newOrdering props.OrderingChoice,
) {
if !c.e.evalCtx.SessionData().OptimizerUseLimitOrderingForStreamingGroupBy {
// This transformation rule is explicitly disabled.
return
}
newPrivate := *private
newPrivate.Ordering = newOrdering
newPrivate.GroupingCols = groupingCols
Expand Down
115 changes: 115 additions & 0 deletions pkg/sql/opt/xform/testdata/rules/limit
Original file line number Diff line number Diff line change
Expand Up @@ -3043,4 +3043,119 @@ project
│ └── count-rows [as=count_rows:27]
└── 20

# A query which normally would produce a streaming group-by with no TopK
# operation produces a TopK if the GenerateStreamingGroupByLimitOrderingHint
# rule is disabled.
opt expect-not=GenerateStreamingGroupByLimitOrderingHint set=optimizer_use_limit_ordering_for_streaming_group_by=false
SELECT
DISTINCT
t1.col1,
t1.col2,
t1.col3,
t1.col4,
t1.col5,
t1.col6,
t1.col7,
t1.col8,
t1.col9,
t1.col10,
t1.col11,
t1.col12,
t1.col13,
t1.col14,
t1.col15,
t1.col16,
array_remove(array_agg(t2.col6), NULL) AS col6s
FROM
t93410 AS t1
LEFT OUTER JOIN t93410_2
AS t2 USING (col1, col2, col3, col4, col5)
WHERE
t1.col2 = 1
AND t1.col1 = 1
GROUP BY
t1.col1,
t1.col2,
t1.col3,
t1.col4,
t1.col5
ORDER BY
col9 DESC, t1.col5 DESC
LIMIT 20
----
project
├── columns: col1:1!null col2:2!null col3:3!null col4:4!null col5:5!null col6:6!null col7:7!null col8:8!null col9:9!null col10:10!null col11:11!null col12:12 col13:13 col14:14!null col15:15 col16:16 col6s:28
├── cardinality: [0 - 20]
├── immutable
├── key: (5)
├── fd: ()-->(1,2), (5)-->(3,4,6-16,28)
├── ordering: -9,-5 opt(1,2) [actual: -9,-5]
├── top-k
│ ├── columns: t1.col1:1!null t1.col2:2!null t1.col3:3!null t1.col4:4!null t1.col5:5!null t1.col6:6!null col7:7!null col8:8!null col9:9!null col10:10!null col11:11!null col12:12 col13:13 col14:14!null col15:15 col16:16 array_agg:27
│ ├── internal-ordering: -9,-5 opt(1,2)
│ ├── k: 20
│ ├── cardinality: [0 - 20]
│ ├── key: (5)
│ ├── fd: ()-->(1,2), (5)-->(1-4,6-16,27)
│ ├── ordering: -9,-5 opt(1,2) [actual: -9,-5]
│ └── group-by (hash)
│ ├── columns: t1.col1:1!null t1.col2:2!null t1.col3:3!null t1.col4:4!null t1.col5:5!null t1.col6:6!null col7:7!null col8:8!null col9:9!null col10:10!null col11:11!null col12:12 col13:13 col14:14!null col15:15 col16:16 array_agg:27
│ ├── grouping columns: t1.col5:5!null
│ ├── key: (5)
│ ├── fd: ()-->(1,2), (5)-->(1-4,6-16,27)
│ ├── left-join (merge)
│ │ ├── columns: t1.col1:1!null t1.col2:2!null t1.col3:3!null t1.col4:4!null t1.col5:5!null t1.col6:6!null col7:7!null col8:8!null col9:9!null col10:10!null col11:11!null col12:12 col13:13 col14:14!null col15:15 col16:16 t2.col1:19 t2.col2:20 t2.col3:21 t2.col4:22 t2.col5:23 t2.col6:24
│ │ ├── left ordering: +3,+4,+5,+1,+2
│ │ ├── right ordering: +21,+22,+23,+19,+20
│ │ ├── key: (5,21,23,24)
│ │ ├── fd: ()-->(1,2), (5)-->(3,4,6-16), (21,23,24)-->(22), (5,21,23,24)-->(19,20)
│ │ ├── scan t93410 [as=t1]
│ │ │ ├── columns: t1.col1:1!null t1.col2:2!null t1.col3:3!null t1.col4:4!null t1.col5:5!null t1.col6:6!null col7:7!null col8:8!null col9:9!null col10:10!null col11:11!null col12:12 col13:13 col14:14!null col15:15 col16:16
│ │ │ ├── constraint: /1/2/3/4/5: [/1/1 - /1/1]
│ │ │ ├── key: (5)
│ │ │ ├── fd: ()-->(1,2), (5)-->(3,4,6-16)
│ │ │ └── ordering: +3,+4,+5 opt(1,2) [actual: +3,+4,+5]
│ │ ├── scan t93410_2@t93410_2_col1_col2_col3_col4_col5_idx [as=t2]
│ │ │ ├── columns: t2.col1:19!null t2.col2:20!null t2.col3:21!null t2.col4:22!null t2.col5:23!null t2.col6:24!null
│ │ │ ├── constraint: /19/20/21/22/23/24: [/1/1 - /1/1]
│ │ │ ├── key: (21,23,24)
│ │ │ ├── fd: ()-->(19,20), (21,23,24)-->(22)
│ │ │ └── ordering: +21,+22,+23 opt(19,20) [actual: +21,+22,+23]
│ │ └── filters (true)
│ └── aggregations
│ ├── array-agg [as=array_agg:27, outer=(24)]
│ │ └── t2.col6:24
│ ├── const-agg [as=t1.col1:1, outer=(1)]
│ │ └── t1.col1:1
│ ├── const-agg [as=t1.col2:2, outer=(2)]
│ │ └── t1.col2:2
│ ├── const-agg [as=t1.col3:3, outer=(3)]
│ │ └── t1.col3:3
│ ├── const-agg [as=t1.col4:4, outer=(4)]
│ │ └── t1.col4:4
│ ├── const-agg [as=t1.col6:6, outer=(6)]
│ │ └── t1.col6:6
│ ├── const-agg [as=col7:7, outer=(7)]
│ │ └── col7:7
│ ├── const-agg [as=col8:8, outer=(8)]
│ │ └── col8:8
│ ├── const-agg [as=col9:9, outer=(9)]
│ │ └── col9:9
│ ├── const-agg [as=col10:10, outer=(10)]
│ │ └── col10:10
│ ├── const-agg [as=col11:11, outer=(11)]
│ │ └── col11:11
│ ├── const-agg [as=col12:12, outer=(12)]
│ │ └── col12:12
│ ├── const-agg [as=col13:13, outer=(13)]
│ │ └── col13:13
│ ├── const-agg [as=col14:14, outer=(14)]
│ │ └── col14:14
│ ├── const-agg [as=col15:15, outer=(15)]
│ │ └── col15:15
│ └── const-agg [as=col16:16, outer=(16)]
│ └── col16:16
└── projections
└── array_remove(array_agg:27, NULL) [as=col6s:28, outer=(27), immutable]

# End Regression Test for #93410
7 changes: 7 additions & 0 deletions pkg/sql/sessiondatapb/local_only_session_data.proto
Original file line number Diff line number Diff line change
Expand Up @@ -317,6 +317,13 @@ message LocalOnlySessionData {
// filters.
bool optimizer_use_improved_disjunction_stats = 86;

// OptimizerUseLimitOrderingForStreamingGroupBy enables the exploration rule
// which optimizes 'SELECT ... GROUP BY ... ORDER BY ... LIMIT n' queries.
// The rule uses the required ordering in the limit expression to inform an
// interesting ordering to require from the input to the group-by expression.
// This can potentially eliminate a top-k operation.
bool optimizer_use_limit_ordering_for_streaming_group_by = 88;

///////////////////////////////////////////////////////////////////////////
// WARNING: consider whether a session parameter you're adding needs to //
// be propagated to the remote nodes. If so, that parameter should live //
Expand Down
17 changes: 17 additions & 0 deletions pkg/sql/vars.go
Original file line number Diff line number Diff line change
Expand Up @@ -2404,6 +2404,23 @@ var varGen = map[string]sessionVar{
},
GlobalDefault: globalTrue,
},

// CockroachDB extension.
`optimizer_use_limit_ordering_for_streaming_group_by`: {
GetStringVal: makePostgresBoolGetStringValFn(`optimizer_use_limit_ordering_for_streaming_group_by`),
Set: func(_ context.Context, m sessionDataMutator, s string) error {
b, err := paramparse.ParseBoolVar("optimizer_use_limit_ordering_for_streaming_group_by", s)
if err != nil {
return err
}
m.SetOptimizerUseLimitOrderingForStreamingGroupBy(b)
return nil
},
Get: func(evalCtx *extendedEvalContext, _ *kv.Txn) (string, error) {
return formatBoolAsPostgresSetting(evalCtx.SessionData().OptimizerUseLimitOrderingForStreamingGroupBy), nil
},
GlobalDefault: globalTrue,
},
}

// We want test coverage for this on and off so make it metamorphic.
Expand Down

0 comments on commit 63d9aed

Please sign in to comment.