Skip to content
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

planner: add trace for partition pruning #30546

Merged
merged 13 commits into from
Dec 16, 2021
3 changes: 2 additions & 1 deletion planner/core/logical_plan_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -59,7 +59,8 @@ type testPlanSuite struct {
}

func (s *testPlanSuite) SetUpSuite(c *C) {
s.is = infoschema.MockInfoSchema([]*model.TableInfo{MockSignedTable(), MockUnsignedTable(), MockView(), MockNoPKTable()})
s.is = infoschema.MockInfoSchema([]*model.TableInfo{MockSignedTable(), MockUnsignedTable(), MockView(), MockNoPKTable(),
MockRangePartitionTable(), MockHashPartitionTable(), MockListPartitionTable()})
s.ctx = MockContext()
domain.GetDomain(s.ctx).MockInfoCacheAndLoadInfoSchema(s.is)
s.ctx.GetSessionVars().EnableWindowFunction = true
Expand Down
77 changes: 77 additions & 0 deletions planner/core/logical_plan_trace_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -86,6 +86,83 @@ func (s *testPlanSuite) TestSingleRuleTraceStep(c *C) {
assertRuleName string
assertRuleSteps []assertTraceStep
}{
{
sql: "select * from pt3 where ptn > 3;",
flags: []uint64{flagPartitionProcessor, flagPredicatePushDown, flagBuildKeyInfo, flagPrunColumns},
assertRuleName: "partition_processor",
assertRuleSteps: []assertTraceStep{
{
assertReason: "Datasource[1] has multiple needed partitions[p1,p2] after pruning",
assertAction: "Datasource[1] becomes PartitionUnion[6] with children[TableScan[1],TableScan[1]]",
},
},
},
{
sql: "select * from pt3 where ptn = 1;",
flags: []uint64{flagPartitionProcessor, flagPredicatePushDown, flagBuildKeyInfo, flagPrunColumns},
assertRuleName: "partition_processor",
assertRuleSteps: []assertTraceStep{
{
assertReason: "Datasource[1] has one needed partition[p1] after pruning",
assertAction: "Datasource[1] becomes TableScan[1]",
},
},
},
{
sql: "select * from pt2 where ptn in (1,2,3);",
flags: []uint64{flagPartitionProcessor, flagPredicatePushDown, flagBuildKeyInfo, flagPrunColumns},
assertRuleName: "partition_processor",
assertRuleSteps: []assertTraceStep{
{
assertReason: "Datasource[1] has multiple needed partitions[p1,p2] after pruning",
assertAction: "Datasource[1] becomes PartitionUnion[7] with children[TableScan[1],TableScan[1]]",
},
},
},
{
sql: "select * from pt2 where ptn = 1;",
flags: []uint64{flagPartitionProcessor, flagPredicatePushDown, flagBuildKeyInfo, flagPrunColumns},
assertRuleName: "partition_processor",
assertRuleSteps: []assertTraceStep{
{
assertReason: "Datasource[1] has one needed partition[p2] after pruning",
assertAction: "Datasource[1] becomes TableScan[1]",
},
},
},
{
sql: "select * from pt1 where ptn > 100;",
flags: []uint64{flagPartitionProcessor, flagPredicatePushDown, flagBuildKeyInfo, flagPrunColumns},
assertRuleName: "partition_processor",
assertRuleSteps: []assertTraceStep{
{
assertReason: "Datasource[1] doesn't have needed partition table after pruning",
assertAction: "Datasource[1] becomes TableDual[5]",
},
},
},
{
sql: "select * from pt1 where ptn in (10,20);",
flags: []uint64{flagPartitionProcessor, flagPredicatePushDown, flagBuildKeyInfo, flagPrunColumns},
assertRuleName: "partition_processor",
assertRuleSteps: []assertTraceStep{
{
assertReason: "Datasource[1] has multiple needed partitions[p1,p2] after pruning",
assertAction: "Datasource[1] becomes PartitionUnion[7] with children[TableScan[1],TableScan[1]]",
},
},
},
{
sql: "select * from pt1 where ptn < 4;",
flags: []uint64{flagPartitionProcessor, flagPredicatePushDown, flagBuildKeyInfo, flagPrunColumns},
assertRuleName: "partition_processor",
assertRuleSteps: []assertTraceStep{
{
assertReason: "Datasource[1] has one needed partition[p1] after pruning",
assertAction: "Datasource[1] becomes TableScan[1]",
},
},
},
{
sql: "select * from (t t1, t t2, t t3,t t4) union all select * from (t t5, t t6, t t7,t t8)",
flags: []uint64{flagBuildKeyInfo, flagPrunColumns, flagDecorrelate, flagPredicatePushDown, flagEliminateOuterJoin, flagJoinReOrder},
Expand Down
119 changes: 119 additions & 0 deletions planner/core/mock.go
Original file line number Diff line number Diff line change
Expand Up @@ -432,3 +432,122 @@ func MockPartitionInfoSchema(definitions []model.PartitionDefinition) infoschema
is := infoschema.MockInfoSchema([]*model.TableInfo{tableInfo})
return is
}

// MockRangePartitionTable mocks a range partition table for test
func MockRangePartitionTable() *model.TableInfo {
definitions := []model.PartitionDefinition{
{
ID: 41,
Name: model.NewCIStr("p1"),
LessThan: []string{"16"},
},
{
ID: 42,
Name: model.NewCIStr("p2"),
LessThan: []string{"32"},
},
}
tableInfo := MockSignedTable()
tableInfo.Name = model.NewCIStr("pt1")
cols := make([]*model.ColumnInfo, 0, len(tableInfo.Columns))
cols = append(cols, tableInfo.Columns...)
last := tableInfo.Columns[len(tableInfo.Columns)-1]
cols = append(cols, &model.ColumnInfo{
State: model.StatePublic,
Offset: last.Offset + 1,
Name: model.NewCIStr("ptn"),
FieldType: newLongType(),
ID: last.ID + 1,
})
partition := &model.PartitionInfo{
Type: model.PartitionTypeRange,
Expr: "ptn",
Enable: true,
Definitions: definitions,
}
tableInfo.Columns = cols
tableInfo.Partition = partition
return tableInfo
}

// MockHashPartitionTable mocks a hash partition table for test
func MockHashPartitionTable() *model.TableInfo {
definitions := []model.PartitionDefinition{
{
ID: 51,
Name: model.NewCIStr("p1"),
},
{
ID: 52,
Name: model.NewCIStr("p2"),
},
}
tableInfo := MockSignedTable()
tableInfo.Name = model.NewCIStr("pt2")
cols := make([]*model.ColumnInfo, 0, len(tableInfo.Columns))
cols = append(cols, tableInfo.Columns...)
last := tableInfo.Columns[len(tableInfo.Columns)-1]
cols = append(cols, &model.ColumnInfo{
State: model.StatePublic,
Offset: last.Offset + 1,
Name: model.NewCIStr("ptn"),
FieldType: newLongType(),
ID: last.ID + 1,
})
partition := &model.PartitionInfo{
Type: model.PartitionTypeHash,
Expr: "ptn",
Enable: true,
Definitions: definitions,
Num: 2,
}
tableInfo.Columns = cols
tableInfo.Partition = partition
return tableInfo
}

// MockListPartitionTable mocks a list partition table for test
func MockListPartitionTable() *model.TableInfo {
definitions := []model.PartitionDefinition{
{
ID: 61,
Name: model.NewCIStr("p1"),
InValues: [][]string{
{
"1",
},
},
},
{
ID: 62,
Name: model.NewCIStr("p2"),
InValues: [][]string{
{
"2",
},
},
},
}
tableInfo := MockSignedTable()
tableInfo.Name = model.NewCIStr("pt3")
cols := make([]*model.ColumnInfo, 0, len(tableInfo.Columns))
cols = append(cols, tableInfo.Columns...)
last := tableInfo.Columns[len(tableInfo.Columns)-1]
cols = append(cols, &model.ColumnInfo{
State: model.StatePublic,
Offset: last.Offset + 1,
Name: model.NewCIStr("ptn"),
FieldType: newLongType(),
ID: last.ID + 1,
})
partition := &model.PartitionInfo{
Type: model.PartitionTypeList,
Expr: "ptn",
Enable: true,
Definitions: definitions,
Num: 2,
}
tableInfo.Columns = cols
tableInfo.Partition = partition
return tableInfo
}
Loading