Skip to content

Commit

Permalink
Optimizer: Add partition by support for derived TopN(filter on row_nu…
Browse files Browse the repository at this point in the history
…mber)
  • Loading branch information
ghazalfamilyusa committed Feb 18, 2023
1 parent d2d91b5 commit 0e10f17
Show file tree
Hide file tree
Showing 15 changed files with 650 additions and 147 deletions.
103 changes: 92 additions & 11 deletions planner/core/casetest/rule_derive_topn_from_window_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -17,22 +17,40 @@ package casetest
import (
"testing"

"github.com/pingcap/tidb/domain"
"github.com/pingcap/tidb/parser/model"
"github.com/pingcap/tidb/planner/core/internal"
"github.com/pingcap/tidb/testkit"
"github.com/pingcap/tidb/testkit/testdata"
"github.com/stretchr/testify/require"
)

// Rule should bot be applied
func setTiFlashReplica(t *testing.T, dom *domain.Domain, dbName, tableName string) {
is := dom.InfoSchema()
db, exists := is.SchemaByName(model.NewCIStr(dbName))
require.True(t, exists)
for _, tblInfo := range db.Tables {
if tblInfo.Name.L == tableName {
tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{
Count: 1,
Available: true,
}
}
}
}

// Rule should bot be applied for TiKV.
func TestPushDerivedTopnNegative(t *testing.T) {
store := testkit.CreateMockStore(t)
tk := testkit.NewTestKit(t, store)
tk.MustExec("set tidb_opt_derive_topn=1")
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)")
tk.MustExec("drop table if exists t")
tk.MustExec("create table t(a int, b int, c int, primary key(b,a))")
tk.MustExec("drop table if exists tt")
tk.MustExec("create table tt(a int, b int, c int, primary key(b,a) nonclustered)")
tk.MustExec("drop table if exists ti")
tk.MustExec("create table ti(a int, b int, c int unique)")
var input Input
var output []struct {
SQL string
Expand All @@ -50,18 +68,56 @@ func TestPushDerivedTopnNegative(t *testing.T) {
}
}

// Rule should be applied
// TiFlash cases. TopN pushed down to storage only when no partition by.
func TestPushDerivedTopnFlash(t *testing.T) {
store := testkit.CreateMockStore(t)
tk := testkit.NewTestKit(t, store)
dom := domain.GetDomain(tk.Session())

tk.MustExec("set tidb_opt_derive_topn=1")
tk.MustExec("use test")
tk.MustExec("drop table if exists t")
tk.MustExec("create table t(a int, b int, primary key(b,a))")
internal.SetTiFlashReplica(t, dom, "test", "t")
tk.MustExec("set tidb_enforce_mpp=1")
tk.MustExec("set @@session.tidb_allow_mpp=ON;")
var input Input
var output []struct {
SQL string
Plan []string
}
suiteData := 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 for TiKV.
func TestPushDerivedTopnPositive(t *testing.T) {
store := testkit.CreateMockStore(t)
tk := testkit.NewTestKit(t, store)
tk.MustExec("set tidb_opt_derive_topn=1")
tk.MustExec("use test")
tk.MustExec("drop table if exists employee")
tk.MustExec("create table t(a int, b int)")
tk.MustExec("drop table if exists t")
tk.MustExec("create table t(a int, b int, primary key(b,a))")
tk.MustExec("drop table if exists tt")
tk.MustExec("create table tt(a int, b int, c int, primary key(b,a) nonclustered)")
tk.MustExec("drop table if exists ti")
tk.MustExec("create table ti(a int, b int, c int unique)")
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)")
tk.MustExec("insert into tt select *,55 from t")
tk.MustExec("insert into ti select *,a from t")
tk.MustExec("create table customer(primary_key VARBINARY(1024), secondary_key VARBINARY(1024), c_timestamp BIGINT, value MEDIUMBLOB, PRIMARY KEY (primary_key, secondary_key, c_timestamp) clustered);")
var input Input
var output []struct {
SQL string
Expand All @@ -82,3 +138,28 @@ func TestPushDerivedTopnPositive(t *testing.T) {
res.Check(testkit.Rows(output[i].Res...))
}
}

// Negative test when tidb_opt_derive_topn is off
func TestPushDerivedTopnFlagOff(t *testing.T) {
store := testkit.CreateMockStore(t)
tk := testkit.NewTestKit(t, store)
tk.MustExec("set tidb_opt_derive_topn=0")
tk.MustExec("use test")
tk.MustExec("drop table if exists t")
tk.MustExec("create table t(a int, b int, primary key(b,a))")
var input Input
var output []struct {
SQL string
Plan []string
}
suiteData := 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...))
}
}
45 changes: 35 additions & 10 deletions planner/core/casetest/testdata/derive_topn_from_window_in.json
Original file line number Diff line number Diff line change
Expand Up @@ -2,28 +2,53 @@
{
"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"
"select * from (select row_number() over () as rownumber from t) DT where rownumber <= 3 and rownumber >= 2 -- pattern is not applicable with complex filter",
"select row_number() over (partition by a) from t -- pattern missing filter on row number",
"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 row_number() over (partition by a) as rownumber from t) DT where rownumber <= 3 -- pattern is not applicable with partition by not prefix of PK",
"select * from (select row_number() over (partition by a) as rownumber from tt) DT where rownumber <= 3 -- pattern is not applicable with no clustered index on table",
"select * from (select row_number() over (partition by a) as rownumber from ti) DT where rownumber <= 3 -- pattern is not applicable. ti clustered on 'c' and partition by 'a'",
"select * from (select a, row_number() over (partition by c) as rownumber from ti) DT where rownumber <= 1 -- pattern is applicable. ti clustered on 'c' and partition by 'c' - not supported yet",
"with agg_t as (select count(*) cnt, a from t group by a) select * from (select row_number() over () as rownumber from agg_t) DT where rownumber <= 3 -- pattern on non-data source"
]
},
{
"name": "TestPushDerivedTopnFlash",
"cases":[
"select * from (select row_number() over (order by b) as rownumber from t) DT where rownumber <= 1 -- applicable with no partition by",
"select * from (select row_number() over (partition by b) as rownumber from t) DT where rownumber <= 1 -- applicable with partition by but no push down to tiflash",
"select * from (select row_number() over (partition by b order by a) as rownumber from t) DT where rownumber <= 1 -- applicable with partition by and order by but no push down to tiflash",
"select * from (select row_number() over (partition by a) as rownumber from t) DT where rownumber <= 3 -- pattern is not applicable with partition by not prefix of PK"
]
},
{
"name": "TestPushDerivedTopnFlagOff",
"cases":[
"select * from (select a,b, row_number() over (order by a) as rownumber from t) DT where rownumber <= 3.5 -- pattern is applicable but flag is turned off and rule not applied.",
"select * from (select row_number() over (order by a) as rownumber from t) DT where rownumber <= 3 -- pattern is applicable but flag is turned off and rule not applied."
]
},
{
"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"
"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. No partition by",
"select * from (select row_number() over (order by a) as rownumber from t) DT where rownumber <= 3 -- pattern is applicable. No partition by",
"select * from (select row_number() over (order by a) as rownumber from t) DT where rownumber < 3 -- pattern is applicable. No partition by",
"select * from (select row_number() over(rows between 1 preceding and 1 following) as rownumber from t) DT where rownumber <= 3 -- pattern is applicable. No partition by",
"select * from (select a,row_number() over (order by a desc) as rownumber,b from t) DT where rownumber <= 3 -- pattern is applicable. Order by desc. No partition by",
"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. Window has column a + row_number",
"select * from (select row_number() over (partition by b) as rownumber from t) DT where rownumber <= 1 -- pattern is applicable with partition by prefix of primary key",
"select * from (select row_number() over (partition by b order by a) as rownumber from t) DT where rownumber <= 1 -- pattern is applicable with order by, partition by prefix",
"select * from (select row_number() over (partition by b) as rownumber from t where a >= 2) DT where rownumber <= 3 order by rownumber -- pattern is applicable with partition by prefix and filter on data source",
"select * from (select row_number() over () as rownumber from tt) DT where rownumber <= 3 -- pattern is applicable with no partition on a non-clustered table",
"select * from (select *, row_number() over (partition by primary_key, secondary_key order by c_timestamp) as rownum from customer where primary_key = 0x002 and secondary_key >= 0x001 and c_timestamp >= 1661883508511000000) as nested where rownum <= 10 order by secondary_key desc;"
]
}
]
Loading

0 comments on commit 0e10f17

Please sign in to comment.