From fb429408c97c71fd276c91cc2f289871a75b6b6c Mon Sep 17 00:00:00 2001 From: Han Fei Date: Mon, 23 Jan 2017 16:23:47 +0800 Subject: [PATCH] *: make schema as a pointer. (#2533) --- executor/aggregate_test.go | 4 +- executor/executor.go | 64 +++++++++++++++--------------- executor/executor_agg.go | 8 ++-- executor/executor_ddl.go | 4 +- executor/executor_distsql.go | 6 +-- executor/executor_join.go | 16 ++++---- executor/executor_set.go | 4 +- executor/executor_simple.go | 4 +- executor/executor_write.go | 22 +++++----- executor/explain.go | 4 +- executor/grant.go | 4 +- executor/prepared.go | 12 +++--- executor/show.go | 4 +- executor/union_scan.go | 4 +- expression/aggregation.go | 12 +++--- expression/column.go | 8 ++-- expression/constant_propagation.go | 2 +- expression/expression.go | 21 +++++----- expression/scalar_function.go | 4 +- expression/schema.go | 39 +++++++++--------- expression/util.go | 2 +- plan/aggregation_push_down.go | 12 +++--- plan/build_key_info.go | 6 +-- plan/column_pruning.go | 2 +- plan/expression_rewriter.go | 10 ++--- plan/logical_plan_builder.go | 19 ++++----- plan/physical_plan_builder.go | 2 +- plan/physical_plans.go | 12 +++--- plan/plan.go | 10 ++--- plan/planbuilder.go | 43 ++++++++++++-------- plan/plans.go | 2 +- plan/predicate_push_down.go | 4 +- 32 files changed, 191 insertions(+), 179 deletions(-) diff --git a/executor/aggregate_test.go b/executor/aggregate_test.go index f2b0490062d7b..108214097713a 100644 --- a/executor/aggregate_test.go +++ b/executor/aggregate_test.go @@ -33,8 +33,8 @@ type MockExec struct { curRowIdx int } -func (m *MockExec) Schema() expression.Schema { - return expression.Schema{} +func (m *MockExec) Schema() *expression.Schema { + return expression.NewSchema() } func (m *MockExec) Fields() []*ast.ResultField { diff --git a/executor/executor.go b/executor/executor.go index 2344de4899649..61f8b85c87363 100644 --- a/executor/executor.go +++ b/executor/executor.go @@ -104,12 +104,12 @@ type RowKeyEntry struct { type Executor interface { Next() (*Row, error) Close() error - Schema() expression.Schema + Schema() *expression.Schema } // ShowDDLExec represents a show DDL executor. type ShowDDLExec struct { - schema expression.Schema + schema *expression.Schema ctx context.Context ddlInfo *inspectkv.DDLInfo bgInfo *inspectkv.DDLInfo @@ -117,7 +117,7 @@ type ShowDDLExec struct { } // Schema implements the Executor Schema interface. -func (e *ShowDDLExec) Schema() expression.Schema { +func (e *ShowDDLExec) Schema() *expression.Schema { return e.schema } @@ -172,8 +172,8 @@ type CheckTableExec struct { } // Schema implements the Executor Schema interface. -func (e *CheckTableExec) Schema() expression.Schema { - return expression.NewSchema(nil) +func (e *CheckTableExec) Schema() *expression.Schema { + return expression.NewSchema() } // Next implements the Executor Next interface. @@ -217,11 +217,11 @@ type SelectLockExec struct { Src Executor Lock ast.SelectLockType ctx context.Context - schema expression.Schema + schema *expression.Schema } // Schema implements the Executor Schema interface. -func (e *SelectLockExec) Schema() expression.Schema { +func (e *SelectLockExec) Schema() *expression.Schema { return e.schema } @@ -260,11 +260,11 @@ type LimitExec struct { Offset uint64 Count uint64 Idx uint64 - schema expression.Schema + schema *expression.Schema } // Schema implements the Executor Schema interface. -func (e *LimitExec) Schema() expression.Schema { +func (e *LimitExec) Schema() *expression.Schema { return e.schema } @@ -315,7 +315,7 @@ type ReverseExec struct { } // Schema implements the Executor Schema interface. -func (e *ReverseExec) Schema() expression.Schema { +func (e *ReverseExec) Schema() *expression.Schema { return e.Src.Schema() } @@ -381,14 +381,14 @@ func init() { // ProjectionExec represents a select fields executor. type ProjectionExec struct { Src Executor - schema expression.Schema + schema *expression.Schema executed bool ctx context.Context exprs []expression.Expression } // Schema implements the Executor Schema interface. -func (e *ProjectionExec) Schema() expression.Schema { +func (e *ProjectionExec) Schema() *expression.Schema { return e.schema } @@ -436,7 +436,7 @@ func (e *ProjectionExec) Close() error { // TableDualExec represents a dual table executor. type TableDualExec struct { - schema expression.Schema + schema *expression.Schema executed bool } @@ -446,7 +446,7 @@ func (e *TableDualExec) Init() { } // Schema implements the Executor Schema interface. -func (e *TableDualExec) Schema() expression.Schema { +func (e *TableDualExec) Schema() *expression.Schema { return e.schema } @@ -469,11 +469,11 @@ type SelectionExec struct { Src Executor Condition expression.Expression ctx context.Context - schema expression.Schema + schema *expression.Schema } // Schema implements the Executor Schema interface. -func (e *SelectionExec) Schema() expression.Schema { +func (e *SelectionExec) Schema() *expression.Schema { return e.schema } @@ -511,7 +511,7 @@ type TableScanExec struct { seekHandle int64 iter kv.Iterator cursor int - schema expression.Schema + schema *expression.Schema columns []*model.ColumnInfo isInfoSchema bool @@ -520,7 +520,7 @@ type TableScanExec struct { } // Schema implements the Executor Schema interface. -func (e *TableScanExec) Schema() expression.Schema { +func (e *TableScanExec) Schema() *expression.Schema { return e.schema } @@ -648,7 +648,7 @@ type SortExec struct { Idx int fetched bool err error - schema expression.Schema + schema *expression.Schema } // Close implements the Executor Close interface. @@ -659,7 +659,7 @@ func (e *SortExec) Close() error { } // Schema implements the Executor Schema interface. -func (e *SortExec) Schema() expression.Schema { +func (e *SortExec) Schema() *expression.Schema { return e.schema } @@ -848,13 +848,13 @@ func (e *TopnExec) Next() (*Row, error) { // ExistsExec represents exists executor. type ExistsExec struct { - schema expression.Schema + schema *expression.Schema Src Executor evaluated bool } // Schema implements the Executor Schema interface. -func (e *ExistsExec) Schema() expression.Schema { +func (e *ExistsExec) Schema() *expression.Schema { return e.schema } @@ -881,13 +881,13 @@ func (e *ExistsExec) Next() (*Row, error) { // MaxOneRowExec checks if the number of rows that a query returns is at maximum one. // It's built from subquery expression. type MaxOneRowExec struct { - schema expression.Schema + schema *expression.Schema Src Executor evaluated bool } // Schema implements the Executor Schema interface. -func (e *MaxOneRowExec) Schema() expression.Schema { +func (e *MaxOneRowExec) Schema() *expression.Schema { return e.schema } @@ -925,13 +925,13 @@ func (e *MaxOneRowExec) Next() (*Row, error) { // For example, in the 'SELECT a from t order by b' statement, // 'b' is needed for ordering, but not needed in the result. type TrimExec struct { - schema expression.Schema + schema *expression.Schema Src Executor len int } // Schema implements the Executor Schema interface. -func (e *TrimExec) Schema() expression.Schema { +func (e *TrimExec) Schema() *expression.Schema { return e.schema } @@ -957,7 +957,7 @@ func (e *TrimExec) Next() (*Row, error) { // UnionExec has multiple source Executors, it executes them sequentially, and do conversion to the same type // as source Executors may has different field type, we need to do conversion. type UnionExec struct { - schema expression.Schema + schema *expression.Schema Srcs []Executor ctx context.Context inited bool @@ -971,7 +971,7 @@ type UnionExec struct { } // Schema implements the Executor Schema interface. -func (e *UnionExec) Schema() expression.Schema { +func (e *UnionExec) Schema() *expression.Schema { return e.schema } @@ -1074,11 +1074,11 @@ func (e *UnionExec) Close() error { // DummyScanExec returns zero results, when some where condition never match, there won't be any // rows to return, so DummyScan is used to avoid real scan on KV. type DummyScanExec struct { - schema expression.Schema + schema *expression.Schema } // Schema implements the Executor Schema interface. -func (e *DummyScanExec) Schema() expression.Schema { +func (e *DummyScanExec) Schema() *expression.Schema { return e.schema } @@ -1095,7 +1095,7 @@ func (e *DummyScanExec) Next() (*Row, error) { // CacheExec represents Cache executor. // it stores the return values of the executor of its child node. type CacheExec struct { - schema expression.Schema + schema *expression.Schema Src Executor storedRows []*Row cursor int @@ -1103,7 +1103,7 @@ type CacheExec struct { } // Schema implements the Executor Schema interface. -func (e *CacheExec) Schema() expression.Schema { +func (e *CacheExec) Schema() *expression.Schema { return e.schema } diff --git a/executor/executor_agg.go b/executor/executor_agg.go index 7244840676463..617f29317be91 100644 --- a/executor/executor_agg.go +++ b/executor/executor_agg.go @@ -27,7 +27,7 @@ import ( // and updates all the items in AggFuncs. type HashAggExec struct { Src Executor - schema expression.Schema + schema *expression.Schema executed bool hasGby bool aggType plan.AggregationType @@ -51,7 +51,7 @@ func (e *HashAggExec) Close() error { } // Schema implements the Executor Schema interface. -func (e *HashAggExec) Schema() expression.Schema { +func (e *HashAggExec) Schema() *expression.Schema { return e.schema } @@ -154,7 +154,7 @@ func (e *HashAggExec) innerNext() (ret bool, err error) { // When Next() is called, it will return a result for the same group. type StreamAggExec struct { Src Executor - schema expression.Schema + schema *expression.Schema executed bool hasData bool Ctx context.Context @@ -176,7 +176,7 @@ func (e *StreamAggExec) Close() error { } // Schema implements the Executor Schema interface. -func (e *StreamAggExec) Schema() expression.Schema { +func (e *StreamAggExec) Schema() *expression.Schema { return e.schema } diff --git a/executor/executor_ddl.go b/executor/executor_ddl.go index c8b23801c0ae5..dd2cfce745c9d 100644 --- a/executor/executor_ddl.go +++ b/executor/executor_ddl.go @@ -42,8 +42,8 @@ type DDLExec struct { } // Schema implements the Executor Schema interface. -func (e *DDLExec) Schema() expression.Schema { - return expression.NewSchema(nil) +func (e *DDLExec) Schema() *expression.Schema { + return expression.NewSchema() } // Next implements Execution Next interface. diff --git a/executor/executor_distsql.go b/executor/executor_distsql.go index 403c98c1f89b1..e8bbd6fcefe7f 100644 --- a/executor/executor_distsql.go +++ b/executor/executor_distsql.go @@ -368,7 +368,7 @@ type XSelectIndexExec struct { } // Schema implements Exec Schema interface. -func (e *XSelectIndexExec) Schema() expression.Schema { +func (e *XSelectIndexExec) Schema() *expression.Schema { return e.indexPlan.GetSchema() } @@ -762,7 +762,7 @@ type XSelectTableExec struct { where *tipb.Expr Columns []*model.ColumnInfo - schema expression.Schema + schema *expression.Schema ranges []plan.TableRange desc bool limitCount *int64 @@ -790,7 +790,7 @@ type XSelectTableExec struct { } // Schema implements the Executor Schema interface. -func (e *XSelectTableExec) Schema() expression.Schema { +func (e *XSelectTableExec) Schema() *expression.Schema { return e.schema } diff --git a/executor/executor_join.go b/executor/executor_join.go index 3a7d156b7d2f8..b32413997ccf3 100644 --- a/executor/executor_join.go +++ b/executor/executor_join.go @@ -44,7 +44,7 @@ type HashJoinExec struct { smallFilter expression.Expression bigFilter expression.Expression otherFilter expression.Expression - schema expression.Schema + schema *expression.Schema outer bool leftSmall bool cursor int @@ -130,7 +130,7 @@ func getHashKey(sc *variable.StatementContext, cols []*expression.Column, row *R } // Schema implements the Executor Schema interface. -func (e *HashJoinExec) Schema() expression.Schema { +func (e *HashJoinExec) Schema() *expression.Schema { return e.schema } @@ -418,11 +418,11 @@ type NestedLoopJoinExec struct { SmallFilter expression.Expression BigFilter expression.Expression OtherFilter expression.Expression - schema expression.Schema + schema *expression.Schema } // Schema implements Executor interface. -func (e *NestedLoopJoinExec) Schema() expression.Schema { +func (e *NestedLoopJoinExec) Schema() *expression.Schema { return e.schema } @@ -552,7 +552,7 @@ type HashSemiJoinExec struct { smallFilter expression.Expression bigFilter expression.Expression otherFilter expression.Expression - schema expression.Schema + schema *expression.Schema resultRows []*Row // In auxMode, the result row always returns with an extra column which stores a boolean // or NULL value to indicate if this row is matched. @@ -577,7 +577,7 @@ func (e *HashSemiJoinExec) Close() error { } // Schema implements the Executor Schema interface. -func (e *HashSemiJoinExec) Schema() expression.Schema { +func (e *HashSemiJoinExec) Schema() *expression.Schema { return e.schema } @@ -742,11 +742,11 @@ type ApplyJoinExec struct { outerSchema []*expression.CorrelatedColumn cursor int resultRows []*Row - schema expression.Schema + schema *expression.Schema } // Schema implements the Executor interface. -func (e *ApplyJoinExec) Schema() expression.Schema { +func (e *ApplyJoinExec) Schema() *expression.Schema { return e.schema } diff --git a/executor/executor_set.go b/executor/executor_set.go index 27fbf5540e4e8..f341ac7780b48 100644 --- a/executor/executor_set.go +++ b/executor/executor_set.go @@ -135,8 +135,8 @@ func (e *SetExecutor) executeSet() error { } // Schema implements the Executor Schema interface. -func (e *SetExecutor) Schema() expression.Schema { - return expression.NewSchema(nil) +func (e *SetExecutor) Schema() *expression.Schema { + return expression.NewSchema() } // Close implements the Executor Close interface. diff --git a/executor/executor_simple.go b/executor/executor_simple.go index 065967180a533..024dce8289002 100644 --- a/executor/executor_simple.go +++ b/executor/executor_simple.go @@ -49,8 +49,8 @@ type SimpleExec struct { } // Schema implements the Executor Schema interface. -func (e *SimpleExec) Schema() expression.Schema { - return expression.NewSchema(nil) +func (e *SimpleExec) Schema() *expression.Schema { + return expression.NewSchema() } // Next implements Execution Next interface. diff --git a/executor/executor_write.go b/executor/executor_write.go index 9538b8c5123fb..2e621c269f39c 100644 --- a/executor/executor_write.go +++ b/executor/executor_write.go @@ -154,8 +154,8 @@ type DeleteExec struct { } // Schema implements the Executor Schema interface. -func (e *DeleteExec) Schema() expression.Schema { - return expression.NewSchema(nil) +func (e *DeleteExec) Schema() *expression.Schema { + return expression.NewSchema() } // Next implements the Executor Next interface. @@ -564,8 +564,8 @@ func (e *LoadData) Next() (*Row, error) { } // Schema implements the Executor Schema interface. -func (e *LoadData) Schema() expression.Schema { - return expression.NewSchema(nil) +func (e *LoadData) Schema() *expression.Schema { + return expression.NewSchema() } // Close implements the Executor Close interface. @@ -601,8 +601,8 @@ type InsertExec struct { } // Schema implements the Executor Schema interface. -func (e *InsertExec) Schema() expression.Schema { - return expression.NewSchema(nil) +func (e *InsertExec) Schema() *expression.Schema { + return expression.NewSchema() } // Next implements the Executor Next interface. @@ -989,8 +989,8 @@ type ReplaceExec struct { } // Schema implements the Executor Schema interface. -func (e *ReplaceExec) Schema() expression.Schema { - return expression.NewSchema(nil) +func (e *ReplaceExec) Schema() *expression.Schema { + return expression.NewSchema() } // Close implements the Executor Close interface. @@ -1096,8 +1096,8 @@ type UpdateExec struct { } // Schema implements the Executor Schema interface. -func (e *UpdateExec) Schema() expression.Schema { - return expression.NewSchema(nil) +func (e *UpdateExec) Schema() *expression.Schema { + return expression.NewSchema() } // Next implements the Executor Next interface. @@ -1187,7 +1187,7 @@ func (e *UpdateExec) fetchRows() error { } } -func getTableOffset(schema expression.Schema, entry *RowKeyEntry) int { +func getTableOffset(schema *expression.Schema, entry *RowKeyEntry) int { t := entry.Tbl var tblName string if entry.TableAsName == nil || len(entry.TableAsName.L) == 0 { diff --git a/executor/explain.go b/executor/explain.go index b2f1590483f41..e9e8acfad6ad0 100644 --- a/executor/explain.go +++ b/executor/explain.go @@ -26,13 +26,13 @@ import ( // See https://dev.mysql.com/doc/refman/5.7/en/explain-output.html type ExplainExec struct { StmtPlan plan.Plan - schema expression.Schema + schema *expression.Schema rows []*Row cursor int } // Schema implements the Executor Schema interface. -func (e *ExplainExec) Schema() expression.Schema { +func (e *ExplainExec) Schema() *expression.Schema { return e.schema } diff --git a/executor/grant.go b/executor/grant.go index 2adfb46b6489f..375f5a98a47b5 100644 --- a/executor/grant.go +++ b/executor/grant.go @@ -50,8 +50,8 @@ type GrantExec struct { } // Schema implements the Executor Schema interface. -func (e *GrantExec) Schema() expression.Schema { - return expression.NewSchema(nil) +func (e *GrantExec) Schema() *expression.Schema { + return expression.NewSchema() } // Next implements Execution Next interface. diff --git a/executor/prepared.go b/executor/prepared.go index 80fc52e59af5b..1b13dd450deba 100644 --- a/executor/prepared.go +++ b/executor/prepared.go @@ -83,9 +83,9 @@ type PrepareExec struct { } // Schema implements the Executor Schema interface. -func (e *PrepareExec) Schema() expression.Schema { +func (e *PrepareExec) Schema() *expression.Schema { // Will never be called. - return expression.NewSchema(nil) + return expression.NewSchema() } // Next implements the Executor Next interface. @@ -179,9 +179,9 @@ type ExecuteExec struct { } // Schema implements the Executor Schema interface. -func (e *ExecuteExec) Schema() expression.Schema { +func (e *ExecuteExec) Schema() *expression.Schema { // Will never be called. - return expression.NewSchema(nil) + return expression.NewSchema() } // Next implements the Executor Next interface. @@ -256,9 +256,9 @@ type DeallocateExec struct { } // Schema implements the Executor Schema interface. -func (e *DeallocateExec) Schema() expression.Schema { +func (e *DeallocateExec) Schema() *expression.Schema { // Will never be called. - return expression.NewSchema(nil) + return expression.NewSchema() } // Next implements the Executor Next interface. diff --git a/executor/show.go b/executor/show.go index 74d720294b8b5..9d1938ad09384 100644 --- a/executor/show.go +++ b/executor/show.go @@ -47,7 +47,7 @@ type ShowExec struct { // Used by show variables GlobalScope bool - schema expression.Schema + schema *expression.Schema ctx context.Context is infoschema.InfoSchema @@ -57,7 +57,7 @@ type ShowExec struct { } // Schema implements the Executor Schema interface. -func (e *ShowExec) Schema() expression.Schema { +func (e *ShowExec) Schema() *expression.Schema { return e.schema } diff --git a/executor/union_scan.go b/executor/union_scan.go index 8a00ffdf203fe..f50f41b1ff9d2 100644 --- a/executor/union_scan.go +++ b/executor/union_scan.go @@ -98,11 +98,11 @@ type UnionScanExec struct { cursor int sortErr error snapshotRow *Row - schema expression.Schema + schema *expression.Schema } // Schema implements the Executor Schema interface. -func (us *UnionScanExec) Schema() expression.Schema { +func (us *UnionScanExec) Schema() *expression.Schema { return us.schema } diff --git a/expression/aggregation.go b/expression/aggregation.go index bf443692eccdb..eb5687d38fa7f 100644 --- a/expression/aggregation.go +++ b/expression/aggregation.go @@ -81,7 +81,7 @@ type AggregationFunction interface { // CalculateDefaultValue gets the default value when the aggregate function's input is null. // The input stands for the schema of Aggregation's child. If the function can't produce a default value, the second // return value will be false. - CalculateDefaultValue(schema Schema, ctx context.Context) (types.Datum, bool) + CalculateDefaultValue(schema *Schema, ctx context.Context) (types.Datum, bool) } // aggEvaluateContext is used to store intermediate result when calculating aggregate functions. @@ -182,7 +182,7 @@ func newAggFunc(name string, args []Expression, dist bool) aggFunction { } // CalculateDefaultValue implements AggregationFunction interface. -func (af *aggFunction) CalculateDefaultValue(schema Schema, ctx context.Context) (types.Datum, bool) { +func (af *aggFunction) CalculateDefaultValue(schema *Schema, ctx context.Context) (types.Datum, bool) { return types.Datum{}, false } @@ -343,7 +343,7 @@ func (sf *sumFunction) GetStreamResult() (d types.Datum) { } // CalculateDefaultValue implements AggregationFunction interface. -func (sf *sumFunction) CalculateDefaultValue(schema Schema, ctx context.Context) (d types.Datum, valid bool) { +func (sf *sumFunction) CalculateDefaultValue(schema *Schema, ctx context.Context) (d types.Datum, valid bool) { arg := sf.Args[0] result, err := EvaluateExprWithNull(ctx, schema, arg) if err != nil { @@ -384,7 +384,7 @@ func (cf *countFunction) Clone() AggregationFunction { } // CalculateDefaultValue implements AggregationFunction interface. -func (cf *countFunction) CalculateDefaultValue(schema Schema, ctx context.Context) (d types.Datum, valid bool) { +func (cf *countFunction) CalculateDefaultValue(schema *Schema, ctx context.Context) (d types.Datum, valid bool) { for _, arg := range cf.Args { result, err := EvaluateExprWithNull(ctx, schema, arg) if err != nil { @@ -727,7 +727,7 @@ func (mmf *maxMinFunction) Clone() AggregationFunction { } // CalculateDefaultValue implements AggregationFunction interface. -func (mmf *maxMinFunction) CalculateDefaultValue(schema Schema, ctx context.Context) (d types.Datum, valid bool) { +func (mmf *maxMinFunction) CalculateDefaultValue(schema *Schema, ctx context.Context) (d types.Datum, valid bool) { arg := mmf.Args[0] result, err := EvaluateExprWithNull(ctx, schema, arg) if err != nil { @@ -887,7 +887,7 @@ func (ff *firstRowFunction) GetStreamResult() (d types.Datum) { } // CalculateDefaultValue implements AggregationFunction interface. -func (ff *firstRowFunction) CalculateDefaultValue(schema Schema, ctx context.Context) (d types.Datum, valid bool) { +func (ff *firstRowFunction) CalculateDefaultValue(schema *Schema, ctx context.Context) (d types.Datum, valid bool) { arg := ff.Args[0] result, err := EvaluateExprWithNull(ctx, schema, arg) if err != nil { diff --git a/expression/column.go b/expression/column.go index 55f1b4fed4833..58c686d3f273d 100644 --- a/expression/column.go +++ b/expression/column.go @@ -55,7 +55,7 @@ func (col *CorrelatedColumn) IsCorrelated() bool { } // Decorrelate implements Expression interface. -func (col *CorrelatedColumn) Decorrelate(schema Schema) Expression { +func (col *CorrelatedColumn) Decorrelate(schema *Schema) Expression { if schema.GetColumnIndex(&col.Column) == -1 { return col } @@ -63,7 +63,7 @@ func (col *CorrelatedColumn) Decorrelate(schema Schema) Expression { } // ResolveIndices implements Expression interface. -func (col *CorrelatedColumn) ResolveIndices(_ Schema) { +func (col *CorrelatedColumn) ResolveIndices(_ *Schema) { } // Column represents a column. @@ -136,7 +136,7 @@ func (col *Column) IsCorrelated() bool { } // Decorrelate implements Expression interface. -func (col *Column) Decorrelate(_ Schema) Expression { +func (col *Column) Decorrelate(_ *Schema) Expression { return col } @@ -150,7 +150,7 @@ func (col *Column) HashCode() []byte { } // ResolveIndices implements Expression interface. -func (col *Column) ResolveIndices(schema Schema) { +func (col *Column) ResolveIndices(schema *Schema) { col.Index = schema.GetColumnIndex(col) // If col's index equals to -1, it means a internal logic error happens. if col.Index == -1 { diff --git a/expression/constant_propagation.go b/expression/constant_propagation.go index 5147c6050ef19..dbd99075d87c8 100644 --- a/expression/constant_propagation.go +++ b/expression/constant_propagation.go @@ -129,7 +129,7 @@ func (s *propagateConstantSolver) propagateEQ() { } for i, cond := range s.conditions { if !visited[i] { - s.conditions[i] = ColumnSubstitute(cond, NewSchema(cols), cons) + s.conditions[i] = ColumnSubstitute(cond, NewSchema(cols...), cons) } } } diff --git a/expression/expression.go b/expression/expression.go index 0dc7d29cf122b..cf9cc7a0e54f2 100644 --- a/expression/expression.go +++ b/expression/expression.go @@ -68,10 +68,10 @@ type Expression interface { IsCorrelated() bool // Decorrelate try to decorrelate the expression by schema. - Decorrelate(schema Schema) Expression + Decorrelate(schema *Schema) Expression // ResolveIndices resolves indices by the given schema. - ResolveIndices(schema Schema) + ResolveIndices(schema *Schema) } // EvalBool evaluates expression to a boolean value. @@ -161,7 +161,7 @@ func (c *Constant) IsCorrelated() bool { } // Decorrelate implements Expression interface. -func (c *Constant) Decorrelate(_ Schema) Expression { +func (c *Constant) Decorrelate(_ *Schema) Expression { return c } @@ -173,7 +173,7 @@ func (c *Constant) HashCode() []byte { } // ResolveIndices implements Expression interface. -func (c *Constant) ResolveIndices(_ Schema) { +func (c *Constant) ResolveIndices(_ *Schema) { } // composeConditionWithBinaryOp composes condition with binary operator into a balance deep tree, which benefits a lot for pb decoder/encoder. @@ -248,7 +248,7 @@ func SplitDNFItems(onExpr Expression) []Expression { // EvaluateExprWithNull sets columns in schema as null and calculate the final result of the scalar function. // If the Expression is a non-constant value, it means the result is unknown. -func EvaluateExprWithNull(ctx context.Context, schema Schema, expr Expression) (Expression, error) { +func EvaluateExprWithNull(ctx context.Context, schema *Schema, expr Expression) (Expression, error) { switch x := expr.(type) { case *ScalarFunction: var err error @@ -276,8 +276,8 @@ func EvaluateExprWithNull(ctx context.Context, schema Schema, expr Expression) ( } // TableInfo2Schema converts table info to schema. -func TableInfo2Schema(tbl *model.TableInfo) Schema { - schema := NewSchema(make([]*Column, 0, len(tbl.Columns))) +func TableInfo2Schema(tbl *model.TableInfo) *Schema { + cols := make([]*Column, 0, len(tbl.Columns)) keys := make([]KeyInfo, 0, len(tbl.Indices)+1) for i, col := range tbl.Columns { newCol := &Column{ @@ -286,7 +286,7 @@ func TableInfo2Schema(tbl *model.TableInfo) Schema { RetType: &col.FieldType, Position: i, } - schema.Append(newCol) + cols = append(cols, newCol) } for _, idx := range tbl.Indices { if !idx.Unique || idx.State != model.StatePublic { @@ -301,7 +301,7 @@ func TableInfo2Schema(tbl *model.TableInfo) Schema { if !mysql.HasNotNullFlag(col.Flag) { break } - newKey = append(newKey, schema.Columns[i]) + newKey = append(newKey, cols[i]) find = true break } @@ -318,11 +318,12 @@ func TableInfo2Schema(tbl *model.TableInfo) Schema { if tbl.PKIsHandle { for i, col := range tbl.Columns { if mysql.HasPriKeyFlag(col.Flag) { - keys = append(keys, []*Column{schema.Columns[i]}) + keys = append(keys, KeyInfo{cols[i]}) break } } } + schema := NewSchema(cols...) schema.SetUniqueKeys(keys) return schema } diff --git a/expression/scalar_function.go b/expression/scalar_function.go index c1b6ac20a25f4..dd3c9077d1d77 100644 --- a/expression/scalar_function.go +++ b/expression/scalar_function.go @@ -133,7 +133,7 @@ func (sf *ScalarFunction) IsCorrelated() bool { } // Decorrelate implements Expression interface. -func (sf *ScalarFunction) Decorrelate(schema Schema) Expression { +func (sf *ScalarFunction) Decorrelate(schema *Schema) Expression { for i, arg := range sf.GetArgs() { sf.GetArgs()[i] = arg.Decorrelate(schema) } @@ -160,7 +160,7 @@ func (sf *ScalarFunction) HashCode() []byte { } // ResolveIndices implements Expression interface. -func (sf *ScalarFunction) ResolveIndices(schema Schema) { +func (sf *ScalarFunction) ResolveIndices(schema *Schema) { for _, arg := range sf.GetArgs() { arg.ResolveIndices(schema) } diff --git a/expression/schema.go b/expression/schema.go index 98fb42197dd46..ab95821a01294 100644 --- a/expression/schema.go +++ b/expression/schema.go @@ -27,8 +27,7 @@ type KeyInfo []*Column func (ki KeyInfo) Clone() KeyInfo { result := make([]*Column, 0, len(ki)) for _, col := range ki { - newCol := *col - result = append(result, &newCol) + result = append(result, col.Clone().(*Column)) } return result } @@ -40,7 +39,7 @@ type Schema struct { } // String implements fmt.Stringer interface. -func (s Schema) String() string { +func (s *Schema) String() string { colStrs := make([]string, 0, len(s.Columns)) for _, col := range s.Columns { colStrs = append(colStrs, col.String()) @@ -57,23 +56,23 @@ func (s Schema) String() string { } // Clone copies the total schema. -func (s Schema) Clone() Schema { - result := NewSchema(make([]*Column, 0, s.Len())) +func (s *Schema) Clone() *Schema { + cols := make([]*Column, 0, s.Len()) keys := make([]KeyInfo, 0, len(s.Keys)) for _, col := range s.Columns { - newCol := *col - result.Append(&newCol) + cols = append(cols, col.Clone().(*Column)) } for _, key := range s.Keys { keys = append(keys, key.Clone()) } - result.SetUniqueKeys(keys) - return result + schema := NewSchema(cols...) + schema.SetUniqueKeys(keys) + return schema } // FindColumn finds an Column from schema for a ast.ColumnName. It compares the db/table/column names. // If there are more than one result, it will raise ambiguous error. -func (s Schema) FindColumn(astCol *ast.ColumnName) (*Column, error) { +func (s *Schema) FindColumn(astCol *ast.ColumnName) (*Column, error) { dbName, tblName, colName := astCol.Schema, astCol.Table, astCol.Name idx := -1 for i, col := range s.Columns { @@ -94,7 +93,7 @@ func (s Schema) FindColumn(astCol *ast.ColumnName) (*Column, error) { } // RetrieveColumn retrieves column in expression from the columns in schema. -func (s Schema) RetrieveColumn(col *Column) *Column { +func (s *Schema) RetrieveColumn(col *Column) *Column { index := s.GetColumnIndex(col) if index != -1 { return s.Columns[index] @@ -103,7 +102,7 @@ func (s Schema) RetrieveColumn(col *Column) *Column { } // GetColumnIndex finds the index for a column. -func (s Schema) GetColumnIndex(col *Column) int { +func (s *Schema) GetColumnIndex(col *Column) int { for i, c := range s.Columns { if c.FromID == col.FromID && c.Position == col.Position { return i @@ -113,13 +112,13 @@ func (s Schema) GetColumnIndex(col *Column) int { } // Len returns the number of columns in schema. -func (s Schema) Len() int { +func (s *Schema) Len() int { return len(s.Columns) } // Append append new column to the columns stored in schema. -func (s *Schema) Append(col *Column) { - s.Columns = append(s.Columns, col) +func (s *Schema) Append(col ...*Column) { + s.Columns = append(s.Columns, col...) } // SetUniqueKeys will set the value of Schema.Keys. @@ -129,7 +128,7 @@ func (s *Schema) SetUniqueKeys(keys []KeyInfo) { // GetColumnsIndices will return a slice which contains the position of each column in schema. // If there is one column that doesn't match, nil will be returned. -func (s Schema) GetColumnsIndices(cols []*Column) (ret []int) { +func (s *Schema) GetColumnsIndices(cols []*Column) (ret []int) { ret = make([]int, 0, len(cols)) for _, col := range cols { pos := s.GetColumnIndex(col) @@ -143,15 +142,15 @@ func (s Schema) GetColumnsIndices(cols []*Column) (ret []int) { } // MergeSchema will merge two schema into one schema. -func MergeSchema(lSchema, rSchema Schema) Schema { +func MergeSchema(lSchema, rSchema *Schema) *Schema { tmpL := lSchema.Clone() tmpR := rSchema.Clone() - ret := NewSchema(append(tmpL.Columns, tmpR.Columns...)) + ret := NewSchema(append(tmpL.Columns, tmpR.Columns...)...) ret.SetUniqueKeys(append(tmpL.Keys, tmpR.Keys...)) return ret } // NewSchema returns a schema made by its parameter. -func NewSchema(cols []*Column) Schema { - return Schema{Columns: cols} +func NewSchema(cols ...*Column) *Schema { + return &Schema{Columns: cols} } diff --git a/expression/util.go b/expression/util.go index 056c235d2f4e4..f3b4f9e14fa44 100644 --- a/expression/util.go +++ b/expression/util.go @@ -37,7 +37,7 @@ func ExtractColumns(expr Expression) (cols []*Column) { // ColumnSubstitute substitutes the columns in filter to expressions in select fields. // e.g. select * from (select b as a from t) k where a < 10 => select * from (select b as a from t where b < 10) k. -func ColumnSubstitute(expr Expression, schema Schema, newExprs []Expression) Expression { +func ColumnSubstitute(expr Expression, schema *Schema, newExprs []Expression) Expression { switch v := expr.(type) { case *Column: id := schema.GetColumnIndex(v) diff --git a/plan/aggregation_push_down.go b/plan/aggregation_push_down.go index c3ce547c4591f..f09145d5301c2 100644 --- a/plan/aggregation_push_down.go +++ b/plan/aggregation_push_down.go @@ -49,7 +49,7 @@ func (a *aggPushDownSolver) isDecomposable(fun expression.AggregationFunction) b } // getAggFuncChildIdx gets which children it belongs to, 0 stands for left, 1 stands for right, -1 stands for both. -func (a *aggPushDownSolver) getAggFuncChildIdx(aggFunc expression.AggregationFunction, schema expression.Schema) int { +func (a *aggPushDownSolver) getAggFuncChildIdx(aggFunc expression.AggregationFunction, schema *expression.Schema) int { fromLeft, fromRight := false, false var cols []*expression.Column for _, arg := range aggFunc.GetArgs() { @@ -171,18 +171,18 @@ func (a *aggPushDownSolver) checkValidJoin(join *Join) bool { // decompose splits an aggregate function to two parts: a final mode function and a partial mode function. Currently // there are no differences between partial mode and complete mode, so we can confuse them. -func (a *aggPushDownSolver) decompose(aggFunc expression.AggregationFunction, schema expression.Schema, id string) ([]expression.AggregationFunction, expression.Schema) { +func (a *aggPushDownSolver) decompose(aggFunc expression.AggregationFunction, schema *expression.Schema, id string) ([]expression.AggregationFunction, *expression.Schema) { // Result is a slice because avg should be decomposed to sum and count. Currently we don't process this case. result := []expression.AggregationFunction{aggFunc.Clone()} for _, aggFunc := range result { schema.Append(&expression.Column{ ColName: model.NewCIStr(fmt.Sprintf("join_agg_%d", schema.Len())), // useless but for debug FromID: id, - Position: len(schema.Columns), + Position: schema.Len(), RetType: aggFunc.GetType(), }) } - aggFunc.SetArgs(expression.Column2Exprs(schema.Columns[len(schema.Columns)-len(result):])) + aggFunc.SetArgs(expression.Column2Exprs(schema.Columns[schema.Len()-len(result):])) aggFunc.SetMode(expression.FinalMode) return result, schema } @@ -253,7 +253,7 @@ func (a *aggPushDownSolver) makeNewAgg(aggFuncs []expression.AggregationFunction } agg.initIDAndContext(a.ctx) var newAggFuncs []expression.AggregationFunction - schema := expression.NewSchema(make([]*expression.Column, 0, len(aggFuncs))) + schema := expression.NewSchema(make([]*expression.Column, 0, len(aggFuncs))...) for _, aggFunc := range aggFuncs { var newFuncs []expression.AggregationFunction newFuncs, schema = a.decompose(aggFunc, schema, agg.GetID()) @@ -269,7 +269,7 @@ func (a *aggPushDownSolver) makeNewAgg(aggFuncs []expression.AggregationFunction return agg } -func (a *aggPushDownSolver) pushAggCrossUnion(agg *Aggregation, unionSchema expression.Schema, unionChild LogicalPlan) LogicalPlan { +func (a *aggPushDownSolver) pushAggCrossUnion(agg *Aggregation, unionSchema *expression.Schema, unionChild LogicalPlan) LogicalPlan { newAgg := &Aggregation{ AggFuncs: make([]expression.AggregationFunction, 0, len(agg.AggFuncs)), GroupByItems: make([]expression.Expression, 0, len(agg.GroupByItems)), diff --git a/plan/build_key_info.go b/plan/build_key_info.go index 5e5c6c923497f..d033b79344ec3 100644 --- a/plan/build_key_info.go +++ b/plan/build_key_info.go @@ -33,7 +33,7 @@ func (p *Aggregation) buildKeyInfo() { } // dealing with p.GroupbyCols // This is only used for optimization and needn't to be pushed up, so only one is enough. - schemaByGroupby := expression.NewSchema(p.groupByCols) + schemaByGroupby := expression.NewSchema(p.groupByCols...) for _, key := range p.GetChildren()[0].GetSchema().Keys { indices := schemaByGroupby.GetColumnsIndices(key) if indices == nil { @@ -50,8 +50,8 @@ func (p *Aggregation) buildKeyInfo() { // A bijection exists between columns of a projection's schema and this projection's Exprs. // Sometimes we need a schema made by expr of Exprs to convert a column in child's schema to a column in this projection's Schema. -func (p *Projection) buildSchemaByExprs() expression.Schema { - schema := expression.NewSchema(make([]*expression.Column, 0, p.schema.Len())) +func (p *Projection) buildSchemaByExprs() *expression.Schema { + schema := expression.NewSchema(make([]*expression.Column, 0, p.schema.Len())...) for _, expr := range p.Exprs { if col, isCol := expr.(*expression.Column); isCol { schema.Append(col) diff --git a/plan/column_pruning.go b/plan/column_pruning.go index 1c60d27c0ebcd..ba778ad9b3c5d 100644 --- a/plan/column_pruning.go +++ b/plan/column_pruning.go @@ -19,7 +19,7 @@ import ( "github.com/pingcap/tidb/expression" ) -func getUsedList(usedCols []*expression.Column, schema expression.Schema) []bool { +func getUsedList(usedCols []*expression.Column, schema *expression.Schema) []bool { used := make([]bool, schema.Len()) for _, col := range usedCols { idx := schema.GetColumnIndex(col) diff --git a/plan/expression_rewriter.go b/plan/expression_rewriter.go index 58942b1749024..893bc26bdcab2 100644 --- a/plan/expression_rewriter.go +++ b/plan/expression_rewriter.go @@ -87,7 +87,7 @@ func (b *planBuilder) rewrite(expr ast.ExprNode, p LogicalPlan, aggMapper map[*a type expressionRewriter struct { ctxStack []expression.Expression p LogicalPlan - schema expression.Schema + schema *expression.Schema err error aggrMap map[*ast.AggregateFuncExpr]int b *planBuilder @@ -301,7 +301,7 @@ func (er *expressionRewriter) handleOtherComparableSubq(lexpr, rexpr expression. Position: 0, RetType: aggFunc.GetType(), } - schema := expression.NewSchema([]*expression.Column{aggCol0}) + schema := expression.NewSchema(aggCol0) agg.SetSchema(schema) cond, _ := expression.NewFunction(er.ctx, cmpFunc, types.NewFieldType(mysql.TypeTiny), lexpr, aggCol0.Clone()) er.buildQuantifierPlan(agg, cond, rexpr, all) @@ -358,7 +358,7 @@ func (er *expressionRewriter) buildQuantifierPlan(agg *Aggregation, cond, rexpr } proj.self = proj proj.initIDAndContext(er.ctx) - proj.SetSchema(expression.NewSchema(joinSchema.Clone().Columns[:outerSchemaLen])) + proj.SetSchema(expression.NewSchema(joinSchema.Clone().Columns[:outerSchemaLen]...)) proj.Exprs = append(proj.Exprs, cond) proj.schema.Append(&expression.Column{ FromID: proj.id, @@ -396,7 +396,7 @@ func (er *expressionRewriter) handleNEAny(lexpr, rexpr expression.Expression, np Position: 1, RetType: countFunc.GetType(), } - agg.SetSchema(expression.NewSchema([]*expression.Column{firstRowResultCol, count})) + agg.SetSchema(expression.NewSchema(firstRowResultCol, count)) gtFunc, _ := expression.NewFunction(er.ctx, ast.GT, types.NewFieldType(mysql.TypeTiny), count.Clone(), expression.One) neCond, _ := expression.NewFunction(er.ctx, ast.NE, types.NewFieldType(mysql.TypeTiny), lexpr, firstRowResultCol.Clone()) cond := expression.ComposeDNFCondition(er.ctx, gtFunc, neCond) @@ -427,7 +427,7 @@ func (er *expressionRewriter) handleEQAll(lexpr, rexpr expression.Expression, np Position: 1, RetType: countFunc.GetType(), } - agg.SetSchema(expression.NewSchema([]*expression.Column{firstRowResultCol, count})) + agg.SetSchema(expression.NewSchema(firstRowResultCol, count)) leFunc, _ := expression.NewFunction(er.ctx, ast.LE, types.NewFieldType(mysql.TypeTiny), count.Clone(), expression.One) eqCond, _ := expression.NewFunction(er.ctx, ast.EQ, types.NewFieldType(mysql.TypeTiny), lexpr, firstRowResultCol.Clone()) cond := expression.ComposeCNFCondition(er.ctx, leFunc, eqCond) diff --git a/plan/logical_plan_builder.go b/plan/logical_plan_builder.go index 29afa9b64e56f..eb692b0861ab1 100644 --- a/plan/logical_plan_builder.go +++ b/plan/logical_plan_builder.go @@ -51,7 +51,7 @@ func (b *planBuilder) buildAggregation(p LogicalPlan, aggFuncList []*ast.Aggrega agg.self = agg agg.initIDAndContext(b.ctx) addChild(agg, p) - schema := expression.NewSchema(make([]*expression.Column, 0, len(aggFuncList)+p.GetSchema().Len())) + schema := expression.NewSchema(make([]*expression.Column, 0, len(aggFuncList)+p.GetSchema().Len())...) // aggIdxMap maps the old index to new index after applying common aggregation functions elimination. aggIndexMap := make(map[int]int) for i, aggFunc := range aggFuncList { @@ -267,7 +267,7 @@ func (b *planBuilder) buildProjection(p LogicalPlan, fields []*ast.SelectField, } proj.self = proj proj.initIDAndContext(b.ctx) - schema := expression.NewSchema(make([]*expression.Column, 0, len(fields))) + schema := expression.NewSchema(make([]*expression.Column, 0, len(fields))...) oldLen := 0 for _, field := range fields { newExpr, np, err := b.rewrite(field.Expr, p, mapper, true) @@ -536,7 +536,7 @@ type havingAndOrderbyExprResolver struct { aggMapper map[*ast.AggregateFuncExpr]int colMapper map[*ast.ColumnNameExpr]int gbyItems []*ast.ByItem - outerSchemas []expression.Schema + outerSchemas []*expression.Schema } // Enter implements Visitor interface. @@ -555,7 +555,7 @@ func (a *havingAndOrderbyExprResolver) Enter(n ast.Node) (node ast.Node, skipChi return n, false } -func (a *havingAndOrderbyExprResolver) resolveFromSchema(v *ast.ColumnNameExpr, schema expression.Schema) (int, error) { +func (a *havingAndOrderbyExprResolver) resolveFromSchema(v *ast.ColumnNameExpr, schema *expression.Schema) (int, error) { col, err := schema.FindColumn(v.Name) if err != nil { return -1, errors.Trace(err) @@ -707,7 +707,7 @@ func (b *planBuilder) extractAggFuncs(fields []*ast.SelectField) ([]*ast.Aggrega // gbyResolver resolves group by items from select fields. type gbyResolver struct { fields []*ast.SelectField - schema expression.Schema + schema *expression.Schema err error inExpr bool } @@ -898,7 +898,7 @@ func (b *planBuilder) buildTrim(p LogicalPlan, len int) LogicalPlan { trim.self = trim trim.initIDAndContext(b.ctx) addChild(trim, p) - schema := expression.NewSchema(p.GetSchema().Clone().Columns[:len]) + schema := expression.NewSchema(p.GetSchema().Clone().Columns[:len]...) trim.SetSchema(schema) trim.SetCorrelated() return trim @@ -908,6 +908,7 @@ func (b *planBuilder) buildTableDual() LogicalPlan { dual := &TableDual{baseLogicalPlan: newBaseLogicalPlan(Dual, b.allocator)} dual.self = dual dual.initIDAndContext(b.ctx) + dual.SetSchema(expression.NewSchema()) return dual } @@ -937,7 +938,7 @@ func (b *planBuilder) buildDataSource(tn *ast.TableName) LogicalPlan { p.self = p p.initIDAndContext(b.ctx) // Equal condition contains a column from previous joined table. - schema := expression.NewSchema(make([]*expression.Column, 0, len(tableInfo.Columns))) + schema := expression.NewSchema(make([]*expression.Column, 0, len(tableInfo.Columns))...) for i, col := range tableInfo.Columns { if b.inUpdateStmt { switch col.State { @@ -1019,7 +1020,7 @@ out: FromID: exists.id, RetType: types.NewFieldType(mysql.TypeTiny), ColName: model.NewCIStr("exists_col")} - exists.SetSchema(expression.NewSchema([]*expression.Column{newCol})) + exists.SetSchema(expression.NewSchema(newCol)) exists.SetCorrelated() return exists } @@ -1170,6 +1171,6 @@ func (b *planBuilder) buildDelete(delete *ast.DeleteStmt) LogicalPlan { del.self = del del.initIDAndContext(b.ctx) addChild(del, p) + del.SetSchema(expression.NewSchema()) return del - } diff --git a/plan/physical_plan_builder.go b/plan/physical_plan_builder.go index b8d6cf8828294..d688ecad30edf 100644 --- a/plan/physical_plan_builder.go +++ b/plan/physical_plan_builder.go @@ -491,7 +491,7 @@ func (p *Join) convert2PhysicalPlanLeft(prop *requiredProperty, innerJoin bool) } // replaceColsInPropBySchema replaces the columns in original prop with the columns in schema. -func replaceColsInPropBySchema(prop *requiredProperty, schema expression.Schema) *requiredProperty { +func replaceColsInPropBySchema(prop *requiredProperty, schema *expression.Schema) *requiredProperty { newProps := make([]*columnProp, 0, len(prop.props)) for _, p := range prop.props { idx := schema.GetColumnIndex(p.col) diff --git a/plan/physical_plans.go b/plan/physical_plans.go index 9d33228450b94..6cf39f2298afa 100644 --- a/plan/physical_plans.go +++ b/plan/physical_plans.go @@ -77,7 +77,7 @@ func (p *PhysicalMemTable) Copy() PhysicalPlan { // physicalDistSQLPlan means the plan that can be executed distributively. // We can push down other plan like selection, limit, aggregation, topn into this plan. type physicalDistSQLPlan interface { - addAggregation(ctx context.Context, agg *PhysicalAggregation) expression.Schema + addAggregation(ctx context.Context, agg *PhysicalAggregation) *expression.Schema addTopN(ctx context.Context, prop *requiredProperty) bool addLimit(limit *Limit) // scanCount means the original row count that need to be scanned and resultCount means the row count after scanning. @@ -260,9 +260,9 @@ func (p *physicalTableSource) addTopN(ctx context.Context, prop *requiredPropert return true } -func (p *physicalTableSource) addAggregation(ctx context.Context, agg *PhysicalAggregation) expression.Schema { +func (p *physicalTableSource) addAggregation(ctx context.Context, agg *PhysicalAggregation) *expression.Schema { if p.client == nil { - return expression.NewSchema(nil) + return expression.NewSchema() } sc := ctx.GetSessionVars().StmtCtx for _, f := range agg.AggFuncs { @@ -270,7 +270,7 @@ func (p *physicalTableSource) addAggregation(ctx context.Context, agg *PhysicalA if pb == nil { // When we fail to convert any agg function to PB struct, we should clear the environments. p.clearForAggPushDown() - return expression.NewSchema(nil) + return expression.NewSchema() } p.AggFuncsPB = append(p.AggFuncsPB, pb) p.aggFuncs = append(p.aggFuncs, f.Clone()) @@ -280,7 +280,7 @@ func (p *physicalTableSource) addAggregation(ctx context.Context, agg *PhysicalA if pb == nil { // When we fail to convert any group-by item to PB struct, we should clear the environments. p.clearForAggPushDown() - return expression.NewSchema(nil) + return expression.NewSchema() } p.GbyItemsPB = append(p.GbyItemsPB, pb) p.gbyItems = append(p.gbyItems, item.Clone()) @@ -290,7 +290,7 @@ func (p *physicalTableSource) addAggregation(ctx context.Context, agg *PhysicalA gk.Charset = charset.CharsetBin gk.Collate = charset.CollationBin p.AggFields = append(p.AggFields, gk) - schema := expression.NewSchema(nil) + schema := expression.NewSchema() cursor := 0 schema.Append(&expression.Column{Index: cursor, ColName: model.NewCIStr(fmt.Sprint(agg.GroupByItems))}) agg.GroupByItems = []expression.Expression{schema.Columns[cursor]} diff --git a/plan/plan.go b/plan/plan.go index b804919a7def1..06edb269c759e 100644 --- a/plan/plan.go +++ b/plan/plan.go @@ -91,9 +91,9 @@ type Plan interface { // Get all the children. GetChildren() []Plan // Set the schema. - SetSchema(schema expression.Schema) + SetSchema(schema *expression.Schema) // Get the schema. - GetSchema() expression.Schema + GetSchema() *expression.Schema // Get the ID. GetID() string // Check whether this plan is correlated or not. @@ -320,7 +320,7 @@ type basePlan struct { parents []Plan children []Plan - schema expression.Schema + schema *expression.Schema tp string id string allocator *idAllocator @@ -360,12 +360,12 @@ func (p *basePlan) GetID() string { } // SetSchema implements Plan SetSchema interface. -func (p *basePlan) SetSchema(schema expression.Schema) { +func (p *basePlan) SetSchema(schema *expression.Schema) { p.schema = schema } // GetSchema implements Plan GetSchema interface. -func (p *basePlan) GetSchema() expression.Schema { +func (p *basePlan) GetSchema() *expression.Schema { return p.schema } diff --git a/plan/planbuilder.go b/plan/planbuilder.go index ff289ae53dc17..9617c683a2469 100644 --- a/plan/planbuilder.go +++ b/plan/planbuilder.go @@ -65,7 +65,7 @@ type planBuilder struct { allocator *idAllocator ctx context.Context is infoschema.InfoSchema - outerSchemas []expression.Schema + outerSchemas []*expression.Schema inUpdateStmt bool // colMapper stores the column that must be pre-resolved. colMapper map[*ast.ColumnNameExpr]int @@ -138,6 +138,7 @@ func (b *planBuilder) buildExecute(v *ast.ExecuteStmt) Plan { vars = append(vars, newExpr) } exe := &Execute{Name: v.Name, UsingVars: vars} + exe.SetSchema(expression.NewSchema()) return exe } @@ -155,6 +156,7 @@ func (b *planBuilder) buildDo(v *ast.DoStmt) Plan { } exprs = append(exprs, expr) } + dual.SetSchema(expression.NewSchema()) p := &Projection{ Exprs: exprs, baseLogicalPlan: newBaseLogicalPlan(Proj, b.allocator), @@ -162,6 +164,7 @@ func (b *planBuilder) buildDo(v *ast.DoStmt) Plan { p.initIDAndContext(b.ctx) addChild(p, dual) p.self = p + p.SetSchema(expression.NewSchema()) return p } @@ -192,6 +195,7 @@ func (b *planBuilder) buildSet(v *ast.SetStmt) Plan { p.VarAssigns = append(p.VarAssigns, assign) } p.initIDAndContext(b.ctx) + p.SetSchema(expression.NewSchema()) return p } @@ -314,6 +318,7 @@ func (b *planBuilder) buildPrepare(x *ast.PrepareStmt) Plan { } else { p.SQLText = x.SQLText } + p.SetSchema(expression.NewSchema()) return p } @@ -323,18 +328,18 @@ func (b *planBuilder) buildAdmin(as *ast.AdminStmt) Plan { switch as.Tp { case ast.AdminCheckTable: p = &CheckTable{Tables: as.Tables} + p.SetSchema(expression.NewSchema()) case ast.AdminShowDDL: p = &ShowDDL{} p.SetSchema(buildShowDDLFields()) default: b.err = ErrUnsupportedType.Gen("Unsupported type %T", as) } - return p } -func buildShowDDLFields() expression.Schema { - schema := expression.NewSchema(make([]*expression.Column, 0, 6)) +func buildShowDDLFields() *expression.Schema { + schema := expression.NewSchema(make([]*expression.Column, 0, 6)...) schema.Append(buildColumn("", "SCHEMA_VER", mysql.TypeLonglong, 4)) schema.Append(buildColumn("", "OWNER", mysql.TypeVarchar, 64)) schema.Append(buildColumn("", "JOB", mysql.TypeVarchar, 128)) @@ -445,13 +450,16 @@ func (b *planBuilder) buildShow(show *ast.ShowStmt) Plan { sel.initIDAndContext(b.ctx) sel.self = sel addChild(sel, p) + sel.SetSchema(p.GetSchema()) resultPlan = sel } return resultPlan } func (b *planBuilder) buildSimple(node ast.StmtNode) Plan { - return &Simple{Statement: node} + p := &Simple{Statement: node} + p.SetSchema(expression.NewSchema()) + return p } func (b *planBuilder) getDefaultValue(col *table.Column) (*expression.Constant, error) { @@ -474,7 +482,6 @@ func (b *planBuilder) findDefaultValue(cols []*table.Column, name *ast.ColumnNam } func (b *planBuilder) buildInsert(insert *ast.InsertStmt) Plan { - // Get Table ts, ok := insert.Table.TableRefs.Left.(*ast.TableSource) if !ok { b.err = infoschema.ErrTableNotExists.GenByArgs() @@ -581,6 +588,7 @@ func (b *planBuilder) buildInsert(insert *ast.InsertStmt) Plan { } addChild(insertPlan, selectPlan) } + insertPlan.SetSchema(expression.NewSchema()) return insertPlan } @@ -592,11 +600,14 @@ func (b *planBuilder) buildLoadData(ld *ast.LoadDataStmt) Plan { FieldsInfo: ld.FieldsInfo, LinesInfo: ld.LinesInfo, } + p.SetSchema(expression.NewSchema()) return p } func (b *planBuilder) buildDDL(node ast.DDLNode) Plan { - return &DDL{Statement: node} + p := &DDL{Statement: node} + p.SetSchema(expression.NewSchema()) + return p } func (b *planBuilder) buildExplain(explain *ast.ExplainStmt) Plan { @@ -610,7 +621,7 @@ func (b *planBuilder) buildExplain(explain *ast.ExplainStmt) Plan { } p := &Explain{StmtPlan: targetPlan} addChild(p, targetPlan) - schema := expression.NewSchema(make([]*expression.Column, 0, 3)) + schema := expression.NewSchema(make([]*expression.Column, 0, 3)...) schema.Append(&expression.Column{ ColName: model.NewCIStr("ID"), RetType: types.NewFieldType(mysql.TypeString), @@ -627,9 +638,9 @@ func (b *planBuilder) buildExplain(explain *ast.ExplainStmt) Plan { return p } -func buildShowProcedureSchema() expression.Schema { +func buildShowProcedureSchema() *expression.Schema { tblName := "ROUTINES" - schema := expression.NewSchema(make([]*expression.Column, 0, 11)) + schema := expression.NewSchema(make([]*expression.Column, 0, 11)...) schema.Append(buildColumn(tblName, "Db", mysql.TypeVarchar, 128)) schema.Append(buildColumn(tblName, "Name", mysql.TypeVarchar, 128)) schema.Append(buildColumn(tblName, "Type", mysql.TypeVarchar, 128)) @@ -644,9 +655,9 @@ func buildShowProcedureSchema() expression.Schema { return schema } -func buildShowTriggerSchema() expression.Schema { +func buildShowTriggerSchema() *expression.Schema { tblName := "TRIGGERS" - schema := expression.NewSchema(make([]*expression.Column, 0, 11)) + schema := expression.NewSchema(make([]*expression.Column, 0, 11)...) schema.Append(buildColumn(tblName, "Trigger", mysql.TypeVarchar, 128)) schema.Append(buildColumn(tblName, "Event", mysql.TypeVarchar, 128)) schema.Append(buildColumn(tblName, "Table", mysql.TypeVarchar, 128)) @@ -661,9 +672,9 @@ func buildShowTriggerSchema() expression.Schema { return schema } -func buildShowEventsSchema() expression.Schema { +func buildShowEventsSchema() *expression.Schema { tblName := "EVENTS" - schema := expression.NewSchema(make([]*expression.Column, 0, 15)) + schema := expression.NewSchema(make([]*expression.Column, 0, 15)...) schema.Append(buildColumn(tblName, "Db", mysql.TypeVarchar, 128)) schema.Append(buildColumn(tblName, "Name", mysql.TypeVarchar, 128)) schema.Append(buildColumn(tblName, "Time zone", mysql.TypeVarchar, 32)) @@ -738,9 +749,9 @@ func getShowColNamesAndTypes(s *ast.ShowStmt) (names []string, ftypes []byte) { return } -func buildShowDefaultSchema(s *ast.ShowStmt) expression.Schema { +func buildShowDefaultSchema(s *ast.ShowStmt) *expression.Schema { names, ftypes := getShowColNamesAndTypes(s) - schema := expression.NewSchema(make([]*expression.Column, 0, len(names))) + schema := expression.NewSchema(make([]*expression.Column, 0, len(names))...) for i, name := range names { col := &expression.Column{ ColName: model.NewCIStr(name), diff --git a/plan/plans.go b/plan/plans.go index ecfdf21067f3a..15469b53c043d 100644 --- a/plan/plans.go +++ b/plan/plans.go @@ -173,7 +173,7 @@ type Insert struct { baseLogicalPlan Table table.Table - tableSchema expression.Schema + tableSchema *expression.Schema Columns []*ast.ColumnName Lists [][]expression.Expression Setlist []*expression.Assignment diff --git a/plan/predicate_push_down.go b/plan/predicate_push_down.go index 91c17521bfeca..9aff05f95b908 100644 --- a/plan/predicate_push_down.go +++ b/plan/predicate_push_down.go @@ -201,7 +201,7 @@ func outerJoinSimplify(p *Join, predicates []expression.Expression) error { // If it is a predicate containing a reference to an inner table that evaluates to UNKNOWN or FALSE when one of its arguments is NULL. // If it is a conjunction containing a null-rejected condition as a conjunct. // If it is a disjunction of null-rejected conditions. -func isNullRejected(ctx context.Context, schema expression.Schema, expr expression.Expression) (bool, error) { +func isNullRejected(ctx context.Context, schema *expression.Schema, expr expression.Expression) (bool, error) { result, err := expression.EvaluateExprWithNull(ctx, schema, expr) if err != nil { return false, errors.Trace(err) @@ -289,7 +289,7 @@ func (p *Union) PredicatePushDown(predicates []expression.Expression) (ret []exp // getGbyColIndex gets the column's index in the group-by columns. func (p *Aggregation) getGbyColIndex(col *expression.Column) int { - return expression.NewSchema(p.groupByCols).GetColumnIndex(col) + return expression.NewSchema(p.groupByCols...).GetColumnIndex(col) } // PredicatePushDown implements LogicalPlan PredicatePushDown interface.