Skip to content

Commit

Permalink
Add projection eliminator instead
Browse files Browse the repository at this point in the history
Signed-off-by: yibin <huyibin@pingcap.com>
  • Loading branch information
yibin87 committed Jan 15, 2024
1 parent 9db7113 commit c5685c1
Show file tree
Hide file tree
Showing 3 changed files with 6 additions and 55 deletions.
7 changes: 6 additions & 1 deletion pkg/planner/core/optimizer.go
Original file line number Diff line number Diff line change
Expand Up @@ -84,6 +84,7 @@ const (
flagPushDownTopN
flagSyncWaitStatsLoadPoint
flagJoinReOrder
flagEliminateProjectionAgain
flagPrunColumnsAgain
flagPushDownSequence
flagResolveExpand
Expand Down Expand Up @@ -111,7 +112,8 @@ var optRuleList = []logicalOptRule{
&pushDownTopNOptimizer{},
&syncWaitStatsLoadPoint{},
&joinReOrderSolver{},
&columnPruner{}, // column pruning again at last, note it will mess up the results of buildKeySolver
&projectionEliminator{}, // joinReOrderSolver may add new projection, thus add projectionEliminator here
&columnPruner{}, // column pruning again at last, note it will mess up the results of buildKeySolver
&pushDownSequenceSolver{},
&resolveExpand{},
}
Expand Down Expand Up @@ -323,6 +325,9 @@ func doOptimize(
// When we use the straight Join Order hint, we should disable the join reorder optimization.
flag &= ^flagJoinReOrder
}
if flag&flagJoinReOrder > 0 {
flag |= flagEliminateProjectionAgain
}
flag |= flagCollectPredicateColumnsPoint
flag |= flagSyncWaitStatsLoadPoint
logic, err := logicalOptimize(ctx, flag, logic)
Expand Down
42 changes: 0 additions & 42 deletions pkg/planner/core/optimizer_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -23,7 +23,6 @@ import (
"github.com/pingcap/tidb/pkg/domain"
"github.com/pingcap/tidb/pkg/expression"
"github.com/pingcap/tidb/pkg/parser/ast"
"github.com/pingcap/tidb/pkg/parser/model"
"github.com/pingcap/tidb/pkg/parser/mysql"
"github.com/pingcap/tidb/pkg/planner/property"
"github.com/pingcap/tidb/pkg/types"
Expand Down Expand Up @@ -506,44 +505,3 @@ func TestPrunePhysicalColumns(t *testing.T) {
require.True(t, len(recv1.Schema().Columns) == 1)
require.True(t, recv1.Schema().Contains(col3))
}

// Projection0(1) <= Projection1(all) <= DataSource(t0)
func TestIssue50358(t *testing.T) {
ctx := MockContext()
defer func() {
domain.GetDomain(ctx).StatsHandle().Close()
}()
ctx.GetSessionVars().PlanID.Store(-1)
ds := DataSource{}.Init(ctx, 0)
tan := model.NewCIStr("t0")
ds.TableAsName = &tan
ds.schema = expression.NewSchema()
col0 := &expression.Column{
UniqueID: ctx.GetSessionVars().PlanColumnID.Add(1),
RetType: types.NewFieldType(mysql.TypeLonglong),
}
ds.schema.Append(col0)

// Projection1
exprs := make([]expression.Expression, 1)
exprs[0] = col0
proj1 := LogicalProjection{Exprs: exprs, AvoidColumnEvaluator: true}.Init(ctx, 0)
proj1.SetSchema(ds.schema.Clone())
proj1.SetChildren(ds)

// Projection0
constOne := expression.NewOne()
exprs0 := make([]expression.Expression, 1)
exprs0[0] = constOne
proj0 := LogicalProjection{Exprs: exprs0, AvoidColumnEvaluator: true}.Init(ctx, 0)
proj0.SetSchema(expression.NewSchema(&expression.Column{
UniqueID: ctx.GetSessionVars().AllocPlanColumnID(),
RetType: constOne.GetType(),
}))
proj0.SetChildren(proj1)

err := proj0.PruneColumns(proj0.schema.Columns, nil, nil)
require.NoError(t, err)
require.Equal(t, len(proj1.schema.Columns), 1)
require.Equal(t, len(proj1.Exprs), 1)
}
12 changes: 0 additions & 12 deletions pkg/planner/core/rule_column_pruning.go
Original file line number Diff line number Diff line change
Expand Up @@ -97,26 +97,14 @@ func (p *LogicalProjection) PruneColumns(parentUsedCols []*expression.Column, op
used := expression.GetUsedList(p.SCtx(), parentUsedCols, p.schema)
prunedColumns := make([]*expression.Column, 0)

leftColumnCount := 0
// for implicit projected cols, once the ancestor doesn't use it, the implicit expr will be automatically pruned here.
for i := len(used) - 1; i >= 0; i-- {
// Keep at least one column here to avoid empty result sets
// For example: select 3 from (select * from t) t1, logical plan might be like this:
// Projection0(3) <= Projection1(all) <= DataSource(t)
// To ensure correctness, Projection1's schema can't be empty here although none of its output columns are actually used
// TODO: The way that picks the "left-one-column" might be optimized further
if i == 0 && leftColumnCount == 0 {
break
}
if !used[i] && !exprHasSetVarOrSleep(p.Exprs[i]) {
prunedColumns = append(prunedColumns, p.schema.Columns[i])
p.schema.Columns = append(p.schema.Columns[:i], p.schema.Columns[i+1:]...)
p.Exprs = append(p.Exprs[:i], p.Exprs[i+1:]...)
} else {
leftColumnCount++
}
}

appendColumnPruneTraceStep(p, prunedColumns, opt)
selfUsedCols := make([]*expression.Column, 0, len(p.Exprs))
selfUsedCols = expression.ExtractColumnsFromExpressions(selfUsedCols, p.Exprs, nil)
Expand Down

0 comments on commit c5685c1

Please sign in to comment.