expression: only do a shallow copy when evaluating a "Column" expression (#5542)

This commit is contained in:
Jian Zhang
2018-01-06 12:24:30 +08:00
committed by HuaiyuXu
parent 07dc46d3cb
commit de992eb570
4 changed files with 133 additions and 7 deletions

View File

@ -707,6 +707,7 @@ func (b *executorBuilder) buildProjection(v *plan.PhysicalProjection) Executor {
e := &ProjectionExec{
baseExecutor: newBaseExecutor(v.Schema(), b.ctx, childExec),
exprs: v.Exprs,
evaluatorSuit: expression.NewEvaluatorSuit(v.Exprs),
calculateNoDelay: v.CalculateNoDelay,
}
e.baseExecutor.supportChk = true

View File

@ -598,8 +598,8 @@ func init() {
type ProjectionExec struct {
baseExecutor
exprs []expression.Expression
vectorizable bool
exprs []expression.Expression // Only used in Next().
evaluatorSuit *expression.EvaluatorSuit
calculateNoDelay bool
}
@ -608,7 +608,6 @@ func (e *ProjectionExec) Open(goCtx goctx.Context) error {
if err := e.baseExecutor.Open(goCtx); err != nil {
return errors.Trace(err)
}
e.vectorizable = expression.Vectorizable(e.exprs)
return nil
}
@ -638,10 +637,7 @@ func (e *ProjectionExec) NextChunk(goCtx goctx.Context, chk *chunk.Chunk) error
if err := e.children[0].NextChunk(goCtx, e.childrenResults[0]); err != nil {
return errors.Trace(err)
}
if e.vectorizable {
return errors.Trace(expression.VectorizedExecute(e.ctx, e.exprs, e.childrenResults[0], chk))
}
return errors.Trace(expression.UnVectorizedExecute(e.ctx, e.exprs, e.childrenResults[0], chk))
return errors.Trace(e.evaluatorSuit.Run(e.ctx, e.childrenResults[0], chk))
}
// TableDualExec represents a dual table executor.

119
expression/evaluator.go Normal file
View File

@ -0,0 +1,119 @@
// Copyright 2018 PingCAP, Inc.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// See the License for the specific language governing permissions and
// limitations under the License.
package expression
import (
"github.com/juju/errors"
"github.com/pingcap/tidb/context"
"github.com/pingcap/tidb/util/chunk"
)
type columnEvaluator struct {
inputIdxToOutputIdxes map[int][]int
}
// run evaluates "Column" expressions.
// NOTE: It should be called after all the other expressions are evaluated
// since it will change the content of the input Chunk.
func (e *columnEvaluator) run(ctx context.Context, input, output *chunk.Chunk) {
for inputIdx, outputIdxes := range e.inputIdxToOutputIdxes {
output.SwapColumn(outputIdxes[0], input, inputIdx)
for i, length := 1, len(outputIdxes); i < length; i++ {
output.MakeRef(outputIdxes[0], outputIdxes[i])
}
}
}
type defaultEvaluator struct {
outputIdxes []int
exprs []Expression
vectorizable bool
}
func (e *defaultEvaluator) run(ctx context.Context, input, output *chunk.Chunk) error {
sc := ctx.GetSessionVars().StmtCtx
if e.vectorizable {
for i := range e.outputIdxes {
err := evalOneColumn(sc, e.exprs[i], input, output, e.outputIdxes[i])
if err != nil {
return errors.Trace(err)
}
}
return nil
}
for row := input.Begin(); row != input.End(); row = row.Next() {
for i := range e.outputIdxes {
err := evalOneCell(sc, e.exprs[i], row, output, e.outputIdxes[i])
if err != nil {
return errors.Trace(err)
}
}
}
return nil
}
// EvaluatorSuit is responsible for the evaluation of a list of expressions.
// It seperates them to "column" and "other" expressions and evaluates "other"
// expressions before "column" expressions.
type EvaluatorSuit struct {
*columnEvaluator // Evaluator for column expressions.
*defaultEvaluator // Evaluator for other expressions.
}
// NewEvaluatorSuit creates an EvaluatorSuit to evaluate all the exprs.
func NewEvaluatorSuit(exprs []Expression) *EvaluatorSuit {
e := &EvaluatorSuit{}
for i, expr := range exprs {
switch x := expr.(type) {
case *Column:
if e.columnEvaluator == nil {
e.columnEvaluator = &columnEvaluator{inputIdxToOutputIdxes: make(map[int][]int)}
}
inputIdx, outputIdx := x.Index, i
e.columnEvaluator.inputIdxToOutputIdxes[inputIdx] = append(e.columnEvaluator.inputIdxToOutputIdxes[inputIdx], outputIdx)
default:
if e.defaultEvaluator == nil {
e.defaultEvaluator = &defaultEvaluator{
outputIdxes: make([]int, 0, len(exprs)),
exprs: make([]Expression, 0, len(exprs)),
}
}
e.defaultEvaluator.exprs = append(e.defaultEvaluator.exprs, x)
e.defaultEvaluator.outputIdxes = append(e.defaultEvaluator.outputIdxes, i)
}
}
if e.defaultEvaluator != nil {
e.defaultEvaluator.vectorizable = Vectorizable(e.defaultEvaluator.exprs)
}
return e
}
// Run evaluates all the expressions hold by this EvaluatorSuit.
// NOTE: "defaultEvaluator" must be evaluated before "columnEvaluator".
func (e *EvaluatorSuit) Run(ctx context.Context, input, output *chunk.Chunk) error {
if e.defaultEvaluator != nil {
err := e.defaultEvaluator.run(ctx, input, output)
if err != nil {
return errors.Trace(err)
}
}
if e.columnEvaluator != nil {
e.columnEvaluator.run(ctx, input, output)
}
return nil
}

View File

@ -101,6 +101,16 @@ func (c *Chunk) addColumnByFieldType(fieldTp *types.FieldType, initCap int) {
}
}
// MakeRef makes column in "dstColIdx" reference to column in "srcColIdx".
func (c *Chunk) MakeRef(srcColIdx, dstColIdx int) {
c.columns[dstColIdx] = c.columns[srcColIdx]
}
// SwapColumn swaps column "c.columns[colIdx]" with column "other.columns[otherIdx]".
func (c *Chunk) SwapColumn(colIdx int, other *Chunk, otherIdx int) {
c.columns[colIdx], other.columns[otherIdx] = other.columns[otherIdx], c.columns[colIdx]
}
// SwapColumns swaps columns with another Chunk.
func (c *Chunk) SwapColumns(other *Chunk) {
c.columns, other.columns = other.columns, c.columns