-
Notifications
You must be signed in to change notification settings - Fork 5.8k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Optimizer: derive TopN from filter on row number #41209
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,122 @@ | ||
// Copyright 2023 PingCAP, Inc. | ||
// | ||
// 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 core | ||
|
||
import ( | ||
"context" | ||
"fmt" | ||
|
||
"github.com/pingcap/tidb/expression" | ||
"github.com/pingcap/tidb/parser/ast" | ||
"github.com/pingcap/tidb/planner/util" | ||
) | ||
|
||
// deriveTopNFromWindow pushes down the topN or limit. In the future we will remove the limit from `requiredProperty` in CBO phase. | ||
type deriveTopNFromWindow struct { | ||
} | ||
|
||
func appendDerivedTopNTrace(topN LogicalPlan, opt *logicalOptimizeOp) { | ||
child := topN.Children()[0] | ||
action := func() string { | ||
return fmt.Sprintf("%v_%v top N added below %v_%v ", topN.TP(), topN.ID(), child.TP(), child.ID()) | ||
} | ||
reason := func() string { | ||
return fmt.Sprintf("%v filter on row number", topN.TP()) | ||
} | ||
opt.appendStepToCurrent(topN.ID(), topN.TP(), reason, action) | ||
} | ||
|
||
/* | ||
Check the following pattern of filter over row number window function: | ||
- Filter is simple condition of row_number < value or row_number <= value | ||
- The window function is a simple row number | ||
- With default frame: rows between current row and current row. Check is not necessary since | ||
current row is only frame applicable to row number | ||
- No partition | ||
- Child is a data source. | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Customer's case there is filter on top of data source, but we can do it in nest PR. |
||
*/ | ||
func windowIsTopN(p *LogicalSelection) (bool, uint64) { | ||
// Check if child is window function. | ||
child, isLogicalWindow := p.Children()[0].(*LogicalWindow) | ||
if !isLogicalWindow { | ||
return false, 0 | ||
} | ||
|
||
if len(p.Conditions) != 1 { | ||
return false, 0 | ||
} | ||
|
||
// Check if filter is column < constant or column <= constant. If it is in this form find column and constant. | ||
column, limitValue := expression.FindUpperBound(p.Conditions[0]) | ||
if column == nil || limitValue <= 0 { | ||
return false, 0 | ||
} | ||
|
||
// Check if filter on window function | ||
windowColumns := child.GetWindowResultColumns() | ||
if len(windowColumns) != 1 || !(column.Equal(p.ctx, windowColumns[0])) { | ||
return false, 0 | ||
} | ||
|
||
grandChild := child.Children()[0] | ||
_, isDataSource := grandChild.(*DataSource) | ||
if !isDataSource { | ||
return false, 0 | ||
} | ||
if len(child.WindowFuncDescs) == 1 && child.WindowFuncDescs[0].Name == "row_number" && len(child.PartitionBy) == 0 && | ||
child.Frame.Type == ast.Rows && child.Frame.Start.Type == ast.CurrentRow && child.Frame.End.Type == ast.CurrentRow { | ||
return true, uint64(limitValue) | ||
} | ||
return false, 0 | ||
} | ||
|
||
func (s *deriveTopNFromWindow) optimize(_ context.Context, p LogicalPlan, opt *logicalOptimizeOp) (LogicalPlan, error) { | ||
return p.deriveTopN(opt), nil | ||
} | ||
|
||
func (s *baseLogicalPlan) deriveTopN(opt *logicalOptimizeOp) LogicalPlan { | ||
p := s.self | ||
for i, child := range p.Children() { | ||
newChild := child.deriveTopN(opt) | ||
p.SetChild(i, newChild) | ||
} | ||
return p | ||
} | ||
|
||
func (s *LogicalSelection) deriveTopN(opt *logicalOptimizeOp) LogicalPlan { | ||
p := s.self.(*LogicalSelection) | ||
windowIsTopN, limitValue := windowIsTopN(p) | ||
if windowIsTopN { | ||
child := p.Children()[0].(*LogicalWindow) | ||
grandChild := child.Children()[0].(*DataSource) | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. You can use There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I think a method (if any) to get struct members is better than accessing struct members directly. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Makes sense. |
||
// Build order by for derived Limit | ||
byItems := make([]*util.ByItems, 0, len(child.OrderBy)) | ||
for _, col := range child.OrderBy { | ||
byItems = append(byItems, &util.ByItems{Expr: col.Col, Desc: col.Desc}) | ||
} | ||
// Build derived Limit | ||
derivedTopN := LogicalTopN{Count: limitValue, ByItems: byItems}.Init(grandChild.ctx, grandChild.blockOffset) | ||
derivedTopN.SetChildren(grandChild) | ||
/* return datasource->topN->window */ | ||
child.SetChildren(derivedTopN) | ||
appendDerivedTopNTrace(child, opt) | ||
return child | ||
} | ||
return p | ||
} | ||
|
||
func (*deriveTopNFromWindow) name() string { | ||
return "derive_topn_from_window" | ||
} |
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,85 @@ | ||
// Copyright 2023 PingCAP, Inc. | ||
// | ||
// 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 core_test | ||
|
||
import ( | ||
"testing" | ||
|
||
plannercore "github.com/pingcap/tidb/planner/core" | ||
"github.com/pingcap/tidb/testkit" | ||
"github.com/pingcap/tidb/testkit/testdata" | ||
) | ||
|
||
// Rule should bot be applied | ||
func TestPushDerivedTopnNegative(t *testing.T) { | ||
store := testkit.CreateMockStore(t) | ||
tk := testkit.NewTestKit(t, store) | ||
tk.MustExec("use test") | ||
tk.MustExec("drop table if exists employee") | ||
tk.MustExec("create table t(a int, b int)") | ||
tk.MustExec("insert into t values(1,1)") | ||
tk.MustExec("insert into t values(2,1)") | ||
tk.MustExec("insert into t values(3,2)") | ||
tk.MustExec("insert into t values(4,2)") | ||
tk.MustExec("insert into t values(5,2)") | ||
var input Input | ||
var output []struct { | ||
SQL string | ||
Plan []string | ||
} | ||
suiteData := plannercore.GetDerivedTopNSuiteData() | ||
suiteData.LoadTestCases(t, &input, &output) | ||
for i, sql := range input { | ||
plan := tk.MustQuery("explain format = 'brief' " + sql) | ||
testdata.OnRecord(func() { | ||
output[i].SQL = sql | ||
output[i].Plan = testdata.ConvertRowsToStrings(plan.Rows()) | ||
}) | ||
plan.Check(testkit.Rows(output[i].Plan...)) | ||
} | ||
} | ||
|
||
// Rule should be applied | ||
func TestPushDerivedTopnPositive(t *testing.T) { | ||
store := testkit.CreateMockStore(t) | ||
tk := testkit.NewTestKit(t, store) | ||
tk.MustExec("use test") | ||
tk.MustExec("drop table if exists employee") | ||
tk.MustExec("create table t(a int, b int)") | ||
tk.MustExec("insert into t values(1,1)") | ||
tk.MustExec("insert into t values(2,1)") | ||
tk.MustExec("insert into t values(3,2)") | ||
tk.MustExec("insert into t values(4,2)") | ||
tk.MustExec("insert into t values(5,2)") | ||
var input Input | ||
var output []struct { | ||
SQL string | ||
Plan []string | ||
Res []string | ||
} | ||
suiteData := plannercore.GetDerivedTopNSuiteData() | ||
suiteData.LoadTestCases(t, &input, &output) | ||
for i, sql := range input { | ||
plan := tk.MustQuery("explain format = 'brief' " + sql) | ||
res := tk.MustQuery(sql) | ||
testdata.OnRecord(func() { | ||
output[i].SQL = sql | ||
output[i].Plan = testdata.ConvertRowsToStrings(plan.Rows()) | ||
output[i].Res = testdata.ConvertRowsToStrings(res.Rows()) | ||
}) | ||
plan.Check(testkit.Rows(output[i].Plan...)) | ||
res.Check(testkit.Rows(output[i].Res...)) | ||
} | ||
} |
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,29 @@ | ||
[ | ||
{ | ||
"name": "TestPushDerivedTopnNegative", | ||
"cases":[ | ||
"select row_number() over (partition by a) from t -- pattern missing filter on row number", | ||
"select * from (select rank() over () as rank_order from t) DT where rank_order <= 3 -- rank not supported in pattern yet", | ||
"select * from (select row_number() over (partition by a) as rownumber from t) DT where rownumber <= 3 -- pattern is applicable but partition by is not supported yet", | ||
"select * from (select row_number() over () as rownumber1, row_number() over (partition by a) as rownumber2 from t) DT where rownumber1 <= 3 -- pattern not applicable with multiple window functions", | ||
"select * from (select b, row_number() over () as rownumber from t) DT where rownumber <= 3 and b > 5 -- pattern is not applicable with complex filter on top of window", | ||
"select * from (select b, row_number() over () as rownumber from t) DT where rownumber > 3 -- pattern is not applicable with filter is not < or <=", | ||
"select * from (select a,b, row_number() over () as rownumber from t) DT where a > b -- pattern is not applicable with filter is not < or <=", | ||
"select * from (select a,b, row_number() over () as rownumber from t) DT where a <= 3 -- pattern is not applicable with filter is not on row number", | ||
"select * from (select a,b, row_number() over () as rownumber from t) DT where 3 >= rownumber -- pattern is not applicable with filter is not < or <=", | ||
"select * from (select a,b, row_number() over () as rownumber from t) DT where rownumber <= -4 -- pattern is not applicable with filter constant negative", | ||
"select * from (select row_number() over () as rownumber from t) DT where rownumber <= 3 and rownumber >= 2 -- pattern is not applicable with complex filter" | ||
] | ||
}, | ||
{ | ||
"name": "TestPushDerivedTopnPositive", | ||
"cases":[ | ||
"select * from (select a,b, row_number() over (order by a) as rownumber from t) DT where rownumber <= 3.5 -- pattern is applicable with N rounded down to an integer", | ||
"select * from (select row_number() over (order by a) as rownumber from t) DT where rownumber <= 3 -- pattern is applicable", | ||
"select * from (select row_number() over (order by a) as rownumber from t) DT where rownumber < 3 -- pattern is applicable", | ||
"select * from (select row_number() over(rows between 1 preceding and 1 following) as rownumber from t) DT where rownumber <= 3 -- pattern is applicable", | ||
"select * from (select a,row_number() over (order by a desc) as rownumber,b from t) DT where rownumber <= 3 -- pattern is applicable", | ||
"select count(*) from (select * from (select a,row_number() over (order by b) as rownumber,b from t) DT1 where rownumber <= 1) DT2 -- pattern is applicable" | ||
] | ||
} | ||
] |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Should we support the patten like
constant > column
orconstant >= column
?There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Theoretically these are applicable but did not cover them since the common pattern is < and <= but we can easily add this and other cases like "= 1"