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/core: always add projection to agg that is pushed to tiflash #26263

Merged
merged 4 commits into from
Jul 15, 2021
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
4 changes: 2 additions & 2 deletions planner/core/rule_eliminate_projection.go
Original file line number Diff line number Diff line change
Expand Up @@ -43,14 +43,14 @@ func canProjectionBeEliminatedStrict(p *PhysicalProjection) bool {
// the align the output schema. In the future, we can solve this in-compatibility by
// passing down the aggregation mode to TiFlash.
if physicalAgg, ok := p.Children()[0].(*PhysicalHashAgg); ok {
if physicalAgg.MppRunMode == Mpp1Phase || physicalAgg.MppRunMode == Mpp2Phase {
if physicalAgg.MppRunMode == Mpp1Phase || physicalAgg.MppRunMode == Mpp2Phase || physicalAgg.MppRunMode == MppScalar {
if physicalAgg.isFinalAgg() {
return false
}
}
}
if physicalAgg, ok := p.Children()[0].(*PhysicalStreamAgg); ok {
if physicalAgg.MppRunMode == Mpp1Phase || physicalAgg.MppRunMode == Mpp2Phase {
if physicalAgg.MppRunMode == Mpp1Phase || physicalAgg.MppRunMode == Mpp2Phase || physicalAgg.MppRunMode == MppScalar {
if physicalAgg.isFinalAgg() {
return false
}
Expand Down
15 changes: 10 additions & 5 deletions planner/core/task.go
Original file line number Diff line number Diff line change
Expand Up @@ -1969,14 +1969,19 @@ func (p *PhysicalHashAgg) attach2TaskForMpp(tasks ...task) task {
prop := &property.PhysicalProperty{TaskTp: property.MppTaskType, ExpectedCnt: math.MaxFloat64, MPPPartitionTp: property.AnyType}
newMpp := mpp.enforceExchangerImpl(prop)
attachPlan2Task(finalAgg, newMpp)
if proj != nil {
attachPlan2Task(proj, newMpp)
if proj == nil {
proj = PhysicalProjection{
Exprs: make([]expression.Expression, 0, len(p.Schema().Columns)),
}.Init(p.ctx, p.statsInfo(), p.SelectBlockOffset())
for _, col := range p.Schema().Columns {
proj.Exprs = append(proj.Exprs, col)
}
proj.SetSchema(p.schema)
}
attachPlan2Task(proj, newMpp)
newMpp.addCost(p.GetCost(inputRows, false, true))
finalAgg.SetCost(newMpp.cost())
if proj != nil {
proj.SetCost(newMpp.cost())
}
proj.SetCost(newMpp.cost())
return newMpp
default:
return invalidTask
Expand Down
1 change: 1 addition & 0 deletions planner/core/testdata/integration_serial_suite_in.json
Original file line number Diff line number Diff line change
Expand Up @@ -279,6 +279,7 @@
"desc format = 'brief' select /*+hash_agg()*/ sum(id) from (select value, id from t where id > value group by id, value)A group by value /*the exchange should have only one partition column: test.t.value*/",
"desc format = 'brief' select /*+hash_agg()*/ sum(B.value) from t as B where B.id+1 > (select count(*) from t where t.id= B.id and t.value=B.value) group by B.id /*the exchange should have only one partition column: test.t.id*/",
"desc format = 'brief' select count(distinct value) from t",
"desc format = 'brief' select count(distinct x ) from (select count(distinct value) x from t) t",
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

would we add a test case : select count(distinct x), avg(y) ? becase avg would add a proj.

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

the test case below this one is exactly what you mentioned

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

ok

"desc format = 'brief' select count(distinct value), count(value), avg(value) from t"
]
},
Expand Down
29 changes: 24 additions & 5 deletions planner/core/testdata/integration_serial_suite_out.json
Original file line number Diff line number Diff line change
Expand Up @@ -2365,11 +2365,30 @@
"Plan": [
"TableReader 1.00 root data:ExchangeSender",
"└─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough",
" └─HashAgg 1.00 batchCop[tiflash] funcs:count(distinct test.t.value)->Column#4",
" └─ExchangeReceiver 1.00 batchCop[tiflash] ",
" └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough",
" └─HashAgg 1.00 batchCop[tiflash] group by:test.t.value, ",
" └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo"
" └─Projection 1.00 batchCop[tiflash] Column#4",
" └─HashAgg 1.00 batchCop[tiflash] funcs:count(distinct test.t.value)->Column#4",
" └─ExchangeReceiver 1.00 batchCop[tiflash] ",
" └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough",
" └─HashAgg 1.00 batchCop[tiflash] group by:test.t.value, ",
" └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo"
]
},
{
"SQL": "desc format = 'brief' select count(distinct x ) from (select count(distinct value) x from t) t",
"Plan": [
"TableReader 1.00 root data:ExchangeSender",
"└─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough",
" └─Projection 1.00 batchCop[tiflash] Column#5",
" └─HashAgg 1.00 batchCop[tiflash] funcs:count(distinct Column#4)->Column#5",
" └─ExchangeReceiver 1.00 batchCop[tiflash] ",
" └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough",
" └─HashAgg 1.00 batchCop[tiflash] group by:Column#4, ",
" └─Projection 1.00 batchCop[tiflash] Column#4",
" └─HashAgg 1.00 batchCop[tiflash] funcs:count(distinct test.t.value)->Column#4",
" └─ExchangeReceiver 1.00 batchCop[tiflash] ",
" └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough",
" └─HashAgg 1.00 batchCop[tiflash] group by:test.t.value, ",
" └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo"
]
},
{
Expand Down