Skip to content

Commit

Permalink
*: change schema from []*Column to struct (pingcap#2321)
Browse files Browse the repository at this point in the history
  • Loading branch information
winoros authored and hanfei1991 committed Dec 27, 2016
1 parent 02ad847 commit 5f96f67
Show file tree
Hide file tree
Showing 32 changed files with 311 additions and 280 deletions.
4 changes: 2 additions & 2 deletions executor/adapter.go
Original file line number Diff line number Diff line change
Expand Up @@ -33,7 +33,7 @@ type recordSet struct {

func (a *recordSet) Fields() ([]*ast.ResultField, error) {
if len(a.fields) == 0 {
for _, col := range a.schema {
for _, col := range a.schema.Columns {
rf := &ast.ResultField{
ColumnAsName: col.ColName,
TableAsName: col.TblName,
Expand Down Expand Up @@ -100,7 +100,7 @@ func (a *statement) Exec(ctx context.Context) (ast.RecordSet, error) {
}

// Fields or Schema are only used for statements that return result set.
if len(e.Schema()) == 0 {
if e.Schema().Len() == 0 {
// Check if "tidb_snapshot" is set for the write executors.
// In history read mode, we can not do write operations.
switch e.(type) {
Expand Down
2 changes: 1 addition & 1 deletion executor/aggregate_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -34,7 +34,7 @@ type MockExec struct {
}

func (m *MockExec) Schema() expression.Schema {
return nil
return expression.Schema{}
}

func (m *MockExec) Fields() []*ast.ResultField {
Expand Down
6 changes: 3 additions & 3 deletions executor/builder.go
Original file line number Diff line number Diff line change
Expand Up @@ -337,7 +337,7 @@ func (b *executorBuilder) buildUnionScanExec(v *plan.PhysicalUnionScan) Executor
case *XSelectIndexExec:
us.desc = x.indexPlan.Desc
for _, ic := range x.indexPlan.Index.Columns {
for i, col := range x.indexPlan.GetSchema() {
for i, col := range x.indexPlan.GetSchema().Columns {
if col.ColName.L == ic.Name.L {
us.usedIndex = append(us.usedIndex, i)
break
Expand Down Expand Up @@ -598,7 +598,7 @@ func (b *executorBuilder) buildApply(v *plan.PhysicalApply) Executor {
apply.checker = &conditionChecker{
all: v.Checker.All,
cond: v.Checker.Condition,
trimLen: len(src.Schema()),
trimLen: src.Schema().Len(),
ctx: b.ctx,
}
}
Expand All @@ -623,7 +623,7 @@ func (b *executorBuilder) buildTrim(v *plan.Trim) Executor {
return &TrimExec{
schema: v.GetSchema(),
Src: b.build(v.GetChildByIndex(0)),
len: len(v.GetSchema()),
len: v.GetSchema().Len(),
}
}

Expand Down
12 changes: 6 additions & 6 deletions executor/executor.go
Original file line number Diff line number Diff line change
Expand Up @@ -180,7 +180,7 @@ type CheckTableExec struct {

// Schema implements the Executor Schema interface.
func (e *CheckTableExec) Schema() expression.Schema {
return nil
return expression.NewSchema(nil)
}

// Next implements the Executor Next interface.
Expand Down Expand Up @@ -744,7 +744,7 @@ func (e *HashJoinExec) constructMatchedRows(ctx *hashJoinCtx, bigRow *Row) (matc
// It is used for outer join, when a row from outer table doesn't have any matching rows.
func (e *HashJoinExec) fillRowWithDefaultValues(bigRow *Row) (returnRow *Row) {
smallRow := &Row{
Data: make([]types.Datum, len(e.smallExec.Schema())),
Data: make([]types.Datum, e.smallExec.Schema().Len()),
}
copy(smallRow.Data, e.defaultValues)
if e.leftSmall {
Expand Down Expand Up @@ -1382,7 +1382,7 @@ func (e *TableScanExec) Next() (*Row, error) {

func (e *TableScanExec) nextForInfoSchema() (*Row, error) {
if e.infoSchemaRows == nil {
columns := make([]*table.Column, len(e.schema))
columns := make([]*table.Column, e.schema.Len())
for i, v := range e.columns {
columns[i] = table.ToColumn(v)
}
Expand Down Expand Up @@ -1425,7 +1425,7 @@ func (e *TableScanExec) getRow(handle int64) (*Row, error) {
row := &Row{}
var err error

columns := make([]*table.Column, len(e.schema))
columns := make([]*table.Column, e.schema.Len())
for i, v := range e.columns {
columns[i] = table.ToColumn(v)
}
Expand Down Expand Up @@ -1835,7 +1835,7 @@ func (e *MaxOneRowExec) Next() (*Row, error) {
return nil, errors.Trace(err)
}
if srcRow == nil {
return &Row{Data: make([]types.Datum, len(e.schema))}, nil
return &Row{Data: make([]types.Datum, e.schema.Len())}, nil
}
srcRow1, err := e.Src.Next()
if err != nil {
Expand Down Expand Up @@ -1933,7 +1933,7 @@ func (e *UnionExec) fetchData(idx int) {
if idx != 0 {
// TODO: Add cast function in plan building phase.
for j := range row.Data {
col := e.schema[j]
col := e.schema.Columns[j]
val, err := row.Data[j].ConvertTo(e.ctx.GetSessionVars().StmtCtx, col.RetType)
if err != nil {
e.finished.Store(true)
Expand Down
2 changes: 1 addition & 1 deletion executor/executor_ddl.go
Original file line number Diff line number Diff line change
Expand Up @@ -42,7 +42,7 @@ type DDLExec struct {

// Schema implements the Executor Schema interface.
func (e *DDLExec) Schema() expression.Schema {
return nil
return expression.NewSchema(nil)
}

// Next implements Execution Next interface.
Expand Down
2 changes: 1 addition & 1 deletion executor/executor_set.go
Original file line number Diff line number Diff line change
Expand Up @@ -136,7 +136,7 @@ func (e *SetExecutor) executeSet() error {

// Schema implements the Executor Schema interface.
func (e *SetExecutor) Schema() expression.Schema {
return nil
return expression.NewSchema(nil)
}

// Close implements the Executor Close interface.
Expand Down
2 changes: 1 addition & 1 deletion executor/executor_simple.go
Original file line number Diff line number Diff line change
Expand Up @@ -49,7 +49,7 @@ type SimpleExec struct {

// Schema implements the Executor Schema interface.
func (e *SimpleExec) Schema() expression.Schema {
return nil
return expression.NewSchema(nil)
}

// Next implements Execution Next interface.
Expand Down
24 changes: 12 additions & 12 deletions executor/executor_write.go
Original file line number Diff line number Diff line change
Expand Up @@ -155,7 +155,7 @@ type DeleteExec struct {

// Schema implements the Executor Schema interface.
func (e *DeleteExec) Schema() expression.Schema {
return nil
return expression.NewSchema(nil)
}

// Next implements the Executor Next interface.
Expand Down Expand Up @@ -530,7 +530,7 @@ func (e *LoadData) Next() (*Row, error) {

// Schema implements the Executor Schema interface.
func (e *LoadData) Schema() expression.Schema {
return nil
return expression.NewSchema(nil)
}

// Close implements the Executor Close interface.
Expand Down Expand Up @@ -566,7 +566,7 @@ type InsertExec struct {

// Schema implements the Executor Schema interface.
func (e *InsertExec) Schema() expression.Schema {
return nil
return expression.NewSchema(nil)
}

// Next implements the Executor Next interface.
Expand Down Expand Up @@ -759,8 +759,8 @@ func (e *InsertValues) getRow(cols []*table.Column, list []expression.Expression

func (e *InsertValues) getRowsSelect(cols []*table.Column) ([][]types.Datum, error) {
// process `insert|replace into ... select ... from ...`
if len(e.SelectExec.Schema()) != len(cols) {
return nil, errors.Errorf("Column count %d doesn't match value count %d", len(cols), len(e.SelectExec.Schema()))
if e.SelectExec.Schema().Len() != len(cols) {
return nil, errors.Errorf("Column count %d doesn't match value count %d", len(cols), e.SelectExec.Schema().Len())
}
var rows [][]types.Datum
for {
Expand Down Expand Up @@ -954,7 +954,7 @@ type ReplaceExec struct {

// Schema implements the Executor Schema interface.
func (e *ReplaceExec) Schema() expression.Schema {
return nil
return expression.NewSchema(nil)
}

// Close implements the Executor Close interface.
Expand Down Expand Up @@ -1061,7 +1061,7 @@ type UpdateExec struct {

// Schema implements the Executor Schema interface.
func (e *UpdateExec) Schema() expression.Schema {
return nil
return expression.NewSchema(nil)
}

// Next implements the Executor Next interface.
Expand Down Expand Up @@ -1132,9 +1132,9 @@ func (e *UpdateExec) fetchRows() error {
if row == nil {
return nil
}
data := make([]types.Datum, len(e.SelectExec.Schema()))
newData := make([]types.Datum, len(e.SelectExec.Schema()))
for i, s := range e.SelectExec.Schema() {
data := make([]types.Datum, e.SelectExec.Schema().Len())
newData := make([]types.Datum, e.SelectExec.Schema().Len())
for i, s := range e.SelectExec.Schema().Columns {
data[i], err = s.Eval(row.Data, e.ctx)
if err != nil {
return errors.Trace(err)
Expand Down Expand Up @@ -1163,8 +1163,8 @@ func (e *UpdateExec) getTableOffset(entry RowKeyEntry) int {
tblName = entry.TableAsName.L
}
schema := e.SelectExec.Schema()
for i := 0; i < len(schema); i++ {
s := schema[i]
for i := 0; i < schema.Len(); i++ {
s := schema.Columns[i]
if s.TblName.L == tblName {
return i
}
Expand Down
2 changes: 1 addition & 1 deletion executor/grant.go
Original file line number Diff line number Diff line change
Expand Up @@ -51,7 +51,7 @@ type GrantExec struct {

// Schema implements the Executor Schema interface.
func (e *GrantExec) Schema() expression.Schema {
return nil
return expression.NewSchema(nil)
}

// Next implements Execution Next interface.
Expand Down
6 changes: 3 additions & 3 deletions executor/prepared.go
Original file line number Diff line number Diff line change
Expand Up @@ -85,7 +85,7 @@ type PrepareExec struct {
// Schema implements the Executor Schema interface.
func (e *PrepareExec) Schema() expression.Schema {
// Will never be called.
return nil
return expression.NewSchema(nil)
}

// Next implements the Executor Next interface.
Expand Down Expand Up @@ -180,7 +180,7 @@ type ExecuteExec struct {
// Schema implements the Executor Schema interface.
func (e *ExecuteExec) Schema() expression.Schema {
// Will never be called.
return nil
return expression.NewSchema(nil)
}

// Next implements the Executor Next interface.
Expand Down Expand Up @@ -253,7 +253,7 @@ type DeallocateExec struct {
// Schema implements the Executor Schema interface.
func (e *DeallocateExec) Schema() expression.Schema {
// Will never be called.
return nil
return expression.NewSchema(nil)
}

// Next implements the Executor Next interface.
Expand Down
4 changes: 2 additions & 2 deletions executor/union_scan.go
Original file line number Diff line number Diff line change
Expand Up @@ -235,10 +235,10 @@ func (us *UnionScanExec) buildAndSortAddedRows(t table.Table, asName *model.CISt
us.addedRows = make([]*Row, 0, len(us.dirty.addedRows))
for h, data := range us.dirty.addedRows {
var newData []types.Datum
if len(us.Src.Schema()) == len(data) {
if us.Src.Schema().Len() == len(data) {
newData = data
} else {
newData = make([]types.Datum, 0, len(us.Src.Schema()))
newData = make([]types.Datum, 0, us.Src.Schema().Len())
var columns []*model.ColumnInfo
if t, ok := us.Src.(*XSelectTableExec); ok {
columns = t.Columns
Expand Down
13 changes: 11 additions & 2 deletions expression/column.go
Original file line number Diff line number Diff line change
Expand Up @@ -56,7 +56,7 @@ func (col *CorrelatedColumn) IsCorrelated() bool {

// Decorrelate implements Expression interface.
func (col *CorrelatedColumn) Decorrelate(schema Schema) Expression {
if schema.GetIndex(&col.Column) == -1 {
if schema.GetColumnIndex(&col.Column) == -1 {
return col
}
return &col.Column
Expand Down Expand Up @@ -151,9 +151,18 @@ func (col *Column) HashCode() []byte {

// ResolveIndices implements Expression interface.
func (col *Column) ResolveIndices(schema Schema) {
col.Index = schema.GetIndex(col)
col.Index = schema.GetColumnIndex(col)
// If col's index equals to -1, it means a internal logic error happens.
if col.Index == -1 {
log.Errorf("Can't find column %s in schema %s", col, schema)
}
}

// Column2Exprs will transfer column slice to expression slice.
func Column2Exprs(cols []*Column) []Expression {
result := make([]Expression, 0, len(cols))
for _, col := range cols {
result = append(result, col.Clone())
}
return result
}
4 changes: 2 additions & 2 deletions expression/constant_propagation.go
Original file line number Diff line number Diff line change
Expand Up @@ -121,15 +121,15 @@ func (s *propagateConstantSolver) propagateEQ() {
if mapper == nil || len(mapper) == 0 {
return
}
cols := make(Schema, 0, len(mapper))
cols := make([]*Column, 0, len(mapper))
cons := make([]Expression, 0, len(mapper))
for id, con := range mapper {
cols = append(cols, s.columns[id])
cons = append(cons, con)
}
for i, cond := range s.conditions {
if !visited[i] {
s.conditions[i] = ColumnSubstitute(cond, Schema(cols), cons)
s.conditions[i] = ColumnSubstitute(cond, NewSchema(cols), cons)
}
}
}
Expand Down
6 changes: 3 additions & 3 deletions expression/expression.go
Original file line number Diff line number Diff line change
Expand Up @@ -243,7 +243,7 @@ func EvaluateExprWithNull(ctx context.Context, schema Schema, expr Expression) (
}
return FoldConstant(ctx, newFunc), nil
case *Column:
if schema.GetIndex(x) == -1 {
if schema.GetColumnIndex(x) == -1 {
return x, nil
}
constant := &Constant{Value: types.Datum{}}
Expand All @@ -255,15 +255,15 @@ func EvaluateExprWithNull(ctx context.Context, schema Schema, expr Expression) (

// TableInfo2Schema converts table info to schema.
func TableInfo2Schema(tbl *model.TableInfo) Schema {
schema := make(Schema, 0, len(tbl.Columns))
schema := NewSchema(make([]*Column, 0, len(tbl.Columns)))
for i, col := range tbl.Columns {
newCol := &Column{
ColName: col.Name,
TblName: tbl.Name,
RetType: &col.FieldType,
Position: i,
}
schema = append(schema, newCol)
schema.Append(newCol)
}
return schema
}
Loading

0 comments on commit 5f96f67

Please sign in to comment.