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 HashJoin<-Receiver specific physicalPlan column pruner #38536

Merged
merged 23 commits into from
Nov 2, 2022
Merged
Show file tree
Hide file tree
Changes from 3 commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Jump to
Jump to file
Failed to load files.
Diff view
Diff view
93 changes: 93 additions & 0 deletions planner/core/optimizer.go
Original file line number Diff line number Diff line change
Expand Up @@ -375,6 +375,7 @@ func mergeContinuousSelections(p PhysicalPlan) {
func postOptimize(sctx sessionctx.Context, plan PhysicalPlan) PhysicalPlan {
// some cases from update optimize will require avoiding projection elimination.
// see comments ahead of call of DoOptimize in function of buildUpdate().
prunePhysicalColumns(sctx, plan)
plan = eliminatePhysicalProjection(plan)
plan = InjectExtraProjection(plan)
mergeContinuousSelections(plan)
Expand All @@ -386,6 +387,98 @@ func postOptimize(sctx sessionctx.Context, plan PhysicalPlan) PhysicalPlan {
return plan
}

// prunePhysicalColumns currently only work for MPP(HashJoin<-Exchange).
winoros marked this conversation as resolved.
Show resolved Hide resolved
func prunePhysicalColumns(sctx sessionctx.Context, plan PhysicalPlan) {
emptyColumns := make([]*expression.Column, 0)
if tableReader, ok := plan.(*PhysicalTableReader); ok {
if _, isExchangeSender := tableReader.tablePlan.(*PhysicalExchangeSender); isExchangeSender {
prunePhysicalColumnsInternal(sctx, tableReader.tablePlan, emptyColumns, false)
}
} else {
for _, child := range plan.Children() {
prunePhysicalColumns(sctx, child)
}
}
}

func (p *PhysicalHashJoin) extractUsedCols(parentUsedCols []*expression.Column) (leftCols []*expression.Column, rightCols []*expression.Column) {
for _, eqCond := range p.EqualConditions {
parentUsedCols = append(parentUsedCols, expression.ExtractColumns(eqCond)...)
}
for _, neCond := range p.NAEqualConditions {
parentUsedCols = append(parentUsedCols, expression.ExtractColumns(neCond)...)
}
for _, leftCond := range p.LeftConditions {
parentUsedCols = append(parentUsedCols, expression.ExtractColumns(leftCond)...)
}
for _, rightCond := range p.RightConditions {
parentUsedCols = append(parentUsedCols, expression.ExtractColumns(rightCond)...)
}
for _, otherCond := range p.OtherConditions {
parentUsedCols = append(parentUsedCols, expression.ExtractColumns(otherCond)...)
}
lChild := p.children[0]
rChild := p.children[1]
for _, col := range parentUsedCols {
if lChild.Schema().Contains(col) {
leftCols = append(leftCols, col)
} else if rChild.Schema().Contains(col) {
rightCols = append(rightCols, col)
}
}
return leftCols, rightCols
}

func prunePhysicalColumnsInternal(sctx sessionctx.Context, plan PhysicalPlan, parentUsedCols []*expression.Column, meaningFulParentUsedCols bool) {
switch x := plan.(type) {
case *PhysicalHashJoin:
child0 := x.children[0]
child1 := x.children[1]
schemaColumns := make([]*expression.Column, 0, len(x.Schema().Columns))
schemaColumns = x.Schema().Columns
leftCols, rightCols := x.extractUsedCols(schemaColumns)
prunePhysicalColumnsInternal(sctx, child0, leftCols, true)
prunePhysicalColumnsInternal(sctx, child1, rightCols, true)
case *PhysicalExchangeReceiver:
// Currently, prune columns only when parent node is HashJoin
if meaningFulParentUsedCols {
Copy link
Contributor

Choose a reason for hiding this comment

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

meaningFulParentUsedCols sounds kinds of confusing to me. Is it possible to rewrite the logic as follows:

if x is PhysicalHashJoin {
    if left child is PhysicalExchangeReceiver and its child is PhysicalExchangeSender {
        // put a projection to prune columns if necessary
    }
    if right child is PhysicalExchangeReceiver and its child is PhysicalExchangeSender {
        // put a projection to prune columns if necessary
    }
}

Copy link
Contributor

Choose a reason for hiding this comment

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

In this way, maybe we don't need parentUsedCols.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Nice suggestion, Done.

used := expression.GetUsedList(parentUsedCols, x.Schema())
needPrune := false
usedExprs := make([]expression.Expression, len(x.Schema().Columns))
prunedSchema := x.Schema().Clone()
for i := len(used) - 1; i >= 0; i-- {
usedExprs[i] = x.Schema().Columns[i]
if !used[i] {
needPrune = true
usedExprs = append(usedExprs[:i], usedExprs[i+1:]...)
prunedSchema.Columns = append(prunedSchema.Columns[:i], prunedSchema.Columns[i+1:]...)
}
}

if needPrune {
switch y := x.children[0].(type) {
case *PhysicalExchangeSender:
ch := y.children[0]
Copy link
Contributor

Choose a reason for hiding this comment

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

Do we need to recursively prune columns for ch?

Copy link
Contributor Author

@yibin87 yibin87 Nov 1, 2022

Choose a reason for hiding this comment

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

Good catch, fixed it.

proj := PhysicalProjection{
Exprs: usedExprs,
}.Init(sctx, ch.statsInfo(), 0)

proj.SetSchema(prunedSchema)
proj.SetChildren(ch)
y.children[0] = proj
}
}
}
emptyColumns := make([]*expression.Column, 0)
prunePhysicalColumnsInternal(sctx, x.children[0], emptyColumns, false)
default:
emptyColumns := make([]*expression.Column, 0)
for _, child := range x.Children() {
prunePhysicalColumnsInternal(sctx, child, emptyColumns, false)
}
}
}

// Only for MPP(Window<-[Sort]<-ExchangeReceiver<-ExchangeSender).
// TiFlashFineGrainedShuffleStreamCount:
// < 0: fine grained shuffle is disabled.
Expand Down
85 changes: 85 additions & 0 deletions planner/core/optimizer_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -15,6 +15,8 @@
package core

import (
"github.com/pingcap/tidb/expression"
"github.com/pingcap/tidb/parser/ast"
Copy link
Member

Choose a reason for hiding this comment

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

Will fail the static check

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Done.

"reflect"
"testing"

Expand Down Expand Up @@ -288,3 +290,86 @@ func TestHandleFineGrainedShuffle(t *testing.T) {
hashSender1.children = []PhysicalPlan{tableScan1}
start(partWindow, expStreamCount, 3, 0)
}

// Test for core.prunePhysicalColumns()
func TestPrunePhysicalColumns(t *testing.T) {
sctx := MockContext()
col0 := &expression.Column{
UniqueID: sctx.GetSessionVars().AllocPlanColumnID(),
RetType: types.NewFieldType(mysql.TypeLonglong),
}
col1 := &expression.Column{
UniqueID: sctx.GetSessionVars().AllocPlanColumnID(),
RetType: types.NewFieldType(mysql.TypeLonglong),
}
col2 := &expression.Column{
UniqueID: sctx.GetSessionVars().AllocPlanColumnID(),
RetType: types.NewFieldType(mysql.TypeLonglong),
}
col3 := &expression.Column{
UniqueID: sctx.GetSessionVars().AllocPlanColumnID(),
RetType: types.NewFieldType(mysql.TypeLonglong),
}

// Join[col0, col3; col0==col3] <- ExchangeReceiver[col0, col1, col2] <- ExchangeSender[col0, col1, col2] <- Selection[col0, col1, col2; col1 < col2] <- TableScan[col0, col1, col2]
// <- ExchangeReceiver1[col3] <- ExchangeSender1[col3] <- TableScan1[col3]
tableReader := &PhysicalTableReader{}
passSender := &PhysicalExchangeSender{
ExchangeType: tipb.ExchangeType_PassThrough,
}
hashJoin := &PhysicalHashJoin{}
recv := &PhysicalExchangeReceiver{}
recv1 := &PhysicalExchangeReceiver{}
hashSender := &PhysicalExchangeSender{
ExchangeType: tipb.ExchangeType_Hash,
}
hashSender1 := &PhysicalExchangeSender{
ExchangeType: tipb.ExchangeType_Hash,
}
tableScan := &PhysicalTableScan{}
tableScan1 := &PhysicalTableScan{}

tableReader.tablePlan = passSender
passSender.children = []PhysicalPlan{hashJoin}
hashJoin.children = []PhysicalPlan{recv, recv1}
selection := &PhysicalSelection{}

cond := expression.NewFunctionInternal(sctx, ast.EQ, types.NewFieldType(mysql.TypeTiny), col0, col3)
sf, isSF := cond.(*expression.ScalarFunction)
require.True(t, isSF)
hashJoin.EqualConditions = append(hashJoin.EqualConditions, sf)
hashJoinSchema := make([]*expression.Column, 0)
hashJoinSchema = append(hashJoinSchema, col3)
hashJoin.SetSchema(expression.NewSchema(hashJoinSchema...))

selection.SetChildren(tableScan)
hashSender.SetChildren(selection)
recv.SetChildren(hashSender)
tableScan.Schema().Columns = append(tableScan.Schema().Columns, col0, col1, col2)

hashSender1.SetChildren(tableScan1)
recv1.SetChildren(hashSender1)
tableScan1.Schema().Columns = append(tableScan1.Schema().Columns, col3)

prunePhysicalColumns(sctx, tableReader)

// Optimized Plan:
// Join[col0, col3; col0==col3] <- ExchangeReceiver[col0] <- ExchangeSender[col0] <- Projection[col0] <- Selection[col0, col1, col2; col1 < col2] <- TableScan[col0, col1, col2]
// <- ExchangeReceiver1[col3] <- ExchangeSender1[col3] <- TableScan1[col3]
require.True(t, len(recv.Schema().Columns) == 1)
require.True(t, recv.Schema().Contains(col0))
require.False(t, recv.Schema().Contains(col1))
require.False(t, recv.Schema().Contains(col2))
require.True(t, len(recv.children[0].Children()) == 1)
physicalProj := recv.children[0].Children()[0]
switch x := physicalProj.(type) {
case *PhysicalProjection:
require.True(t, x.Schema().Contains(col0))
require.False(t, recv.Schema().Contains(col1))
require.False(t, recv.Schema().Contains(col2))
default:
require.True(t, false)
}
require.True(t, len(recv1.Schema().Columns) == 1)
require.True(t, recv1.Schema().Contains(col3))
}