Skip to content

Commit

Permalink
plan: Clean up (#3606)
Browse files Browse the repository at this point in the history
  • Loading branch information
zimulala authored and winoros committed Jul 3, 2017
1 parent 3f99d7e commit 4023bb8
Show file tree
Hide file tree
Showing 5 changed files with 38 additions and 33 deletions.
26 changes: 12 additions & 14 deletions plan/logical_plan_builder.go
Original file line number Diff line number Diff line change
Expand Up @@ -25,6 +25,7 @@ import (
"github.com/pingcap/tidb/sessionctx"
"github.com/pingcap/tidb/sessionctx/variable"
"github.com/pingcap/tidb/statistics"
"github.com/pingcap/tidb/table"
"github.com/pingcap/tidb/util/types"
)

Expand Down Expand Up @@ -952,8 +953,8 @@ func (b *planBuilder) pushTableHints(hints []*ast.TableOptimizerHint) bool {
}
if len(sortMergeTables) != 0 || len(INLJTables) != 0 {
b.tableHintInfo = append(b.tableHintInfo, tableHintInfo{
sortMergeJoinTables: sortMergeTables,
INLJTables: INLJTables,
sortMergeJoinTables: sortMergeTables,
indexNestedLoopJoinTables: INLJTables,
})
return true
}
Expand Down Expand Up @@ -1107,23 +1108,20 @@ func (b *planBuilder) buildDataSource(tn *ast.TableName) LogicalPlan {
tableInfo: tableInfo,
statisticTable: statisticTable,
DBName: schemaName,
Columns: make([]*model.ColumnInfo, 0, len(tableInfo.Columns)),
}.init(b.allocator, b.ctx)

b.visitInfo = appendVisitInfo(b.visitInfo, mysql.SelectPriv, schemaName.L, tableInfo.Name.L, "")

// Equal condition contains a column from previous joined table.
schema := expression.NewSchema(make([]*expression.Column, 0, len(tableInfo.Columns))...)
for i, col := range tableInfo.Columns {
if b.inUpdateStmt {
switch col.State {
case model.StatePublic, model.StateWriteOnly, model.StateWriteReorganization:
default:
continue
}
} else if col.State != model.StatePublic {
continue
}
p.Columns = append(p.Columns, col)
var columns []*table.Column
if b.inUpdateStmt {
columns = tbl.WritableCols()
} else {
columns = tbl.Cols()
}
for i, col := range columns {
p.Columns = append(p.Columns, col.ColumnInfo)
schema.Append(&expression.Column{
FromID: p.id,
ColName: col.Name,
Expand Down
28 changes: 14 additions & 14 deletions plan/new_physical_plan_builder.go
Original file line number Diff line number Diff line change
Expand Up @@ -109,17 +109,15 @@ func (p *LogicalJoin) convertToIndexJoin(prop *requiredProp, outerIdx int) (task
outerChild := p.children[outerIdx].(LogicalPlan)
innerChild := p.children[1-outerIdx].(LogicalPlan)
var (
outerTask task
useTableScan bool
usedIndexInfo *model.IndexInfo
rightConds expression.CNFExprs
leftConds expression.CNFExprs
innerTask task
err error
innerJoinKeys []*expression.Column
outerJoinKeys []*expression.Column
)
outerTask, err = outerChild.convert2NewPhysicalPlan(&requiredProp{taskTp: rootTaskType})
outerTask, err := outerChild.convert2NewPhysicalPlan(&requiredProp{taskTp: rootTaskType})
if err != nil {
return nil, errors.Trace(err)
}
Expand Down Expand Up @@ -152,18 +150,20 @@ func (p *LogicalJoin) convertToIndexJoin(prop *requiredProp, outerIdx int) (task
break
}
for _, indexInfo := range indices {
if matchedOffsets := joinKeysMatchIndex(innerJoinKeys, indexInfo); matchedOffsets != nil {
usedIndexInfo = indexInfo
newOuterJoinKeys := make([]*expression.Column, len(outerJoinKeys))
newInnerJoinKeys := make([]*expression.Column, len(innerJoinKeys))
for i, offset := range matchedOffsets {
newOuterJoinKeys[i] = outerJoinKeys[offset]
newInnerJoinKeys[i] = innerJoinKeys[offset]
}
outerJoinKeys = newOuterJoinKeys
innerJoinKeys = newInnerJoinKeys
break
matchedOffsets := joinKeysMatchIndex(innerJoinKeys, indexInfo)
if matchedOffsets == nil {
continue
}
usedIndexInfo = indexInfo
newOuterJoinKeys := make([]*expression.Column, len(outerJoinKeys))
newInnerJoinKeys := make([]*expression.Column, len(innerJoinKeys))
for i, offset := range matchedOffsets {
newOuterJoinKeys[i] = outerJoinKeys[offset]
newInnerJoinKeys[i] = innerJoinKeys[offset]
}
outerJoinKeys = newOuterJoinKeys
innerJoinKeys = newInnerJoinKeys
break
}
if usedIndexInfo != nil {
innerTask, err = x.convertToIndexScan(&requiredProp{taskTp: rootTaskType}, usedIndexInfo)
Expand Down
2 changes: 1 addition & 1 deletion plan/optimizer.go
Original file line number Diff line number Diff line change
Expand Up @@ -122,7 +122,6 @@ func doOptimize(flag uint64, logic LogicalPlan, ctx context.Context, allocator *
if UseDAGPlanBuilder(ctx) {
return dagPhysicalOptimize(logic)
}
logic.ResolveIndices()
return physicalOptimize(flag, logic, allocator)
}

Expand Down Expand Up @@ -155,6 +154,7 @@ func dagPhysicalOptimize(logic LogicalPlan) (PhysicalPlan, error) {
}

func physicalOptimize(flag uint64, logic LogicalPlan, allocator *idAllocator) (PhysicalPlan, error) {
logic.ResolveIndices()
info, err := logic.convert2PhysicalPlan(&requiredProperty{})
if err != nil {
return nil, errors.Trace(err)
Expand Down
6 changes: 3 additions & 3 deletions plan/planbuilder.go
Original file line number Diff line number Diff line change
Expand Up @@ -71,8 +71,8 @@ type visitInfo struct {
}

type tableHintInfo struct {
INLJTables []model.CIStr
sortMergeJoinTables []model.CIStr
indexNestedLoopJoinTables []model.CIStr
sortMergeJoinTables []model.CIStr
}

func (info *tableHintInfo) ifPreferMergeJoin(tableNames ...*model.CIStr) bool {
Expand Down Expand Up @@ -101,7 +101,7 @@ func (info *tableHintInfo) ifPreferINLJ(tableNames ...*model.CIStr) bool {
if tableName == nil {
continue
}
for _, curEntry := range info.INLJTables {
for _, curEntry := range info.indexNestedLoopJoinTables {
if curEntry.L == tableName.L {
return true
}
Expand Down
9 changes: 8 additions & 1 deletion table/tables/tables.go
Original file line number Diff line number Diff line change
Expand Up @@ -56,7 +56,14 @@ type Table struct {

// MockTableFromMeta only serves for test.
func MockTableFromMeta(tableInfo *model.TableInfo) table.Table {
return &Table{ID: 0, meta: tableInfo}
columns := make([]*table.Column, 0, len(tableInfo.Columns))
for _, colInfo := range tableInfo.Columns {
col := table.ToColumn(colInfo)
columns = append(columns, col)
}
t := newTable(tableInfo.ID, columns, nil)
t.meta = tableInfo
return t
}

// TableFromMeta creates a Table instance from model.TableInfo.
Expand Down

0 comments on commit 4023bb8

Please sign in to comment.