From 5db58a91a4215e17ad4d2b005d96be70f259f6c3 Mon Sep 17 00:00:00 2001 From: Yiding Cui Date: Tue, 14 Mar 2017 20:08:10 -0700 Subject: [PATCH 01/27] *: add indexScan controlled by correlated column --- executor/builder.go | 6 + executor/executor.go | 91 +++++++++++++++ plan/expr_to_pb.go | 2 +- plan/logical_plans.go | 7 ++ plan/physical_plan_builder.go | 205 +++++++++++++++++++++++++++++++++- plan/refiner.go | 24 ++-- 6 files changed, 318 insertions(+), 17 deletions(-) diff --git a/executor/builder.go b/executor/builder.go index 7916b439a20d8..fa1fdf1fbb753 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -27,6 +27,7 @@ import ( "github.com/pingcap/tidb/model" "github.com/pingcap/tidb/plan" "github.com/pingcap/tidb/util/types" + "github.com/ngaut/log" ) // executorBuilder builds an Executor from a Plan. @@ -477,7 +478,12 @@ func (b *executorBuilder) buildSelection(v *plan.Selection) Executor { Condition: expression.ComposeCNFCondition(b.ctx, v.Conditions...), schema: v.Schema(), ctx: b.ctx, + scanController: v.ScanController, + accessConditions: v.AccessConditions, + idxFilterConditions: v.IdxConditions, + tblFilterConditions: v.TblConditions, } + log.Warnf("child type: %T", exec.Src) return exec } diff --git a/executor/executor.go b/executor/executor.go index dee616e4347a3..4eb5a398912a4 100644 --- a/executor/executor.go +++ b/executor/executor.go @@ -31,6 +31,7 @@ import ( "github.com/pingcap/tidb/tablecodec" "github.com/pingcap/tidb/terror" "github.com/pingcap/tidb/util/types" + "github.com/ngaut/log" ) var ( @@ -462,12 +463,42 @@ func (e *TableDualExec) Close() error { return nil } +func convertCorCol2Constant(expr expression.Expression) (expression.Expression, error){ + switch x := expr.(type) { + case *expression.ScalarFunction: + newArgs := make([]expression.Expression, 0, len(x.GetArgs())) + for _, arg := range x.GetArgs() { + newArg, err := convertCorCol2Constant(arg) + if err != nil { + return nil, errors.Trace(err) + } + newArgs = append(newArgs, newArg) + } + newSf, _ := expression.NewFunction(x.GetCtx(), x.FuncName.L, x.GetType(), newArgs...) + return newSf, nil + case *expression.CorrelatedColumn: + val, err := x.Eval(nil) + if err != nil { + return nil, errors.Trace(err) + } + return &expression.Constant{Value: val, RetType: x.GetType()}, nil + default: + return x.Clone(), nil + } +} + // SelectionExec represents a filter executor. type SelectionExec struct { Src Executor Condition expression.Expression ctx context.Context schema *expression.Schema + + scanController bool + controllerInit bool + accessConditions []expression.Expression + idxFilterConditions []expression.Expression + tblFilterConditions []expression.Expression } // Schema implements the Executor Schema interface. @@ -475,8 +506,62 @@ func (e *SelectionExec) Schema() *expression.Schema { return e.schema } +func (e *SelectionExec) initController() error { + sc := e.ctx.GetSessionVars().StmtCtx + client := e.ctx.GetClient() + accesses := make([]expression.Expression, 0, len(e.accessConditions)) + log.Warnf("access Conditions: %v", e.accessConditions) + for _, cond := range e.accessConditions { + newCond, err := convertCorCol2Constant(cond) + if err != nil { + return errors.Trace(err) + } + accesses = append(accesses, newCond) + } + tblFilters := make([]expression.Expression, 0, len(e.tblFilterConditions)) + for _, cond := range e.tblFilterConditions { + newCond, err := convertCorCol2Constant(cond) + if err != nil { + return errors.Trace(err) + } + tblFilters = append(tblFilters, newCond) + } + switch x := e.Src.(type) { + case *XSelectTableExec: + log.Warnf("access Conditions: %v", accesses) + ranges, err := plan.BuildTableRange(accesses, sc) + if err != nil { + return errors.Trace(err) + } + x.ranges = ranges + var conds []expression.Expression + x.where, _, conds = plan.ExpressionsToPB(sc, tblFilters, client) + e.Condition = expression.ComposeCNFCondition(e.ctx, conds...) + return nil + case *XSelectIndexExec: + err := plan.BuildIndexRange(sc, x.indexPlan) + if err != nil { + return errors.Trace(err) + } + var idxConds, tblConds []expression.Expression + x.indexPlan.IndexConditionPBExpr, _, idxConds = plan.ExpressionsToPB(sc, e.idxFilterConditions, client) + x.indexPlan.TableConditionPBExpr, _, tblConds = plan.ExpressionsToPB(sc, e.tblFilterConditions, client) + e.Condition = expression.ComposeCNFCondition(e.ctx, append(idxConds, tblConds...)...) + return nil + default: + return errors.New("Error type of PhysicalPlan") + } +} + // Next implements the Executor Next interface. func (e *SelectionExec) Next() (*Row, error) { + if e.scanController && !e.controllerInit { + err := e.initController() + if err != nil { + return nil, errors.Trace(err) + } + e.controllerInit = true + } for { srcRow, err := e.Src.Next() if err != nil { @@ -485,6 +570,9 @@ func (e *SelectionExec) Next() (*Row, error) { if srcRow == nil { return nil, nil } + if e.Condition == nil { + return srcRow, nil + } match, err := expression.EvalBool(e.Condition, srcRow.Data, e.ctx) if err != nil { return nil, errors.Trace(err) @@ -497,6 +585,9 @@ func (e *SelectionExec) Next() (*Row, error) { // Close implements the Executor Close interface. func (e *SelectionExec) Close() error { + if e.scanController { + e.controllerInit = false + } return e.Src.Close() } diff --git a/plan/expr_to_pb.go b/plan/expr_to_pb.go index 454888424b6f5..a4bdb85c2dbc2 100644 --- a/plan/expr_to_pb.go +++ b/plan/expr_to_pb.go @@ -25,7 +25,7 @@ import ( "github.com/pingcap/tipb/go-tipb" ) -func expressionsToPB(sc *variable.StatementContext, exprs []expression.Expression, client kv.Client) (pbExpr *tipb.Expr, pushed []expression.Expression, remained []expression.Expression) { +func ExpressionsToPB(sc *variable.StatementContext, exprs []expression.Expression, client kv.Client) (pbExpr *tipb.Expr, pushed []expression.Expression, remained []expression.Expression) { pc := pbConverter{client: client, sc: sc} for _, expr := range exprs { v := pc.exprToPB(expr) diff --git a/plan/logical_plans.go b/plan/logical_plans.go index db5e79bb520df..145a6163dfa0c 100644 --- a/plan/logical_plans.go +++ b/plan/logical_plans.go @@ -146,6 +146,13 @@ type Selection struct { // onTable means if this selection's child is a table scan or index scan. onTable bool + + // If ScanController is true, then the child of this selection is a scan, + // which use pk or index + ScanController bool + AccessConditions []expression.Expression + IdxConditions []expression.Expression + TblConditions []expression.Expression } func (p *Selection) extractCorrelatedCols() []*expression.CorrelatedColumn { diff --git a/plan/physical_plan_builder.go b/plan/physical_plan_builder.go index 07939c1a334e0..00c9e41ab0e0b 100644 --- a/plan/physical_plan_builder.go +++ b/plan/physical_plan_builder.go @@ -71,11 +71,12 @@ func (p *DataSource) convert2TableScan(prop *requiredProperty) (*physicalPlanInf } ts.AccessCondition, newSel.Conditions = detachTableScanConditions(conds, table) ts.TableConditionPBExpr, ts.tableFilterConditions, newSel.Conditions = - expressionsToPB(sc, newSel.Conditions, client) - err := buildTableRange(ts) + ExpressionsToPB(sc, newSel.Conditions, client) + ranges, err := BuildTableRange(ts.AccessCondition, sc) if err != nil { return nil, errors.Trace(err) } + ts.Ranges = ranges if len(newSel.Conditions) > 0 { newSel.SetChildren(ts) newSel.onTable = true @@ -113,6 +114,7 @@ func (p *DataSource) convert2TableScan(prop *requiredProperty) (*physicalPlanInf } func (p *DataSource) convert2IndexScan(prop *requiredProperty, index *model.IndexInfo) (*physicalPlanInfo, error) { + log.Warnf("begin conver idxScan") client := p.ctx.GetClient() is := &PhysicalIndexScan{ Index: index, @@ -149,11 +151,11 @@ func (p *DataSource) convert2IndexScan(prop *requiredProperty, index *model.Inde isDistReq := !memDB && client != nil && client.SupportRequestType(kv.ReqTypeIndex, 0) if isDistReq { idxConds, tblConds := detachIndexFilterConditions(newSel.Conditions, is.Index.Columns, is.Table) - is.IndexConditionPBExpr, is.indexFilterConditions, idxConds = expressionsToPB(sc, idxConds, client) - is.TableConditionPBExpr, is.tableFilterConditions, tblConds = expressionsToPB(sc, tblConds, client) + is.IndexConditionPBExpr, is.indexFilterConditions, idxConds = ExpressionsToPB(sc, idxConds, client) + is.TableConditionPBExpr, is.tableFilterConditions, tblConds = ExpressionsToPB(sc, tblConds, client) newSel.Conditions = append(idxConds, tblConds...) } - err := buildIndexRange(p.ctx.GetSessionVars().StmtCtx, is) + err := BuildIndexRange(p.ctx.GetSessionVars().StmtCtx, is) if err != nil { if !terror.ErrorEqual(err, types.ErrTruncated) { return nil, errors.Trace(err) @@ -783,6 +785,191 @@ func (p *Union) convert2PhysicalPlan(prop *requiredProperty) (*physicalPlanInfo, return info, nil } +func buildTableScanByKeyAndCorCol(p *DataSource, pkName model.CIStr, fakeConds []expression.Expression, origConds []expression.Expression) (*physicalPlanInfo, []expression.Expression, []expression.Expression) { + client := p.ctx.GetClient() + ts := &PhysicalTableScan{ + Table: p.tableInfo, + Columns: p.Columns, + TableAsName: p.TableAsName, + DBName: p.DBName, + physicalTableSource: physicalTableSource{client: client}, + } + ts.tp = Tbl + ts.allocator = p.allocator + ts.SetSchema(p.Schema()) + ts.initIDAndContext(p.ctx) + if p.ctx.Txn() != nil { + ts.readOnly = p.ctx.Txn().IsReadOnly() + } else { + ts.readOnly = true + } + ts.Ranges = []TableRange{{math.MinInt64, math.MaxInt64}} + var accessConditions, filterConditions []expression.Expression + checker := conditionChecker{ + tableName: p.tableInfo.Name, + pkName: pkName, + length: types.UnspecifiedLength, + } + for id, cond := range fakeConds { + if !checker.check(cond) { + filterConditions = append(filterConditions, origConds[id]) + continue + } + accessConditions = append(accessConditions, origConds[id]) + // TODO: it will lead to repeated computation cost. + if checker.shouldReserve { + filterConditions= append(filterConditions, origConds[id]) + checker.shouldReserve = false + } + } + rowCount := uint64(p.statisticTable.Count) + ts.AccessCondition = accessConditions + ts.tableFilterConditions = filterConditions + return ts.matchProperty(&requiredProperty{}, &physicalPlanInfo{count: rowCount}), accessConditions, filterConditions +} + +func buildIndexScanByKeyAndCorCol(p *DataSource, idx *model.IndexInfo, fakeConds []expression.Expression, origConds []expression.Expression) (*physicalPlanInfo, + []expression.Expression, []expression.Expression, []expression.Expression) { + client := p.ctx.GetClient() + is := &PhysicalIndexScan{ + Index: idx, + Table: p.tableInfo, + Columns: p.Columns, + TableAsName: p.TableAsName, + OutOfOrder: true, + DBName: p.DBName, + physicalTableSource: physicalTableSource{client: p.ctx.GetClient()}, + } + is.tp = Idx + is.allocator = p.allocator + is.initIDAndContext(p.ctx) + is.SetSchema(p.schema) + if p.ctx.Txn() != nil { + is.readOnly = p.ctx.Txn().IsReadOnly() + } else { + is.readOnly = true + } + + fakeAccessConditions, restFakeConds := detachIndexScanConditions(fakeConds, is) + accessConditions := make([]expression.Expression, 0, len(fakeAccessConditions)) + restOrigConds := make([]expression.Expression, 0, len(restFakeConds)) + for i, j := 0, 0; i < len(fakeConds); { + if fakeConds[i].Equal(fakeAccessConditions[j], p.ctx) { + accessConditions = append(accessConditions, origConds[i]) + i, j = i+1, j+1 + } else { + restOrigConds = append(restOrigConds, origConds[i]) + i++ + } + } + memDB := infoschema.IsMemoryDB(p.DBName.L) + isDistReq := !memDB && client != nil && client.SupportRequestType(kv.ReqTypeIndex, 0) + var idxConds, tblConds []expression.Expression + if isDistReq { + fakeIdxConds, fakeTblConds := detachIndexFilterConditions(restFakeConds, is.Index.Columns, is.Table) + idxConds = make([]expression.Expression, 0, len(fakeIdxConds)) + tblConds = make([]expression.Expression, 0, len(fakeTblConds)) + for i, j := 0, 0; i < len(restFakeConds); { + if restFakeConds[i].Equal(fakeIdxConds[j], p.ctx) { + idxConds = append(idxConds, restOrigConds[i]) + i, j = i+1, j+1 + } else { + tblConds = append(tblConds, restOrigConds[i]) + i++ + } + } + } + + rb := rangeBuilder{sc: p.ctx.GetSessionVars().StmtCtx} + is.Ranges = rb.buildIndexRanges(fullRange, types.NewFieldType(mysql.TypeNull)) + is.DoubleRead = !isCoveringIndex(is.Columns, is.Index.Columns, is.Table.PKIsHandle) + is.AccessCondition = accessConditions + is.indexFilterConditions = idxConds + is.tableFilterConditions = tblConds + return is.matchProperty(&requiredProperty{}, &physicalPlanInfo{count: uint64(p.statisticTable.Count)}), accessConditions, idxConds, tblConds +} + +func convertCorCol2Constant(cond expression.Expression) expression.Expression { + switch x := cond.(type) { + case *expression.ScalarFunction: + newArgs := make([]expression.Expression, 0, len(x.GetArgs())) + for _, arg := range x.GetArgs() { + newArgs = append(newArgs, convertCorCol2Constant(arg)) + } + newSf, _ := expression.NewFunction(x.GetCtx(), x.FuncName.L, x.GetType(), newArgs...) + return newSf + case *expression.CorrelatedColumn: + return expression.One + default: + return x.Clone() + } +} + +// tryToBuildIndexScan will check the conditions contain correlated columns whether it can make indexScan. +func (p *Selection) tryToBuildScanByKey(prop *requiredProperty) *physicalPlanInfo { + if ds, ok := p.children[0].(*DataSource); ok && len(p.extractCorrelatedCols()) > 0 { + conds := make([]expression.Expression, 0, len(p.Conditions)) + for _, cond := range p.Conditions { + cond = pushDownNot(cond.Clone(), false, nil) + // In this way, we could use the code of refiner.go. + conds = append(conds, convertCorCol2Constant(cond)) + } + indices, _ := availableIndices(ds.indexHints, ds.tableInfo) + var usableIdxs []*model.IndexInfo + for _, idx := range indices { + // Currently we don't consider composite index. + if len(idx.Columns) > 1 { + continue + } + usableIdxs = append(usableIdxs, idx) + } + var pkName model.CIStr + if ds.tableInfo.PKIsHandle { + for _, col := range ds.Columns { + if mysql.HasPriKeyFlag(col.Flag) { + pkName = col.Name + } + } + } + // If pk is handle, we will try to build TableScan by pk, otherwise we will try to build IndexScan by index. + if pkName.L == "" { + var finalInfo *physicalPlanInfo + for _, idx := range usableIdxs { + info, accessConds, idxConds, tblConds := buildIndexScanByKeyAndCorCol(ds, idx, conds, p.Conditions) + if info == nil { + continue + } + p.ScanController = true + p.AccessConditions = accessConds + p.IdxConditions = idxConds + p.TblConditions = tblConds + if finalInfo == nil || finalInfo.cost > info.cost { + finalInfo = info + } + return info + } + if finalInfo == nil { + return nil + } + finalInfo = p.matchProperty(prop, finalInfo) + return finalInfo + } else { + info, accessCondition, filterCondition := buildTableScanByKeyAndCorCol(ds, pkName, conds, p.Conditions) + if info == nil { + return nil + } + p.ScanController = true + p.AccessConditions = accessCondition + p.TblConditions = filterCondition + info = p.matchProperty(prop, info) + return info + } + } else { + return nil + } + return nil +} + // convert2PhysicalPlan implements the LogicalPlan convert2PhysicalPlan interface. func (p *Selection) convert2PhysicalPlan(prop *requiredProperty) (*physicalPlanInfo, error) { info, err := p.getPlanInfo(prop) @@ -792,6 +979,11 @@ func (p *Selection) convert2PhysicalPlan(prop *requiredProperty) (*physicalPlanI if info != nil { return info, nil } + info = p.tryToBuildScanByKey(prop) + if info != nil { + p.storePlanInfo(prop, info) + return info, nil + } // Firstly, we try to push order. info, err = p.convert2PhysicalPlanPushOrder(prop) if err != nil { @@ -1121,7 +1313,8 @@ func addCachePlan(p PhysicalPlan, allocator *idAllocator) []*expression.Correlat newChildren := make([]Plan, 0, len(p.Children())) for _, child := range p.Children() { childCorCols := addCachePlan(child.(PhysicalPlan), allocator) - if len(selfCorCols) > 0 && len(childCorCols) == 0 { + // If p is a Selection and controls the access condition of below scan plan, there shouldn't have a cache plan. + if sel, ok := p.(*Selection); len(selfCorCols) > 0 && len(childCorCols) == 0 && (!ok || !sel.ScanController) { newChild := &Cache{} newChild.tp = "Cache" newChild.allocator = allocator diff --git a/plan/refiner.go b/plan/refiner.go index 92c2f27470dec..a7ed3b892e260 100644 --- a/plan/refiner.go +++ b/plan/refiner.go @@ -31,7 +31,8 @@ var fullRange = []rangePoint{ {value: types.MaxValueDatum()}, } -func buildIndexRange(sc *variable.StatementContext, p *PhysicalIndexScan) error { +// BuildIndexRange will build range of index for PhysicalIndexScan +func BuildIndexRange(sc *variable.StatementContext, p *PhysicalIndexScan) error { rb := rangeBuilder{sc: sc} for i := 0; i < p.accessInAndEqCount; i++ { // Build ranges for equal or in access conditions. @@ -285,22 +286,25 @@ func detachTableScanConditions(conditions []expression.Expression, table *model. return accessConditions, filterConditions } -func buildTableRange(p *PhysicalTableScan) error { - if len(p.AccessCondition) == 0 { - p.Ranges = []TableRange{{math.MinInt64, math.MaxInt64}} - return nil +// BuildTableRange will build range of pk for PhysicalTableScan +func BuildTableRange(accessConditions []expression.Expression, sc *variable.StatementContext) ([]TableRange, error) { + if len(accessConditions) == 0 { + return []TableRange{{math.MinInt64, math.MaxInt64}}, nil } - rb := rangeBuilder{sc: p.ctx.GetSessionVars().StmtCtx} + rb := rangeBuilder{sc: sc} rangePoints := fullRange - for _, cond := range p.AccessCondition { + for _, cond := range accessConditions { rangePoints = rb.intersection(rangePoints, rb.build(cond)) if rb.err != nil { - return errors.Trace(rb.err) + return nil, errors.Trace(rb.err) } } - p.Ranges = rb.buildTableRanges(rangePoints) - return errors.Trace(rb.err) + ranges := rb.buildTableRanges(rangePoints) + if rb.err != nil { + return nil, errors.Trace(rb.err) + } + return ranges, nil } // conditionChecker checks if this condition can be pushed to index plan. From 5ce91072e362cf32e52fbcb6b6fa2c39c637f4fe Mon Sep 17 00:00:00 2001 From: Yiding Cui Date: Tue, 14 Mar 2017 22:38:13 -0700 Subject: [PATCH 02/27] tiny change --- plan/physical_plan_builder.go | 1 - 1 file changed, 1 deletion(-) diff --git a/plan/physical_plan_builder.go b/plan/physical_plan_builder.go index 51ff4aa3f2a46..88b0a3f70c377 100644 --- a/plan/physical_plan_builder.go +++ b/plan/physical_plan_builder.go @@ -806,7 +806,6 @@ func buildTableScanByKeyAndCorCol(p *DataSource, pkName model.CIStr, fakeConds [ ts.Ranges = []TableRange{{math.MinInt64, math.MaxInt64}} var accessConditions, filterConditions []expression.Expression checker := conditionChecker{ - tableName: p.tableInfo.Name, pkName: pkName, length: types.UnspecifiedLength, } From 364e1721572bb562e02b62a9fafaa15fddf751d0 Mon Sep 17 00:00:00 2001 From: Yiding Cui Date: Wed, 15 Mar 2017 03:52:11 -0700 Subject: [PATCH 03/27] fix merge bug --- executor/builder.go | 14 ++-- executor/executor.go | 10 +-- plan/expr_to_pb.go | 1 + plan/logical_plans.go | 6 +- plan/physical_plan_builder.go | 132 +++++++++++++++++++++------------- 5 files changed, 100 insertions(+), 63 deletions(-) diff --git a/executor/builder.go b/executor/builder.go index a28696bc30378..d4a2dde947d88 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -18,6 +18,7 @@ import ( "strings" "github.com/juju/errors" + "github.com/ngaut/log" "github.com/pingcap/tidb/ast" "github.com/pingcap/tidb/context" "github.com/pingcap/tidb/expression" @@ -27,7 +28,6 @@ import ( "github.com/pingcap/tidb/model" "github.com/pingcap/tidb/plan" "github.com/pingcap/tidb/util/types" - "github.com/ngaut/log" ) // executorBuilder builds an Executor from a Plan. @@ -472,12 +472,12 @@ func (b *executorBuilder) buildAggregation(v *plan.PhysicalAggregation) Executor func (b *executorBuilder) buildSelection(v *plan.Selection) Executor { exec := &SelectionExec{ - Src: b.build(v.Children()[0]), - Condition: expression.ComposeCNFCondition(b.ctx, v.Conditions...), - schema: v.Schema(), - ctx: b.ctx, - scanController: v.ScanController, - accessConditions: v.AccessConditions, + Src: b.build(v.Children()[0]), + Condition: expression.ComposeCNFCondition(b.ctx, v.Conditions...), + schema: v.Schema(), + ctx: b.ctx, + scanController: v.ScanController, + accessConditions: v.AccessConditions, idxFilterConditions: v.IdxConditions, tblFilterConditions: v.TblConditions, } diff --git a/executor/executor.go b/executor/executor.go index 0e291de26d6ce..23a89df139c5e 100644 --- a/executor/executor.go +++ b/executor/executor.go @@ -18,6 +18,7 @@ import ( "sync/atomic" "github.com/juju/errors" + "github.com/ngaut/log" "github.com/pingcap/tidb/ast" "github.com/pingcap/tidb/context" "github.com/pingcap/tidb/expression" @@ -31,7 +32,6 @@ import ( "github.com/pingcap/tidb/tablecodec" "github.com/pingcap/tidb/terror" "github.com/pingcap/tidb/util/types" - "github.com/ngaut/log" ) var ( @@ -462,7 +462,7 @@ func (e *TableDualExec) Close() error { return nil } -func convertCorCol2Constant(expr expression.Expression) (expression.Expression, error){ +func convertCorCol2Constant(expr expression.Expression) (expression.Expression, error) { switch x := expr.(type) { case *expression.ScalarFunction: newArgs := make([]expression.Expression, 0, len(x.GetArgs())) @@ -493,9 +493,9 @@ type SelectionExec struct { ctx context.Context schema *expression.Schema - scanController bool - controllerInit bool - accessConditions []expression.Expression + scanController bool + controllerInit bool + accessConditions []expression.Expression idxFilterConditions []expression.Expression tblFilterConditions []expression.Expression } diff --git a/plan/expr_to_pb.go b/plan/expr_to_pb.go index a4bdb85c2dbc2..078ff0264129d 100644 --- a/plan/expr_to_pb.go +++ b/plan/expr_to_pb.go @@ -25,6 +25,7 @@ import ( "github.com/pingcap/tipb/go-tipb" ) +// ExpressionsToPB change expression to tipb.Expr func ExpressionsToPB(sc *variable.StatementContext, exprs []expression.Expression, client kv.Client) (pbExpr *tipb.Expr, pushed []expression.Expression, remained []expression.Expression) { pc := pbConverter{client: client, sc: sc} for _, expr := range exprs { diff --git a/plan/logical_plans.go b/plan/logical_plans.go index 1b4c82c31727c..63fb69696bffd 100644 --- a/plan/logical_plans.go +++ b/plan/logical_plans.go @@ -149,10 +149,10 @@ type Selection struct { // If ScanController is true, then the child of this selection is a scan, // which use pk or index - ScanController bool + ScanController bool AccessConditions []expression.Expression - IdxConditions []expression.Expression - TblConditions []expression.Expression + IdxConditions []expression.Expression + TblConditions []expression.Expression } func (p *Selection) extractCorrelatedCols() []*expression.CorrelatedColumn { diff --git a/plan/physical_plan_builder.go b/plan/physical_plan_builder.go index 88b0a3f70c377..8f213013e5bea 100644 --- a/plan/physical_plan_builder.go +++ b/plan/physical_plan_builder.go @@ -788,10 +788,10 @@ func (p *Union) convert2PhysicalPlan(prop *requiredProperty) (*physicalPlanInfo, func buildTableScanByKeyAndCorCol(p *DataSource, pkName model.CIStr, fakeConds []expression.Expression, origConds []expression.Expression) (*physicalPlanInfo, []expression.Expression, []expression.Expression) { client := p.ctx.GetClient() ts := &PhysicalTableScan{ - Table: p.tableInfo, - Columns: p.Columns, - TableAsName: p.TableAsName, - DBName: p.DBName, + Table: p.tableInfo, + Columns: p.Columns, + TableAsName: p.TableAsName, + DBName: p.DBName, physicalTableSource: physicalTableSource{client: client}, } ts.tp = Tbl @@ -817,7 +817,7 @@ func buildTableScanByKeyAndCorCol(p *DataSource, pkName model.CIStr, fakeConds [ accessConditions = append(accessConditions, origConds[id]) // TODO: it will lead to repeated computation cost. if checker.shouldReserve { - filterConditions= append(filterConditions, origConds[id]) + filterConditions = append(filterConditions, origConds[id]) checker.shouldReserve = false } } @@ -831,12 +831,12 @@ func buildIndexScanByKeyAndCorCol(p *DataSource, idx *model.IndexInfo, fakeConds []expression.Expression, []expression.Expression, []expression.Expression) { client := p.ctx.GetClient() is := &PhysicalIndexScan{ - Index: idx, - Table: p.tableInfo, - Columns: p.Columns, - TableAsName: p.TableAsName, - OutOfOrder: true, - DBName: p.DBName, + Index: idx, + Table: p.tableInfo, + Columns: p.Columns, + TableAsName: p.TableAsName, + OutOfOrder: true, + DBName: p.DBName, physicalTableSource: physicalTableSource{client: p.ctx.GetClient()}, } is.tp = Idx @@ -904,8 +904,61 @@ func convertCorCol2Constant(cond expression.Expression) expression.Expression { } } +func (p *Selection) getUsableIndicesAndPk(ds *DataSource) ([]*model.IndexInfo, model.CIStr) { + indices, _ := availableIndices(ds.indexHints, ds.tableInfo) + var usableIdxs []*model.IndexInfo + for _, idx := range indices { + // Currently we don't consider composite index. + if len(idx.Columns) > 1 { + continue + } + var idxCol *expression.Column + for _, col := range ds.schema.Columns { + if idx.Columns[0].Name.L == col.ColName.L { + idxCol = col + } + } + // This idx column should occur in one condition which contains both column and correlated column. + var usable bool + for _, cond := range p.Conditions { + cols := expression.ExtractColumns(cond) + if len(cols) != 1 || !cond.IsCorrelated() || !cols[0].Equal(idxCol, p.ctx) { + continue + } + usable = true + } + if usable { + usableIdxs = append(usableIdxs, idx) + } + } + var pkName model.CIStr + if ds.tableInfo.PKIsHandle { + var pkCol *expression.Column + for i, col := range ds.Columns { + if mysql.HasPriKeyFlag(col.Flag) { + pkCol = ds.schema.Columns[i] + } + } + // Pk should satisfies the same property. + var usable bool + for _, cond := range p.Conditions { + cols := expression.ExtractColumns(cond) + if !cond.IsCorrelated() || !cols[0].Equal(pkCol, p.ctx) { + continue + } + usable = true + } + if usable { + pkName = pkCol.ColName + } + } + return usableIdxs, pkName +} + // tryToBuildIndexScan will check the conditions contain correlated columns whether it can make indexScan. func (p *Selection) tryToBuildScanByKey(prop *requiredProperty) *physicalPlanInfo { + log.Warnf("cond: %v", p.Conditions) + log.Warnf("magic: %T %v", p.children[0], len(p.extractCorrelatedCols())) if ds, ok := p.children[0].(*DataSource); ok && len(p.extractCorrelatedCols()) > 0 { conds := make([]expression.Expression, 0, len(p.Conditions)) for _, cond := range p.Conditions { @@ -913,58 +966,39 @@ func (p *Selection) tryToBuildScanByKey(prop *requiredProperty) *physicalPlanInf // In this way, we could use the code of refiner.go. conds = append(conds, convertCorCol2Constant(cond)) } - indices, _ := availableIndices(ds.indexHints, ds.tableInfo) - var usableIdxs []*model.IndexInfo - for _, idx := range indices { - // Currently we don't consider composite index. - if len(idx.Columns) > 1 { - continue - } - usableIdxs = append(usableIdxs, idx) - } - var pkName model.CIStr - if ds.tableInfo.PKIsHandle { - for _, col := range ds.Columns { - if mysql.HasPriKeyFlag(col.Flag) { - pkName = col.Name - } - } - } + indices, pkName := p.getUsableIndicesAndPk(ds) + // If pk is handle, we will try to build TableScan by pk, otherwise we will try to build IndexScan by index. if pkName.L == "" { var finalInfo *physicalPlanInfo - for _, idx := range usableIdxs { + for _, idx := range indices { info, accessConds, idxConds, tblConds := buildIndexScanByKeyAndCorCol(ds, idx, conds, p.Conditions) if info == nil { continue } - p.ScanController = true - p.AccessConditions = accessConds - p.IdxConditions = idxConds - p.TblConditions = tblConds if finalInfo == nil || finalInfo.cost > info.cost { + p.onTable = true + p.ScanController = true + p.AccessConditions = accessConds + p.IdxConditions = idxConds + p.TblConditions = tblConds finalInfo = info } - return info } if finalInfo == nil { return nil } - finalInfo = p.matchProperty(prop, finalInfo) - return finalInfo - } else { - info, accessCondition, filterCondition := buildTableScanByKeyAndCorCol(ds, pkName, conds, p.Conditions) - if info == nil { - return nil - } - p.ScanController = true - p.AccessConditions = accessCondition - p.TblConditions = filterCondition - info = p.matchProperty(prop, info) - return info + return p.appendSelToInfo(finalInfo) } - } else { - return nil + info, accessCondition, filterCondition := buildTableScanByKeyAndCorCol(ds, pkName, conds, p.Conditions) + if info == nil { + return nil + } + p.onTable = true + p.ScanController = true + p.AccessConditions = accessCondition + p.TblConditions = filterCondition + return p.appendSelToInfo(info) } return nil } @@ -979,7 +1013,9 @@ func (p *Selection) convert2PhysicalPlan(prop *requiredProperty) (*physicalPlanI return info, nil } info = p.tryToBuildScanByKey(prop) + log.Warnf("info type: %T", info) if info != nil { + log.Warnf("info p type: %T", info.p) p.storePlanInfo(prop, info) return info, nil } From 36283cf9690bf0dc2722a4fa327b70379be3e2d6 Mon Sep 17 00:00:00 2001 From: Yiding Cui Date: Wed, 15 Mar 2017 03:59:28 -0700 Subject: [PATCH 04/27] remove log --- executor/builder.go | 2 -- executor/executor.go | 3 --- plan/physical_plan_builder.go | 3 --- 3 files changed, 8 deletions(-) diff --git a/executor/builder.go b/executor/builder.go index d4a2dde947d88..ecaea8558dbd7 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -18,7 +18,6 @@ import ( "strings" "github.com/juju/errors" - "github.com/ngaut/log" "github.com/pingcap/tidb/ast" "github.com/pingcap/tidb/context" "github.com/pingcap/tidb/expression" @@ -481,7 +480,6 @@ func (b *executorBuilder) buildSelection(v *plan.Selection) Executor { idxFilterConditions: v.IdxConditions, tblFilterConditions: v.TblConditions, } - log.Warnf("child type: %T", exec.Src) return exec } diff --git a/executor/executor.go b/executor/executor.go index 23a89df139c5e..aac211af2561c 100644 --- a/executor/executor.go +++ b/executor/executor.go @@ -18,7 +18,6 @@ import ( "sync/atomic" "github.com/juju/errors" - "github.com/ngaut/log" "github.com/pingcap/tidb/ast" "github.com/pingcap/tidb/context" "github.com/pingcap/tidb/expression" @@ -509,7 +508,6 @@ func (e *SelectionExec) initController() error { sc := e.ctx.GetSessionVars().StmtCtx client := e.ctx.GetClient() accesses := make([]expression.Expression, 0, len(e.accessConditions)) - log.Warnf("access Conditions: %v", e.accessConditions) for _, cond := range e.accessConditions { newCond, err := convertCorCol2Constant(cond) if err != nil { @@ -527,7 +525,6 @@ func (e *SelectionExec) initController() error { } switch x := e.Src.(type) { case *XSelectTableExec: - log.Warnf("access Conditions: %v", accesses) ranges, err := plan.BuildTableRange(accesses, sc) if err != nil { return errors.Trace(err) diff --git a/plan/physical_plan_builder.go b/plan/physical_plan_builder.go index 8f213013e5bea..c95f48b652d6e 100644 --- a/plan/physical_plan_builder.go +++ b/plan/physical_plan_builder.go @@ -114,7 +114,6 @@ func (p *DataSource) convert2TableScan(prop *requiredProperty) (*physicalPlanInf } func (p *DataSource) convert2IndexScan(prop *requiredProperty, index *model.IndexInfo) (*physicalPlanInfo, error) { - log.Warnf("begin conver idxScan") client := p.ctx.GetClient() is := &PhysicalIndexScan{ Index: index, @@ -1013,9 +1012,7 @@ func (p *Selection) convert2PhysicalPlan(prop *requiredProperty) (*physicalPlanI return info, nil } info = p.tryToBuildScanByKey(prop) - log.Warnf("info type: %T", info) if info != nil { - log.Warnf("info p type: %T", info.p) p.storePlanInfo(prop, info) return info, nil } From 7451287f79c793cec7bbdd576cd078b8205ed9a1 Mon Sep 17 00:00:00 2001 From: Yiding Cui Date: Wed, 15 Mar 2017 04:11:49 -0700 Subject: [PATCH 05/27] remove log --- plan/physical_plan_builder.go | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/plan/physical_plan_builder.go b/plan/physical_plan_builder.go index c95f48b652d6e..790dabf0574bc 100644 --- a/plan/physical_plan_builder.go +++ b/plan/physical_plan_builder.go @@ -955,9 +955,7 @@ func (p *Selection) getUsableIndicesAndPk(ds *DataSource) ([]*model.IndexInfo, m } // tryToBuildIndexScan will check the conditions contain correlated columns whether it can make indexScan. -func (p *Selection) tryToBuildScanByKey(prop *requiredProperty) *physicalPlanInfo { - log.Warnf("cond: %v", p.Conditions) - log.Warnf("magic: %T %v", p.children[0], len(p.extractCorrelatedCols())) +func (p *Selection) tryToBuildScanByKey() *physicalPlanInfo { if ds, ok := p.children[0].(*DataSource); ok && len(p.extractCorrelatedCols()) > 0 { conds := make([]expression.Expression, 0, len(p.Conditions)) for _, cond := range p.Conditions { @@ -1011,7 +1009,7 @@ func (p *Selection) convert2PhysicalPlan(prop *requiredProperty) (*physicalPlanI if info != nil { return info, nil } - info = p.tryToBuildScanByKey(prop) + info = p.tryToBuildScanByKey() if info != nil { p.storePlanInfo(prop, info) return info, nil From 053c2d4bea18fd23952502a0399c2333948f5a20 Mon Sep 17 00:00:00 2001 From: Yiding Cui Date: Wed, 15 Mar 2017 20:42:47 -0700 Subject: [PATCH 06/27] fix bug and tiny change --- executor/executor.go | 17 +++++++++-------- plan/physical_plan_builder.go | 6 ++---- 2 files changed, 11 insertions(+), 12 deletions(-) diff --git a/executor/executor.go b/executor/executor.go index aac211af2561c..a153b5808141a 100644 --- a/executor/executor.go +++ b/executor/executor.go @@ -515,14 +515,6 @@ func (e *SelectionExec) initController() error { } accesses = append(accesses, newCond) } - tblFilters := make([]expression.Expression, 0, len(e.tblFilterConditions)) - for _, cond := range e.tblFilterConditions { - newCond, err := convertCorCol2Constant(cond) - if err != nil { - return errors.Trace(err) - } - tblFilters = append(tblFilters, newCond) - } switch x := e.Src.(type) { case *XSelectTableExec: ranges, err := plan.BuildTableRange(accesses, sc) @@ -530,11 +522,20 @@ func (e *SelectionExec) initController() error { return errors.Trace(err) } x.ranges = ranges + tblFilters := make([]expression.Expression, 0, len(e.tblFilterConditions)) + for _, cond := range e.tblFilterConditions { + newCond, err := convertCorCol2Constant(cond) + if err != nil { + return errors.Trace(err) + } + tblFilters = append(tblFilters, newCond) + } var conds []expression.Expression x.where, _, conds = plan.ExpressionsToPB(sc, tblFilters, client) e.Condition = expression.ComposeCNFCondition(e.ctx, conds...) return nil case *XSelectIndexExec: + x.indexPlan.AccessCondition = accesses err := plan.BuildIndexRange(sc, x.indexPlan) if err != nil { return errors.Trace(err) diff --git a/plan/physical_plan_builder.go b/plan/physical_plan_builder.go index 790dabf0574bc..f906a948025ca 100644 --- a/plan/physical_plan_builder.go +++ b/plan/physical_plan_builder.go @@ -955,7 +955,7 @@ func (p *Selection) getUsableIndicesAndPk(ds *DataSource) ([]*model.IndexInfo, m } // tryToBuildIndexScan will check the conditions contain correlated columns whether it can make indexScan. -func (p *Selection) tryToBuildScanByKey() *physicalPlanInfo { +func (p *Selection) tryToBuildScanByKeyAndCorCol() *physicalPlanInfo { if ds, ok := p.children[0].(*DataSource); ok && len(p.extractCorrelatedCols()) > 0 { conds := make([]expression.Expression, 0, len(p.Conditions)) for _, cond := range p.Conditions { @@ -974,7 +974,6 @@ func (p *Selection) tryToBuildScanByKey() *physicalPlanInfo { continue } if finalInfo == nil || finalInfo.cost > info.cost { - p.onTable = true p.ScanController = true p.AccessConditions = accessConds p.IdxConditions = idxConds @@ -991,7 +990,6 @@ func (p *Selection) tryToBuildScanByKey() *physicalPlanInfo { if info == nil { return nil } - p.onTable = true p.ScanController = true p.AccessConditions = accessCondition p.TblConditions = filterCondition @@ -1009,7 +1007,7 @@ func (p *Selection) convert2PhysicalPlan(prop *requiredProperty) (*physicalPlanI if info != nil { return info, nil } - info = p.tryToBuildScanByKey() + info = p.tryToBuildScanByKeyAndCorCol() if info != nil { p.storePlanInfo(prop, info) return info, nil From e6632918bf8c051fa87906a1df8b92a4a794db0f Mon Sep 17 00:00:00 2001 From: Yiding Cui Date: Thu, 16 Mar 2017 02:27:46 -0700 Subject: [PATCH 07/27] address some comments --- executor/executor.go | 35 +++++++++++------------------------ plan/expr_to_pb.go | 2 +- 2 files changed, 12 insertions(+), 25 deletions(-) diff --git a/executor/executor.go b/executor/executor.go index a153b5808141a..2585c0e306103 100644 --- a/executor/executor.go +++ b/executor/executor.go @@ -461,27 +461,19 @@ func (e *TableDualExec) Close() error { return nil } -func convertCorCol2Constant(expr expression.Expression) (expression.Expression, error) { +func substituteCorCol2Constant(expr expression.Expression) (expression.Expression) { switch x := expr.(type) { case *expression.ScalarFunction: newArgs := make([]expression.Expression, 0, len(x.GetArgs())) for _, arg := range x.GetArgs() { - newArg, err := convertCorCol2Constant(arg) - if err != nil { - return nil, errors.Trace(err) - } - newArgs = append(newArgs, newArg) + newArgs = append(newArgs, substituteCorCol2Constant(arg)) } newSf, _ := expression.NewFunction(x.GetCtx(), x.FuncName.L, x.GetType(), newArgs...) return newSf, nil case *expression.CorrelatedColumn: - val, err := x.Eval(nil) - if err != nil { - return nil, errors.Trace(err) - } - return &expression.Constant{Value: val, RetType: x.GetType()}, nil + return &expression.Constant{Value: x.Data, RetType: x.GetType()} default: - return x.Clone(), nil + return x.Clone() } } @@ -492,6 +484,8 @@ type SelectionExec struct { ctx context.Context schema *expression.Schema + // scanController will tell whether this selection need to + // control the condition of below scan executor. scanController bool controllerInit bool accessConditions []expression.Expression @@ -504,15 +498,13 @@ func (e *SelectionExec) Schema() *expression.Schema { return e.schema } +// initController will init the conditions of the below scan executor. func (e *SelectionExec) initController() error { sc := e.ctx.GetSessionVars().StmtCtx client := e.ctx.GetClient() accesses := make([]expression.Expression, 0, len(e.accessConditions)) for _, cond := range e.accessConditions { - newCond, err := convertCorCol2Constant(cond) - if err != nil { - return errors.Trace(err) - } + newCond := substituteCorCol2Constant(cond) accesses = append(accesses, newCond) } switch x := e.Src.(type) { @@ -524,10 +516,7 @@ func (e *SelectionExec) initController() error { x.ranges = ranges tblFilters := make([]expression.Expression, 0, len(e.tblFilterConditions)) for _, cond := range e.tblFilterConditions { - newCond, err := convertCorCol2Constant(cond) - if err != nil { - return errors.Trace(err) - } + newCond := substituteCorCol2Constant(cond) tblFilters = append(tblFilters, newCond) } var conds []expression.Expression @@ -540,10 +529,8 @@ func (e *SelectionExec) initController() error { if err != nil { return errors.Trace(err) } - var idxConds, tblConds []expression.Expression - x.indexPlan.IndexConditionPBExpr, _, idxConds = plan.ExpressionsToPB(sc, e.idxFilterConditions, client) - x.indexPlan.TableConditionPBExpr, _, tblConds = plan.ExpressionsToPB(sc, e.tblFilterConditions, client) - e.Condition = expression.ComposeCNFCondition(e.ctx, append(idxConds, tblConds...)...) + x.indexPlan.IndexConditionPBExpr, _, _ = plan.ExpressionsToPB(sc, e.idxFilterConditions, client) + x.indexPlan.TableConditionPBExpr, _, _ = plan.ExpressionsToPB(sc, e.tblFilterConditions, client) return nil default: return errors.New("Error type of PhysicalPlan") diff --git a/plan/expr_to_pb.go b/plan/expr_to_pb.go index 078ff0264129d..e954d9b2a4a9d 100644 --- a/plan/expr_to_pb.go +++ b/plan/expr_to_pb.go @@ -25,7 +25,7 @@ import ( "github.com/pingcap/tipb/go-tipb" ) -// ExpressionsToPB change expression to tipb.Expr +// ExpressionsToPB change expression to tipb.Expr. func ExpressionsToPB(sc *variable.StatementContext, exprs []expression.Expression, client kv.Client) (pbExpr *tipb.Expr, pushed []expression.Expression, remained []expression.Expression) { pc := pbConverter{client: client, sc: sc} for _, expr := range exprs { From a25a3eb04f466e474978e97cf7080525ad554012 Mon Sep 17 00:00:00 2001 From: Yiding Cui Date: Thu, 16 Mar 2017 21:17:29 -0700 Subject: [PATCH 08/27] manage cast in plan build phase --- executor/executor.go | 8 +++----- plan/physical_plan_builder.go | 12 +++++++++--- 2 files changed, 12 insertions(+), 8 deletions(-) diff --git a/executor/executor.go b/executor/executor.go index 2585c0e306103..b70a312dffaa4 100644 --- a/executor/executor.go +++ b/executor/executor.go @@ -469,9 +469,9 @@ func substituteCorCol2Constant(expr expression.Expression) (expression.Expressio newArgs = append(newArgs, substituteCorCol2Constant(arg)) } newSf, _ := expression.NewFunction(x.GetCtx(), x.FuncName.L, x.GetType(), newArgs...) - return newSf, nil + return newSf case *expression.CorrelatedColumn: - return &expression.Constant{Value: x.Data, RetType: x.GetType()} + return &expression.Constant{Value: *x.Data, RetType: x.GetType()} default: return x.Clone() } @@ -519,9 +519,7 @@ func (e *SelectionExec) initController() error { newCond := substituteCorCol2Constant(cond) tblFilters = append(tblFilters, newCond) } - var conds []expression.Expression - x.where, _, conds = plan.ExpressionsToPB(sc, tblFilters, client) - e.Condition = expression.ComposeCNFCondition(e.ctx, conds...) + x.where, _, _ = plan.ExpressionsToPB(sc, tblFilters, client) return nil case *XSelectIndexExec: x.indexPlan.AccessCondition = accesses diff --git a/plan/physical_plan_builder.go b/plan/physical_plan_builder.go index f906a948025ca..f23901b4620cf 100644 --- a/plan/physical_plan_builder.go +++ b/plan/physical_plan_builder.go @@ -26,6 +26,7 @@ import ( "github.com/pingcap/tidb/mysql" "github.com/pingcap/tidb/terror" "github.com/pingcap/tidb/util/types" + "github.com/pingcap/tidb/ast" ) const ( @@ -852,7 +853,7 @@ func buildIndexScanByKeyAndCorCol(p *DataSource, idx *model.IndexInfo, fakeConds accessConditions := make([]expression.Expression, 0, len(fakeAccessConditions)) restOrigConds := make([]expression.Expression, 0, len(restFakeConds)) for i, j := 0, 0; i < len(fakeConds); { - if fakeConds[i].Equal(fakeAccessConditions[j], p.ctx) { + if j < len(fakeAccessConditions) && fakeConds[i].Equal(fakeAccessConditions[j], p.ctx) { accessConditions = append(accessConditions, origConds[i]) i, j = i+1, j+1 } else { @@ -868,7 +869,7 @@ func buildIndexScanByKeyAndCorCol(p *DataSource, idx *model.IndexInfo, fakeConds idxConds = make([]expression.Expression, 0, len(fakeIdxConds)) tblConds = make([]expression.Expression, 0, len(fakeTblConds)) for i, j := 0, 0; i < len(restFakeConds); { - if restFakeConds[i].Equal(fakeIdxConds[j], p.ctx) { + if j < len(fakeIdxConds) && restFakeConds[i].Equal(fakeIdxConds[j], p.ctx) { idxConds = append(idxConds, restOrigConds[i]) i, j = i+1, j+1 } else { @@ -894,7 +895,12 @@ func convertCorCol2Constant(cond expression.Expression) expression.Expression { for _, arg := range x.GetArgs() { newArgs = append(newArgs, convertCorCol2Constant(arg)) } - newSf, _ := expression.NewFunction(x.GetCtx(), x.FuncName.L, x.GetType(), newArgs...) + var newSf expression.Expression + if x.FuncName.L == ast.Cast { + newSf = expression.NewCastFunc(x.RetType, newArgs[0], x.GetCtx()) + } else { + newSf, _ = expression.NewFunction(x.GetCtx(), x.FuncName.L, x.GetType(), newArgs...) + } return newSf case *expression.CorrelatedColumn: return expression.One From 6aadeaacdec2f47c23650373b0cb8daf2c08e51f Mon Sep 17 00:00:00 2001 From: Yiding Cui Date: Fri, 17 Mar 2017 02:03:27 -0700 Subject: [PATCH 09/27] address comments and manage cast in executor phase --- executor/executor.go | 31 +++++++++++++++++++++++-------- plan/expr_to_pb.go | 2 +- plan/physical_plan_builder.go | 28 +++++++++++++++++++++------- 3 files changed, 45 insertions(+), 16 deletions(-) diff --git a/executor/executor.go b/executor/executor.go index b70a312dffaa4..76f17e4fbc62c 100644 --- a/executor/executor.go +++ b/executor/executor.go @@ -461,19 +461,33 @@ func (e *TableDualExec) Close() error { return nil } -func substituteCorCol2Constant(expr expression.Expression) (expression.Expression) { +func substituteCorCol2Constant(expr expression.Expression) (expression.Expression, bool) { switch x := expr.(type) { case *expression.ScalarFunction: + allConstant := true newArgs := make([]expression.Expression, 0, len(x.GetArgs())) for _, arg := range x.GetArgs() { - newArgs = append(newArgs, substituteCorCol2Constant(arg)) + newArg, ok := substituteCorCol2Constant(arg) + newArgs = append(newArgs, newArg) + allConstant = allConstant && ok } - newSf, _ := expression.NewFunction(x.GetCtx(), x.FuncName.L, x.GetType(), newArgs...) - return newSf + if allConstant { + val, _ := x.Eval(nil) + return &expression.Constant{Value: val}, true + } + var newSf expression.Expression + if x.FuncName.L == ast.Cast { + newSf = expression.NewCastFunc(x.RetType, newArgs[0], x.GetCtx()) + } else { + newSf, _ = expression.NewFunction(x.GetCtx(), x.FuncName.L, x.GetType(), newArgs...) + } + return newSf, false case *expression.CorrelatedColumn: - return &expression.Constant{Value: *x.Data, RetType: x.GetType()} + return &expression.Constant{Value: *x.Data, RetType: x.GetType()}, true + case *expression.Constant: + return x.Clone(), true default: - return x.Clone() + return x.Clone(), false } } @@ -499,12 +513,13 @@ func (e *SelectionExec) Schema() *expression.Schema { } // initController will init the conditions of the below scan executor. +// It will first substitute the correlated column to constant, then calc the range by new condition. func (e *SelectionExec) initController() error { sc := e.ctx.GetSessionVars().StmtCtx client := e.ctx.GetClient() accesses := make([]expression.Expression, 0, len(e.accessConditions)) for _, cond := range e.accessConditions { - newCond := substituteCorCol2Constant(cond) + newCond, _ := substituteCorCol2Constant(cond) accesses = append(accesses, newCond) } switch x := e.Src.(type) { @@ -516,7 +531,7 @@ func (e *SelectionExec) initController() error { x.ranges = ranges tblFilters := make([]expression.Expression, 0, len(e.tblFilterConditions)) for _, cond := range e.tblFilterConditions { - newCond := substituteCorCol2Constant(cond) + newCond, _ := substituteCorCol2Constant(cond) tblFilters = append(tblFilters, newCond) } x.where, _, _ = plan.ExpressionsToPB(sc, tblFilters, client) diff --git a/plan/expr_to_pb.go b/plan/expr_to_pb.go index e954d9b2a4a9d..8d47579424b0d 100644 --- a/plan/expr_to_pb.go +++ b/plan/expr_to_pb.go @@ -25,7 +25,7 @@ import ( "github.com/pingcap/tipb/go-tipb" ) -// ExpressionsToPB change expression to tipb.Expr. +// ExpressionsToPB converts expression to tipb.Expr. func ExpressionsToPB(sc *variable.StatementContext, exprs []expression.Expression, client kv.Client) (pbExpr *tipb.Expr, pushed []expression.Expression, remained []expression.Expression) { pc := pbConverter{client: client, sc: sc} for _, expr := range exprs { diff --git a/plan/physical_plan_builder.go b/plan/physical_plan_builder.go index f23901b4620cf..c89664d5ae96f 100644 --- a/plan/physical_plan_builder.go +++ b/plan/physical_plan_builder.go @@ -18,6 +18,7 @@ import ( "github.com/juju/errors" "github.com/ngaut/log" + "github.com/pingcap/tidb/ast" "github.com/pingcap/tidb/context" "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/infoschema" @@ -26,7 +27,6 @@ import ( "github.com/pingcap/tidb/mysql" "github.com/pingcap/tidb/terror" "github.com/pingcap/tidb/util/types" - "github.com/pingcap/tidb/ast" ) const ( @@ -888,12 +888,21 @@ func buildIndexScanByKeyAndCorCol(p *DataSource, idx *model.IndexInfo, fakeConds return is.matchProperty(&requiredProperty{}, &physicalPlanInfo{count: uint64(p.statisticTable.Count)}), accessConditions, idxConds, tblConds } -func convertCorCol2Constant(cond expression.Expression) expression.Expression { +// The first return value is the new expression, the second is a bool value tell whether the below expression is all constant. +// The Second is used for simplify the scalar function. +// If the args of one scalar function are all constant, we will substitute it to constant. +func substituteCorCol2Constant(cond expression.Expression) (expression.Expression, bool) { switch x := cond.(type) { case *expression.ScalarFunction: newArgs := make([]expression.Expression, 0, len(x.GetArgs())) + allConstant := true for _, arg := range x.GetArgs() { - newArgs = append(newArgs, convertCorCol2Constant(arg)) + newArg, ok := substituteCorCol2Constant(arg) + allConstant = allConstant && ok + newArgs = append(newArgs, newArg) + } + if allConstant { + return expression.One, true } var newSf expression.Expression if x.FuncName.L == ast.Cast { @@ -901,14 +910,18 @@ func convertCorCol2Constant(cond expression.Expression) expression.Expression { } else { newSf, _ = expression.NewFunction(x.GetCtx(), x.FuncName.L, x.GetType(), newArgs...) } - return newSf + return newSf, false case *expression.CorrelatedColumn: - return expression.One + return expression.One, true + case *expression.Constant: + return x.Clone(), true default: - return x.Clone() + return x.Clone(), false } } +// getUsableIndicesAndPk will simply check whether the pk or one index could used in this situation by +// checking whether this index or pk is contained in one condition that has correlated column. func (p *Selection) getUsableIndicesAndPk(ds *DataSource) ([]*model.IndexInfo, model.CIStr) { indices, _ := availableIndices(ds.indexHints, ds.tableInfo) var usableIdxs []*model.IndexInfo @@ -967,7 +980,8 @@ func (p *Selection) tryToBuildScanByKeyAndCorCol() *physicalPlanInfo { for _, cond := range p.Conditions { cond = pushDownNot(cond.Clone(), false, nil) // In this way, we could use the code of refiner.go. - conds = append(conds, convertCorCol2Constant(cond)) + newCond, _ := substituteCorCol2Constant(cond) + conds = append(conds, newCond) } indices, pkName := p.getUsableIndicesAndPk(ds) From ba29a95e6157e3b3591839d1551fa58aaba688c8 Mon Sep 17 00:00:00 2001 From: Yiding Cui Date: Sun, 19 Mar 2017 20:02:05 -0700 Subject: [PATCH 10/27] address some comments --- executor/executor.go | 35 ++++++++++++++++++++++------------- plan/logical_plans.go | 3 ++- 2 files changed, 24 insertions(+), 14 deletions(-) diff --git a/executor/executor.go b/executor/executor.go index 76f17e4fbc62c..1191387eea06e 100644 --- a/executor/executor.go +++ b/executor/executor.go @@ -461,19 +461,25 @@ func (e *TableDualExec) Close() error { return nil } -func substituteCorCol2Constant(expr expression.Expression) (expression.Expression, bool) { +func substituteCorCol2Constant(expr expression.Expression) (expression.Expression, bool, error) { switch x := expr.(type) { case *expression.ScalarFunction: allConstant := true newArgs := make([]expression.Expression, 0, len(x.GetArgs())) for _, arg := range x.GetArgs() { - newArg, ok := substituteCorCol2Constant(arg) + newArg, ok, err := substituteCorCol2Constant(arg) + if err != nil { + return nil, false, errors.Trace(err) + } newArgs = append(newArgs, newArg) allConstant = allConstant && ok } if allConstant { - val, _ := x.Eval(nil) - return &expression.Constant{Value: val}, true + val, err := x.Eval(nil) + if err != nil { + return nil, false, errors.Trace(err) + } + return &expression.Constant{Value: val}, true, nil } var newSf expression.Expression if x.FuncName.L == ast.Cast { @@ -481,13 +487,13 @@ func substituteCorCol2Constant(expr expression.Expression) (expression.Expressio } else { newSf, _ = expression.NewFunction(x.GetCtx(), x.FuncName.L, x.GetType(), newArgs...) } - return newSf, false + return newSf, false, nil case *expression.CorrelatedColumn: - return &expression.Constant{Value: *x.Data, RetType: x.GetType()}, true + return &expression.Constant{Value: *x.Data, RetType: x.GetType()}, true, nil case *expression.Constant: - return x.Clone(), true + return x.Clone(), true, nil default: - return x.Clone(), false + return x.Clone(), false, nil } } @@ -519,7 +525,10 @@ func (e *SelectionExec) initController() error { client := e.ctx.GetClient() accesses := make([]expression.Expression, 0, len(e.accessConditions)) for _, cond := range e.accessConditions { - newCond, _ := substituteCorCol2Constant(cond) + newCond, _, err := substituteCorCol2Constant(cond) + if err != nil { + return errors.Trace(err) + } accesses = append(accesses, newCond) } switch x := e.Src.(type) { @@ -531,7 +540,10 @@ func (e *SelectionExec) initController() error { x.ranges = ranges tblFilters := make([]expression.Expression, 0, len(e.tblFilterConditions)) for _, cond := range e.tblFilterConditions { - newCond, _ := substituteCorCol2Constant(cond) + newCond, _, err := substituteCorCol2Constant(cond) + if err != nil { + return errors.Trace(err) + } tblFilters = append(tblFilters, newCond) } x.where, _, _ = plan.ExpressionsToPB(sc, tblFilters, client) @@ -567,9 +579,6 @@ func (e *SelectionExec) Next() (*Row, error) { if srcRow == nil { return nil, nil } - if e.Condition == nil { - return srcRow, nil - } match, err := expression.EvalBool(e.Condition, srcRow.Data, e.ctx) if err != nil { return nil, errors.Trace(err) diff --git a/plan/logical_plans.go b/plan/logical_plans.go index 63fb69696bffd..e8bc8af6cb109 100644 --- a/plan/logical_plans.go +++ b/plan/logical_plans.go @@ -148,7 +148,8 @@ type Selection struct { onTable bool // If ScanController is true, then the child of this selection is a scan, - // which use pk or index + // which use pk or index. we will record the accessConditions, idxConditions, + // and tblConditions to control the below plan. ScanController bool AccessConditions []expression.Expression IdxConditions []expression.Expression From 02cbcf1b21425ea48f73f06a6f2ca212b0df0734 Mon Sep 17 00:00:00 2001 From: Yiding Cui Date: Sun, 19 Mar 2017 22:50:50 -0700 Subject: [PATCH 11/27] remove useless args --- executor/executor.go | 23 ++++++++++++----------- plan/physical_plan_builder.go | 19 ++++++++++--------- 2 files changed, 22 insertions(+), 20 deletions(-) diff --git a/executor/executor.go b/executor/executor.go index 1191387eea06e..879f8ffefa2d0 100644 --- a/executor/executor.go +++ b/executor/executor.go @@ -461,15 +461,16 @@ func (e *TableDualExec) Close() error { return nil } -func substituteCorCol2Constant(expr expression.Expression) (expression.Expression, bool, error) { +func substituteCorCol2Constant(expr expression.Expression) (expression.Expression, error) { switch x := expr.(type) { case *expression.ScalarFunction: allConstant := true newArgs := make([]expression.Expression, 0, len(x.GetArgs())) for _, arg := range x.GetArgs() { - newArg, ok, err := substituteCorCol2Constant(arg) + newArg, err := substituteCorCol2Constant(arg) + _, ok := newArg.(*expression.Constant) if err != nil { - return nil, false, errors.Trace(err) + return nil, errors.Trace(err) } newArgs = append(newArgs, newArg) allConstant = allConstant && ok @@ -477,9 +478,9 @@ func substituteCorCol2Constant(expr expression.Expression) (expression.Expressio if allConstant { val, err := x.Eval(nil) if err != nil { - return nil, false, errors.Trace(err) + return nil, errors.Trace(err) } - return &expression.Constant{Value: val}, true, nil + return &expression.Constant{Value: val}, nil } var newSf expression.Expression if x.FuncName.L == ast.Cast { @@ -487,13 +488,13 @@ func substituteCorCol2Constant(expr expression.Expression) (expression.Expressio } else { newSf, _ = expression.NewFunction(x.GetCtx(), x.FuncName.L, x.GetType(), newArgs...) } - return newSf, false, nil + return newSf, nil case *expression.CorrelatedColumn: - return &expression.Constant{Value: *x.Data, RetType: x.GetType()}, true, nil + return &expression.Constant{Value: *x.Data, RetType: x.GetType()}, nil case *expression.Constant: - return x.Clone(), true, nil + return x.Clone(), nil default: - return x.Clone(), false, nil + return x.Clone(), nil } } @@ -525,7 +526,7 @@ func (e *SelectionExec) initController() error { client := e.ctx.GetClient() accesses := make([]expression.Expression, 0, len(e.accessConditions)) for _, cond := range e.accessConditions { - newCond, _, err := substituteCorCol2Constant(cond) + newCond, err := substituteCorCol2Constant(cond) if err != nil { return errors.Trace(err) } @@ -540,7 +541,7 @@ func (e *SelectionExec) initController() error { x.ranges = ranges tblFilters := make([]expression.Expression, 0, len(e.tblFilterConditions)) for _, cond := range e.tblFilterConditions { - newCond, _, err := substituteCorCol2Constant(cond) + newCond, err := substituteCorCol2Constant(cond) if err != nil { return errors.Trace(err) } diff --git a/plan/physical_plan_builder.go b/plan/physical_plan_builder.go index c89664d5ae96f..5e90cf6a9413f 100644 --- a/plan/physical_plan_builder.go +++ b/plan/physical_plan_builder.go @@ -837,7 +837,7 @@ func buildIndexScanByKeyAndCorCol(p *DataSource, idx *model.IndexInfo, fakeConds TableAsName: p.TableAsName, OutOfOrder: true, DBName: p.DBName, - physicalTableSource: physicalTableSource{client: p.ctx.GetClient()}, + physicalTableSource: physicalTableSource{client: client}, } is.tp = Idx is.allocator = p.allocator @@ -891,18 +891,19 @@ func buildIndexScanByKeyAndCorCol(p *DataSource, idx *model.IndexInfo, fakeConds // The first return value is the new expression, the second is a bool value tell whether the below expression is all constant. // The Second is used for simplify the scalar function. // If the args of one scalar function are all constant, we will substitute it to constant. -func substituteCorCol2Constant(cond expression.Expression) (expression.Expression, bool) { +func substituteCorCol2Constant(cond expression.Expression) (expression.Expression) { switch x := cond.(type) { case *expression.ScalarFunction: newArgs := make([]expression.Expression, 0, len(x.GetArgs())) allConstant := true for _, arg := range x.GetArgs() { - newArg, ok := substituteCorCol2Constant(arg) + newArg := substituteCorCol2Constant(arg) + _, ok := newArg.(*expression.Constant) allConstant = allConstant && ok newArgs = append(newArgs, newArg) } if allConstant { - return expression.One, true + return expression.One } var newSf expression.Expression if x.FuncName.L == ast.Cast { @@ -910,13 +911,13 @@ func substituteCorCol2Constant(cond expression.Expression) (expression.Expressio } else { newSf, _ = expression.NewFunction(x.GetCtx(), x.FuncName.L, x.GetType(), newArgs...) } - return newSf, false + return newSf case *expression.CorrelatedColumn: - return expression.One, true + return expression.One case *expression.Constant: - return x.Clone(), true + return x.Clone() default: - return x.Clone(), false + return x.Clone() } } @@ -980,7 +981,7 @@ func (p *Selection) tryToBuildScanByKeyAndCorCol() *physicalPlanInfo { for _, cond := range p.Conditions { cond = pushDownNot(cond.Clone(), false, nil) // In this way, we could use the code of refiner.go. - newCond, _ := substituteCorCol2Constant(cond) + newCond := substituteCorCol2Constant(cond) conds = append(conds, newCond) } indices, pkName := p.getUsableIndicesAndPk(ds) From 9a9ae32e89828828db239ba00561c7d579e7d1c5 Mon Sep 17 00:00:00 2001 From: Yiding Cui Date: Mon, 20 Mar 2017 02:07:10 -0700 Subject: [PATCH 12/27] change some logic --- plan/logical_plans.go | 7 ++ plan/physical_plan_builder.go | 125 +++++++++++++++++++--------------- 2 files changed, 76 insertions(+), 56 deletions(-) diff --git a/plan/logical_plans.go b/plan/logical_plans.go index e8bc8af6cb109..a25ebf300f83e 100644 --- a/plan/logical_plans.go +++ b/plan/logical_plans.go @@ -154,6 +154,13 @@ type Selection struct { AccessConditions []expression.Expression IdxConditions []expression.Expression TblConditions []expression.Expression + + // Since one selection may call convert2PhysicalScan many times. We extract the PkName and indices + // used for scanController only once and store them to judge whether this selection can convert to + // scanController mode. + usefulPkName model.CIStr + usefulIndices []*model.IndexInfo + extractedUsefulThing bool } func (p *Selection) extractCorrelatedCols() []*expression.CorrelatedColumn { diff --git a/plan/physical_plan_builder.go b/plan/physical_plan_builder.go index 5e90cf6a9413f..fe37632ee27cd 100644 --- a/plan/physical_plan_builder.go +++ b/plan/physical_plan_builder.go @@ -891,7 +891,7 @@ func buildIndexScanByKeyAndCorCol(p *DataSource, idx *model.IndexInfo, fakeConds // The first return value is the new expression, the second is a bool value tell whether the below expression is all constant. // The Second is used for simplify the scalar function. // If the args of one scalar function are all constant, we will substitute it to constant. -func substituteCorCol2Constant(cond expression.Expression) (expression.Expression) { +func substituteCorCol2Constant(cond expression.Expression) expression.Expression { switch x := cond.(type) { case *expression.ScalarFunction: newArgs := make([]expression.Expression, 0, len(x.GetArgs())) @@ -922,29 +922,35 @@ func substituteCorCol2Constant(cond expression.Expression) (expression.Expressio } // getUsableIndicesAndPk will simply check whether the pk or one index could used in this situation by -// checking whether this index or pk is contained in one condition that has correlated column. +// checking whether this index or pk is contained in one condition that has correlated column, +// and whether this condition can be used as an access condition. func (p *Selection) getUsableIndicesAndPk(ds *DataSource) ([]*model.IndexInfo, model.CIStr) { indices, _ := availableIndices(ds.indexHints, ds.tableInfo) var usableIdxs []*model.IndexInfo for _, idx := range indices { - // Currently we don't consider composite index. + // TODO: Currently we don't consider composite index. if len(idx.Columns) > 1 { continue } - var idxCol *expression.Column - for _, col := range ds.schema.Columns { - if idx.Columns[0].Name.L == col.ColName.L { - idxCol = col - } + checker := &conditionChecker{ + idx: idx, + columnOffset: 0, + length: idx.Columns[0].Length, } // This idx column should occur in one condition which contains both column and correlated column. + // And conditionChecker.check(this condition) should be true. var usable bool - for _, cond := range p.Conditions { - cols := expression.ExtractColumns(cond) - if len(cols) != 1 || !cond.IsCorrelated() || !cols[0].Equal(idxCol, p.ctx) { + for _, expr := range p.Conditions { + cond := pushDownNot(expr.Clone(), false, nil) + if !cond.IsCorrelated() { continue } - usable = true + newCond := substituteCorCol2Constant(cond) + // If one cond is ok, then this index is useful. + if checker.check(newCond) { + usable = true + break + } } if usable { usableIdxs = append(usableIdxs, idx) @@ -958,13 +964,23 @@ func (p *Selection) getUsableIndicesAndPk(ds *DataSource) ([]*model.IndexInfo, m pkCol = ds.schema.Columns[i] } } - // Pk should satisfies the same property. + checker := conditionChecker{ + pkName: pkCol.ColName, + length: types.UnspecifiedLength, + } + // Pk should satisfies the same property as the index. var usable bool - for _, cond := range p.Conditions { - cols := expression.ExtractColumns(cond) - if !cond.IsCorrelated() || !cols[0].Equal(pkCol, p.ctx) { + for _, expr := range p.Conditions { + cond := pushDownNot(expr.Clone(), false, nil) + if !cond.IsCorrelated() { continue } + newCond := substituteCorCol2Constant(cond) + // If one cond is ok, then this index is useful. + if checker.check(newCond) { + usable = true + break + } usable = true } if usable { @@ -974,49 +990,40 @@ func (p *Selection) getUsableIndicesAndPk(ds *DataSource) ([]*model.IndexInfo, m return usableIdxs, pkName } -// tryToBuildIndexScan will check the conditions contain correlated columns whether it can make indexScan. -func (p *Selection) tryToBuildScanByKeyAndCorCol() *physicalPlanInfo { - if ds, ok := p.children[0].(*DataSource); ok && len(p.extractCorrelatedCols()) > 0 { - conds := make([]expression.Expression, 0, len(p.Conditions)) - for _, cond := range p.Conditions { - cond = pushDownNot(cond.Clone(), false, nil) - // In this way, we could use the code of refiner.go. - newCond := substituteCorCol2Constant(cond) - conds = append(conds, newCond) - } - indices, pkName := p.getUsableIndicesAndPk(ds) +// buildScanByKeyAndCorCol will build a scan which the condition is controlled by this selection. +func (p *Selection) buildScanByKeyAndCorCol() *physicalPlanInfo { + ds := p.children[0].(*DataSource) + conds := make([]expression.Expression, 0, len(p.Conditions)) + for _, expr := range p.Conditions { + cond := pushDownNot(expr.Clone(), false, nil) + // In this way, we could use the code of refiner.go. + newCond := substituteCorCol2Constant(cond) + conds = append(conds, newCond) + } - // If pk is handle, we will try to build TableScan by pk, otherwise we will try to build IndexScan by index. - if pkName.L == "" { - var finalInfo *physicalPlanInfo - for _, idx := range indices { - info, accessConds, idxConds, tblConds := buildIndexScanByKeyAndCorCol(ds, idx, conds, p.Conditions) - if info == nil { - continue - } - if finalInfo == nil || finalInfo.cost > info.cost { - p.ScanController = true - p.AccessConditions = accessConds - p.IdxConditions = idxConds - p.TblConditions = tblConds - finalInfo = info - } + // If pk is handle, we will try to build TableScan by pk, otherwise we will try to build IndexScan by index. + if p.usefulPkName.L == "" { + var finalInfo *physicalPlanInfo + for _, idx := range p.usefulIndices { + info, accessConds, idxConds, tblConds := buildIndexScanByKeyAndCorCol(ds, idx, conds, p.Conditions) + if info == nil { + continue } - if finalInfo == nil { - return nil + if finalInfo == nil || finalInfo.cost > info.cost { + p.ScanController = true + p.AccessConditions = accessConds + p.IdxConditions = idxConds + p.TblConditions = tblConds + finalInfo = info } - return p.appendSelToInfo(finalInfo) } - info, accessCondition, filterCondition := buildTableScanByKeyAndCorCol(ds, pkName, conds, p.Conditions) - if info == nil { - return nil - } - p.ScanController = true - p.AccessConditions = accessCondition - p.TblConditions = filterCondition - return p.appendSelToInfo(info) + return p.appendSelToInfo(finalInfo) } - return nil + info, accessCondition, filterCondition := buildTableScanByKeyAndCorCol(ds, p.usefulPkName, conds, p.Conditions) + p.ScanController = true + p.AccessConditions = accessCondition + p.TblConditions = filterCondition + return p.appendSelToInfo(info) } // convert2PhysicalPlan implements the LogicalPlan convert2PhysicalPlan interface. @@ -1028,8 +1035,14 @@ func (p *Selection) convert2PhysicalPlan(prop *requiredProperty) (*physicalPlanI if info != nil { return info, nil } - info = p.tryToBuildScanByKeyAndCorCol() - if info != nil { + if !p.extractedUsefulThing { + if ds, ok := p.children[0].(*DataSource); ok { + p.getUsableIndicesAndPk(ds) + } + p.extractedUsefulThing = true + } + if p.usefulPkName.L != "" || len(p.usefulIndices) > 0 { + info = p.buildScanByKeyAndCorCol() p.storePlanInfo(prop, info) return info, nil } From ef178a3513f0665009dc7f461582e15bab8da898 Mon Sep 17 00:00:00 2001 From: Yiding Cui Date: Mon, 20 Mar 2017 17:34:44 +0800 Subject: [PATCH 13/27] fix bug --- plan/physical_plan_builder.go | 14 ++++++++++++++ 1 file changed, 14 insertions(+) diff --git a/plan/physical_plan_builder.go b/plan/physical_plan_builder.go index fe37632ee27cd..0fcb3e4dcf712 100644 --- a/plan/physical_plan_builder.go +++ b/plan/physical_plan_builder.go @@ -852,6 +852,16 @@ func buildIndexScanByKeyAndCorCol(p *DataSource, idx *model.IndexInfo, fakeConds fakeAccessConditions, restFakeConds := detachIndexScanConditions(fakeConds, is) accessConditions := make([]expression.Expression, 0, len(fakeAccessConditions)) restOrigConds := make([]expression.Expression, 0, len(restFakeConds)) + // accessConditions is not in original order. So we could only use brute force there. + for i := 0; i < len(fakeConds); i++ { + for _, cond := range fakeAccessConditions { + if fakeConds[i].Equal(cond, p.ctx) { + accessConditions = append(accessConditions, origConds[i]) + } else { + restOrigConds = append(restOrigConds, origConds[i]) + } + } + } for i, j := 0, 0; i < len(fakeConds); { if j < len(fakeAccessConditions) && fakeConds[i].Equal(fakeAccessConditions[j], p.ctx) { accessConditions = append(accessConditions, origConds[i]) @@ -962,8 +972,12 @@ func (p *Selection) getUsableIndicesAndPk(ds *DataSource) ([]*model.IndexInfo, m for i, col := range ds.Columns { if mysql.HasPriKeyFlag(col.Flag) { pkCol = ds.schema.Columns[i] + break } } + if pkCol == nil { + return usableIdxs, pkName + } checker := conditionChecker{ pkName: pkCol.ColName, length: types.UnspecifiedLength, From 8029c1e7695082528b7706c88f9f291700a85f01 Mon Sep 17 00:00:00 2001 From: Yiding Cui Date: Tue, 21 Mar 2017 15:19:48 +0800 Subject: [PATCH 14/27] refactor --- executor/builder.go | 16 +-- executor/executor.go | 68 ++++++---- plan/logical_plans.go | 9 +- plan/physical_plan_builder.go | 240 ++++++++++------------------------ plan/physical_plan_test.go | 2 +- plan/refiner.go | 10 +- plan/statistics_helper.go | 3 +- 7 files changed, 134 insertions(+), 214 deletions(-) diff --git a/executor/builder.go b/executor/builder.go index ecaea8558dbd7..6e54ce7c20784 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -471,14 +471,14 @@ func (b *executorBuilder) buildAggregation(v *plan.PhysicalAggregation) Executor func (b *executorBuilder) buildSelection(v *plan.Selection) Executor { exec := &SelectionExec{ - Src: b.build(v.Children()[0]), - Condition: expression.ComposeCNFCondition(b.ctx, v.Conditions...), - schema: v.Schema(), - ctx: b.ctx, - scanController: v.ScanController, - accessConditions: v.AccessConditions, - idxFilterConditions: v.IdxConditions, - tblFilterConditions: v.TblConditions, + Src: b.build(v.Children()[0]), + Condition: expression.ComposeCNFCondition(b.ctx, v.Conditions...), + schema: v.Schema(), + ctx: b.ctx, + scanController: v.ScanController, + Conditions: v.Conditions, + usableIndices: v.UsefulIndices, + statsTbl: v.UsefulTbl, } return exec } diff --git a/executor/executor.go b/executor/executor.go index 879f8ffefa2d0..cc165b308b4a6 100644 --- a/executor/executor.go +++ b/executor/executor.go @@ -27,6 +27,7 @@ import ( "github.com/pingcap/tidb/model" "github.com/pingcap/tidb/mysql" "github.com/pingcap/tidb/plan" + "github.com/pingcap/tidb/plan/statistics" "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/tablecodec" "github.com/pingcap/tidb/terror" @@ -507,11 +508,11 @@ type SelectionExec struct { // scanController will tell whether this selection need to // control the condition of below scan executor. - scanController bool - controllerInit bool - accessConditions []expression.Expression - idxFilterConditions []expression.Expression - tblFilterConditions []expression.Expression + scanController bool + controllerInit bool + Conditions []expression.Expression + usableIndices []*model.IndexInfo + statsTbl *statistics.Table } // Schema implements the Executor Schema interface. @@ -520,47 +521,62 @@ func (e *SelectionExec) Schema() *expression.Schema { } // initController will init the conditions of the below scan executor. -// It will first substitute the correlated column to constant, then calc the range by new condition. +// It will first substitute the correlated column to constant, then build range and filter by new conditions. func (e *SelectionExec) initController() error { sc := e.ctx.GetSessionVars().StmtCtx client := e.ctx.GetClient() - accesses := make([]expression.Expression, 0, len(e.accessConditions)) - for _, cond := range e.accessConditions { - newCond, err := substituteCorCol2Constant(cond) + newConds := make([]expression.Expression, 0, len(e.Conditions)) + for _, cond := range e.Conditions { + newCond, err := substituteCorCol2Constant(cond.Clone()) if err != nil { return errors.Trace(err) } - accesses = append(accesses, newCond) + newConds = append(newConds, newCond) } + switch x := e.Src.(type) { case *XSelectTableExec: - ranges, err := plan.BuildTableRange(accesses, sc) + accessCondition, restCondtion := plan.DetachTableScanConditions(newConds, x.tableInfo) + x.where, _, _ = plan.ExpressionsToPB(sc, restCondtion, client) + ranges, err := plan.BuildTableRange(accessCondition, sc) if err != nil { return errors.Trace(err) } x.ranges = ranges - tblFilters := make([]expression.Expression, 0, len(e.tblFilterConditions)) - for _, cond := range e.tblFilterConditions { - newCond, err := substituteCorCol2Constant(cond) + case *XSelectIndexExec: + var chosenPlan *plan.PhysicalIndexScan + var chosenRowCount uint64 + for _, idx := range e.usableIndices { + is := *x.indexPlan + is.Index = idx + condsBackUp := make([]expression.Expression, 0, len(newConds)) + for _, cond := range newConds { + condsBackUp = append(condsBackUp, cond.Clone()) + } + is.AccessCondition, condsBackUp = plan.DetachIndexScanConditions(condsBackUp, &is) + idxConds, tblConds := plan.DetachIndexFilterConditions(condsBackUp, is.Index.Columns, is.Table) + is.IndexConditionPBExpr, _, _ = plan.ExpressionsToPB(sc, idxConds, client) + is.TableConditionPBExpr, _, _ = plan.ExpressionsToPB(sc, tblConds, client) + err := plan.BuildIndexRange(sc, &is) if err != nil { return errors.Trace(err) } - tblFilters = append(tblFilters, newCond) - } - x.where, _, _ = plan.ExpressionsToPB(sc, tblFilters, client) - return nil - case *XSelectIndexExec: - x.indexPlan.AccessCondition = accesses - err := plan.BuildIndexRange(sc, x.indexPlan) - if err != nil { - return errors.Trace(err) + rowCount, err := is.GetRowCountByIndexRanges(sc, e.statsTbl) + if err != nil { + return errors.Trace(err) + } + if chosenPlan == nil || chosenRowCount > rowCount { + chosenPlan = &is + chosenRowCount = rowCount + } } - x.indexPlan.IndexConditionPBExpr, _, _ = plan.ExpressionsToPB(sc, e.idxFilterConditions, client) - x.indexPlan.TableConditionPBExpr, _, _ = plan.ExpressionsToPB(sc, e.tblFilterConditions, client) - return nil + x.indexPlan = chosenPlan + x.where = x.indexPlan.TableConditionPBExpr + x.singleReadMode = plan.IsCoveringIndex(chosenPlan.Columns, chosenPlan.Index.Columns, chosenPlan.Table.PKIsHandle) default: return errors.New("Error type of PhysicalPlan") } + return nil } // Next implements the Executor Next interface. diff --git a/plan/logical_plans.go b/plan/logical_plans.go index a25ebf300f83e..bdb09e2a0a021 100644 --- a/plan/logical_plans.go +++ b/plan/logical_plans.go @@ -150,16 +150,15 @@ type Selection struct { // If ScanController is true, then the child of this selection is a scan, // which use pk or index. we will record the accessConditions, idxConditions, // and tblConditions to control the below plan. - ScanController bool - AccessConditions []expression.Expression - IdxConditions []expression.Expression - TblConditions []expression.Expression + ScanController bool // Since one selection may call convert2PhysicalScan many times. We extract the PkName and indices // used for scanController only once and store them to judge whether this selection can convert to // scanController mode. + // The usefulTbl is used during execution phase. usefulPkName model.CIStr - usefulIndices []*model.IndexInfo + UsefulIndices []*model.IndexInfo + UsefulTbl *statistics.Table extractedUsefulThing bool } diff --git a/plan/physical_plan_builder.go b/plan/physical_plan_builder.go index 0fcb3e4dcf712..7f9afdf105cf5 100644 --- a/plan/physical_plan_builder.go +++ b/plan/physical_plan_builder.go @@ -70,7 +70,7 @@ func (p *DataSource) convert2TableScan(prop *requiredProperty) (*physicalPlanInf for _, cond := range sel.Conditions { conds = append(conds, cond.Clone()) } - ts.AccessCondition, newSel.Conditions = detachTableScanConditions(conds, table) + ts.AccessCondition, newSel.Conditions = DetachTableScanConditions(conds, table) ts.TableConditionPBExpr, ts.tableFilterConditions, newSel.Conditions = ExpressionsToPB(sc, newSel.Conditions, client) ranges, err := BuildTableRange(ts.AccessCondition, sc) @@ -146,11 +146,11 @@ func (p *DataSource) convert2IndexScan(prop *requiredProperty, index *model.Inde for _, cond := range sel.Conditions { conds = append(conds, cond.Clone()) } - is.AccessCondition, newSel.Conditions = detachIndexScanConditions(conds, is) + is.AccessCondition, newSel.Conditions = DetachIndexScanConditions(conds, is) memDB := infoschema.IsMemoryDB(p.DBName.L) isDistReq := !memDB && client != nil && client.SupportRequestType(kv.ReqTypeIndex, 0) if isDistReq { - idxConds, tblConds := detachIndexFilterConditions(newSel.Conditions, is.Index.Columns, is.Table) + idxConds, tblConds := DetachIndexFilterConditions(newSel.Conditions, is.Index.Columns, is.Table) is.IndexConditionPBExpr, is.indexFilterConditions, idxConds = ExpressionsToPB(sc, idxConds, client) is.TableConditionPBExpr, is.tableFilterConditions, tblConds = ExpressionsToPB(sc, tblConds, client) newSel.Conditions = append(idxConds, tblConds...) @@ -162,7 +162,7 @@ func (p *DataSource) convert2IndexScan(prop *requiredProperty, index *model.Inde } log.Warn("truncate error in buildIndexRange") } - rowCount, err = is.getRowCountByIndexRanges(sc, statsTbl) + rowCount, err = is.GetRowCountByIndexRanges(sc, statsTbl) if err != nil { return nil, errors.Trace(err) } @@ -175,11 +175,12 @@ func (p *DataSource) convert2IndexScan(prop *requiredProperty, index *model.Inde rb := rangeBuilder{sc: p.ctx.GetSessionVars().StmtCtx} is.Ranges = rb.buildIndexRanges(fullRange, types.NewFieldType(mysql.TypeNull)) } - is.DoubleRead = !isCoveringIndex(is.Columns, is.Index.Columns, is.Table.PKIsHandle) + is.DoubleRead = !IsCoveringIndex(is.Columns, is.Index.Columns, is.Table.PKIsHandle) return resultPlan.matchProperty(prop, &physicalPlanInfo{count: rowCount}), nil } -func isCoveringIndex(columns []*model.ColumnInfo, indexColumns []*model.IndexColumn, pkIsHandle bool) bool { +// IsCoveringIndex will check whether the pk and index can cover all the column. +func IsCoveringIndex(columns []*model.ColumnInfo, indexColumns []*model.IndexColumn, pkIsHandle bool) bool { for _, colInfo := range columns { if pkIsHandle && mysql.HasPriKeyFlag(colInfo.Flag) { continue @@ -785,119 +786,6 @@ func (p *Union) convert2PhysicalPlan(prop *requiredProperty) (*physicalPlanInfo, return info, nil } -func buildTableScanByKeyAndCorCol(p *DataSource, pkName model.CIStr, fakeConds []expression.Expression, origConds []expression.Expression) (*physicalPlanInfo, []expression.Expression, []expression.Expression) { - client := p.ctx.GetClient() - ts := &PhysicalTableScan{ - Table: p.tableInfo, - Columns: p.Columns, - TableAsName: p.TableAsName, - DBName: p.DBName, - physicalTableSource: physicalTableSource{client: client}, - } - ts.tp = Tbl - ts.allocator = p.allocator - ts.SetSchema(p.Schema()) - ts.initIDAndContext(p.ctx) - if p.ctx.Txn() != nil { - ts.readOnly = p.ctx.Txn().IsReadOnly() - } else { - ts.readOnly = true - } - ts.Ranges = []TableRange{{math.MinInt64, math.MaxInt64}} - var accessConditions, filterConditions []expression.Expression - checker := conditionChecker{ - pkName: pkName, - length: types.UnspecifiedLength, - } - for id, cond := range fakeConds { - if !checker.check(cond) { - filterConditions = append(filterConditions, origConds[id]) - continue - } - accessConditions = append(accessConditions, origConds[id]) - // TODO: it will lead to repeated computation cost. - if checker.shouldReserve { - filterConditions = append(filterConditions, origConds[id]) - checker.shouldReserve = false - } - } - rowCount := uint64(p.statisticTable.Count) - ts.AccessCondition = accessConditions - ts.tableFilterConditions = filterConditions - return ts.matchProperty(&requiredProperty{}, &physicalPlanInfo{count: rowCount}), accessConditions, filterConditions -} - -func buildIndexScanByKeyAndCorCol(p *DataSource, idx *model.IndexInfo, fakeConds []expression.Expression, origConds []expression.Expression) (*physicalPlanInfo, - []expression.Expression, []expression.Expression, []expression.Expression) { - client := p.ctx.GetClient() - is := &PhysicalIndexScan{ - Index: idx, - Table: p.tableInfo, - Columns: p.Columns, - TableAsName: p.TableAsName, - OutOfOrder: true, - DBName: p.DBName, - physicalTableSource: physicalTableSource{client: client}, - } - is.tp = Idx - is.allocator = p.allocator - is.initIDAndContext(p.ctx) - is.SetSchema(p.schema) - if p.ctx.Txn() != nil { - is.readOnly = p.ctx.Txn().IsReadOnly() - } else { - is.readOnly = true - } - - fakeAccessConditions, restFakeConds := detachIndexScanConditions(fakeConds, is) - accessConditions := make([]expression.Expression, 0, len(fakeAccessConditions)) - restOrigConds := make([]expression.Expression, 0, len(restFakeConds)) - // accessConditions is not in original order. So we could only use brute force there. - for i := 0; i < len(fakeConds); i++ { - for _, cond := range fakeAccessConditions { - if fakeConds[i].Equal(cond, p.ctx) { - accessConditions = append(accessConditions, origConds[i]) - } else { - restOrigConds = append(restOrigConds, origConds[i]) - } - } - } - for i, j := 0, 0; i < len(fakeConds); { - if j < len(fakeAccessConditions) && fakeConds[i].Equal(fakeAccessConditions[j], p.ctx) { - accessConditions = append(accessConditions, origConds[i]) - i, j = i+1, j+1 - } else { - restOrigConds = append(restOrigConds, origConds[i]) - i++ - } - } - memDB := infoschema.IsMemoryDB(p.DBName.L) - isDistReq := !memDB && client != nil && client.SupportRequestType(kv.ReqTypeIndex, 0) - var idxConds, tblConds []expression.Expression - if isDistReq { - fakeIdxConds, fakeTblConds := detachIndexFilterConditions(restFakeConds, is.Index.Columns, is.Table) - idxConds = make([]expression.Expression, 0, len(fakeIdxConds)) - tblConds = make([]expression.Expression, 0, len(fakeTblConds)) - for i, j := 0, 0; i < len(restFakeConds); { - if j < len(fakeIdxConds) && restFakeConds[i].Equal(fakeIdxConds[j], p.ctx) { - idxConds = append(idxConds, restOrigConds[i]) - i, j = i+1, j+1 - } else { - tblConds = append(tblConds, restOrigConds[i]) - i++ - } - } - } - - rb := rangeBuilder{sc: p.ctx.GetSessionVars().StmtCtx} - is.Ranges = rb.buildIndexRanges(fullRange, types.NewFieldType(mysql.TypeNull)) - is.DoubleRead = !isCoveringIndex(is.Columns, is.Index.Columns, is.Table.PKIsHandle) - is.AccessCondition = accessConditions - is.indexFilterConditions = idxConds - is.tableFilterConditions = tblConds - return is.matchProperty(&requiredProperty{}, &physicalPlanInfo{count: uint64(p.statisticTable.Count)}), accessConditions, idxConds, tblConds -} - // The first return value is the new expression, the second is a bool value tell whether the below expression is all constant. // The Second is used for simplify the scalar function. // If the args of one scalar function are all constant, we will substitute it to constant. @@ -937,6 +825,14 @@ func substituteCorCol2Constant(cond expression.Expression) expression.Expression func (p *Selection) getUsableIndicesAndPk(ds *DataSource) ([]*model.IndexInfo, model.CIStr) { indices, _ := availableIndices(ds.indexHints, ds.tableInfo) var usableIdxs []*model.IndexInfo + var newConds []expression.Expression + for _, expr := range p.Conditions { + cond := pushDownNot(expr.Clone(), false, nil) + if !cond.IsCorrelated() { + continue + } + newConds = append(newConds, substituteCorCol2Constant(cond)) + } for _, idx := range indices { // TODO: Currently we don't consider composite index. if len(idx.Columns) > 1 { @@ -950,14 +846,9 @@ func (p *Selection) getUsableIndicesAndPk(ds *DataSource) ([]*model.IndexInfo, m // This idx column should occur in one condition which contains both column and correlated column. // And conditionChecker.check(this condition) should be true. var usable bool - for _, expr := range p.Conditions { - cond := pushDownNot(expr.Clone(), false, nil) - if !cond.IsCorrelated() { - continue - } - newCond := substituteCorCol2Constant(cond) + for _, cond := range newConds { // If one cond is ok, then this index is useful. - if checker.check(newCond) { + if checker.check(cond) { usable = true break } @@ -984,18 +875,11 @@ func (p *Selection) getUsableIndicesAndPk(ds *DataSource) ([]*model.IndexInfo, m } // Pk should satisfies the same property as the index. var usable bool - for _, expr := range p.Conditions { - cond := pushDownNot(expr.Clone(), false, nil) - if !cond.IsCorrelated() { - continue - } - newCond := substituteCorCol2Constant(cond) - // If one cond is ok, then this index is useful. - if checker.check(newCond) { + for _, cond := range newConds { + if checker.check(cond) { usable = true break } - usable = true } if usable { pkName = pkCol.ColName @@ -1004,40 +888,58 @@ func (p *Selection) getUsableIndicesAndPk(ds *DataSource) ([]*model.IndexInfo, m return usableIdxs, pkName } -// buildScanByKeyAndCorCol will build a scan which the condition is controlled by this selection. -func (p *Selection) buildScanByKeyAndCorCol() *physicalPlanInfo { +func (p *Selection) makeScanController() *physicalPlanInfo { + newSel := *p + newSel.ScanController = true + var child PhysicalPlan ds := p.children[0].(*DataSource) - conds := make([]expression.Expression, 0, len(p.Conditions)) - for _, expr := range p.Conditions { - cond := pushDownNot(expr.Clone(), false, nil) - // In this way, we could use the code of refiner.go. - newCond := substituteCorCol2Constant(cond) - conds = append(conds, newCond) - } - - // If pk is handle, we will try to build TableScan by pk, otherwise we will try to build IndexScan by index. - if p.usefulPkName.L == "" { - var finalInfo *physicalPlanInfo - for _, idx := range p.usefulIndices { - info, accessConds, idxConds, tblConds := buildIndexScanByKeyAndCorCol(ds, idx, conds, p.Conditions) - if info == nil { - continue - } - if finalInfo == nil || finalInfo.cost > info.cost { - p.ScanController = true - p.AccessConditions = accessConds - p.IdxConditions = idxConds - p.TblConditions = tblConds - finalInfo = info - } + newSel.UsefulTbl = ds.statisticTable + if p.usefulPkName.L != "" { + ts := &PhysicalTableScan{ + Table: ds.tableInfo, + Columns: ds.Columns, + TableAsName: ds.TableAsName, + DBName: ds.DBName, + physicalTableSource: physicalTableSource{client: ds.ctx.GetClient()}, + } + ts.tp = Tbl + ts.allocator = ds.allocator + ts.SetSchema(ds.schema) + ts.initIDAndContext(ds.ctx) + if ds.ctx.Txn() != nil { + ts.readOnly = p.ctx.Txn().IsReadOnly() + } else { + ts.readOnly = true } - return p.appendSelToInfo(finalInfo) + child = ts + } else { + is := &PhysicalIndexScan{ + Table: ds.tableInfo, + Index: p.UsefulIndices[0], + Columns: ds.Columns, + TableAsName: ds.TableAsName, + OutOfOrder: true, + DBName: ds.DBName, + physicalTableSource: physicalTableSource{client: ds.ctx.GetClient()}, + } + is.tp = Idx + is.allocator = ds.allocator + is.SetSchema(ds.schema) + is.initIDAndContext(ds.ctx) + if is.ctx.Txn() != nil { + is.readOnly = p.ctx.Txn().IsReadOnly() + } else { + is.readOnly = true + } + child = is } - info, accessCondition, filterCondition := buildTableScanByKeyAndCorCol(ds, p.usefulPkName, conds, p.Conditions) - p.ScanController = true - p.AccessConditions = accessCondition - p.TblConditions = filterCondition - return p.appendSelToInfo(info) + newSel.SetChildren(child) + info := &physicalPlanInfo{ + p: &newSel, + count: uint64(ds.statisticTable.Count), + } + info.cost = float64(info.count) * selectionFactor + return info } // convert2PhysicalPlan implements the LogicalPlan convert2PhysicalPlan interface. @@ -1051,12 +953,12 @@ func (p *Selection) convert2PhysicalPlan(prop *requiredProperty) (*physicalPlanI } if !p.extractedUsefulThing { if ds, ok := p.children[0].(*DataSource); ok { - p.getUsableIndicesAndPk(ds) + p.UsefulIndices, p.usefulPkName = p.getUsableIndicesAndPk(ds) } p.extractedUsefulThing = true } - if p.usefulPkName.L != "" || len(p.usefulIndices) > 0 { - info = p.buildScanByKeyAndCorCol() + if p.usefulPkName.L != "" || len(p.UsefulIndices) > 0 { + info = p.makeScanController() p.storePlanInfo(prop, info) return info, nil } diff --git a/plan/physical_plan_test.go b/plan/physical_plan_test.go index eea745295e4a9..846fa317f2148 100644 --- a/plan/physical_plan_test.go +++ b/plan/physical_plan_test.go @@ -693,7 +693,7 @@ func (s *testPlanSuite) TestCoveringIndex(c *C) { icl := ca.indexLens[i] indexCols = append(indexCols, &model.IndexColumn{Name: model.NewCIStr(icn), Length: icl}) } - covering := isCoveringIndex(columns, indexCols, pkIsHandle) + covering := IsCoveringIndex(columns, indexCols, pkIsHandle) c.Assert(covering, Equals, ca.isCovering) } } diff --git a/plan/refiner.go b/plan/refiner.go index 91353ef0296ef..07b4b24038a38 100644 --- a/plan/refiner.go +++ b/plan/refiner.go @@ -168,7 +168,8 @@ func checkIndexCondition(condition expression.Expression, indexColumns []*model. return true } -func detachIndexFilterConditions(conditions []expression.Expression, indexColumns []*model.IndexColumn, table *model.TableInfo) ([]expression.Expression, []expression.Expression) { +// DetachIndexFilterConditions will detach the access conditions from other conditions. +func DetachIndexFilterConditions(conditions []expression.Expression, indexColumns []*model.IndexColumn, table *model.TableInfo) ([]expression.Expression, []expression.Expression) { var pKName model.CIStr if table.PKIsHandle { for _, colInfo := range table.Columns { @@ -189,7 +190,8 @@ func detachIndexFilterConditions(conditions []expression.Expression, indexColumn return indexConditions, tableConditions } -func detachIndexScanConditions(conditions []expression.Expression, indexScan *PhysicalIndexScan) ([]expression.Expression, []expression.Expression) { +// DetachIndexScanConditions will detach the index filters from table filters. +func DetachIndexScanConditions(conditions []expression.Expression, indexScan *PhysicalIndexScan) ([]expression.Expression, []expression.Expression) { accessConds := make([]expression.Expression, len(indexScan.Index.Columns)) var filterConds []expression.Expression // pushDownNot here can convert query 'not (a != 1)' to 'a = 1'. @@ -247,8 +249,8 @@ func detachIndexScanConditions(conditions []expression.Expression, indexScan *Ph return accessConds, filterConds } -// detachTableScanConditions distinguishes between access conditions and filter conditions from conditions. -func detachTableScanConditions(conditions []expression.Expression, table *model.TableInfo) ([]expression.Expression, []expression.Expression) { +// DetachTableScanConditions distinguishes between access conditions and filter conditions from conditions. +func DetachTableScanConditions(conditions []expression.Expression, table *model.TableInfo) ([]expression.Expression, []expression.Expression) { var pkName model.CIStr if table.PKIsHandle { for _, colInfo := range table.Columns { diff --git a/plan/statistics_helper.go b/plan/statistics_helper.go index 06c932aac32d3..58e7782c4a324 100644 --- a/plan/statistics_helper.go +++ b/plan/statistics_helper.go @@ -24,7 +24,8 @@ import ( "github.com/pingcap/tidb/util/types" ) -func (is *PhysicalIndexScan) getRowCountByIndexRanges(sc *variable.StatementContext, statsTbl *statistics.Table) (uint64, error) { +// GetRowCountByIndexRanges will return the row count calculated from ranges. +func (is *PhysicalIndexScan) GetRowCountByIndexRanges(sc *variable.StatementContext, statsTbl *statistics.Table) (uint64, error) { var offset int for i := range is.Table.Indices { if is.Table.Indices[i].Name.L == is.Index.Name.L { From 977112966255b9694d92326e68f98d649236d9c2 Mon Sep 17 00:00:00 2001 From: Yiding Cui Date: Tue, 21 Mar 2017 19:43:24 +0800 Subject: [PATCH 15/27] address comments --- executor/builder.go | 3 -- executor/executor.go | 49 +++++++++++------------------ plan/logical_plans.go | 3 +- plan/physical_plan_builder.go | 58 ++++++++++++++++++++++------------- 4 files changed, 56 insertions(+), 57 deletions(-) diff --git a/executor/builder.go b/executor/builder.go index 6e54ce7c20784..94911f9da8ea0 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -472,13 +472,10 @@ func (b *executorBuilder) buildAggregation(v *plan.PhysicalAggregation) Executor func (b *executorBuilder) buildSelection(v *plan.Selection) Executor { exec := &SelectionExec{ Src: b.build(v.Children()[0]), - Condition: expression.ComposeCNFCondition(b.ctx, v.Conditions...), schema: v.Schema(), ctx: b.ctx, scanController: v.ScanController, Conditions: v.Conditions, - usableIndices: v.UsefulIndices, - statsTbl: v.UsefulTbl, } return exec } diff --git a/executor/executor.go b/executor/executor.go index cc165b308b4a6..18a907b8b7558 100644 --- a/executor/executor.go +++ b/executor/executor.go @@ -544,35 +544,15 @@ func (e *SelectionExec) initController() error { } x.ranges = ranges case *XSelectIndexExec: - var chosenPlan *plan.PhysicalIndexScan - var chosenRowCount uint64 - for _, idx := range e.usableIndices { - is := *x.indexPlan - is.Index = idx - condsBackUp := make([]expression.Expression, 0, len(newConds)) - for _, cond := range newConds { - condsBackUp = append(condsBackUp, cond.Clone()) - } - is.AccessCondition, condsBackUp = plan.DetachIndexScanConditions(condsBackUp, &is) - idxConds, tblConds := plan.DetachIndexFilterConditions(condsBackUp, is.Index.Columns, is.Table) - is.IndexConditionPBExpr, _, _ = plan.ExpressionsToPB(sc, idxConds, client) - is.TableConditionPBExpr, _, _ = plan.ExpressionsToPB(sc, tblConds, client) - err := plan.BuildIndexRange(sc, &is) - if err != nil { - return errors.Trace(err) - } - rowCount, err := is.GetRowCountByIndexRanges(sc, e.statsTbl) - if err != nil { - return errors.Trace(err) - } - if chosenPlan == nil || chosenRowCount > rowCount { - chosenPlan = &is - chosenRowCount = rowCount - } + x.indexPlan.AccessCondition, newConds = plan.DetachIndexScanConditions(newConds, x.indexPlan) + idxConds, tblConds := plan.DetachIndexFilterConditions(newConds, x.indexPlan.Index.Columns, x.indexPlan.Table) + x.indexPlan.IndexConditionPBExpr, _, _ = plan.ExpressionsToPB(sc, idxConds, client) + x.indexPlan.TableConditionPBExpr, _, _ = plan.ExpressionsToPB(sc, tblConds, client) + err := plan.BuildIndexRange(sc, x.indexPlan) + if err != nil { + return errors.Trace(err) } - x.indexPlan = chosenPlan x.where = x.indexPlan.TableConditionPBExpr - x.singleReadMode = plan.IsCoveringIndex(chosenPlan.Columns, chosenPlan.Index.Columns, chosenPlan.Table.PKIsHandle) default: return errors.New("Error type of PhysicalPlan") } @@ -596,11 +576,18 @@ func (e *SelectionExec) Next() (*Row, error) { if srcRow == nil { return nil, nil } - match, err := expression.EvalBool(e.Condition, srcRow.Data, e.ctx) - if err != nil { - return nil, errors.Trace(err) + allMatch := true + for _, cond := range e.Conditions { + match, err := expression.EvalBool(cond, srcRow.Data, e.ctx) + if err != nil { + return nil, errors.Trace(err) + } + if !match { + allMatch = false + break + } } - if match { + if allMatch { return srcRow, nil } } diff --git a/plan/logical_plans.go b/plan/logical_plans.go index bdb09e2a0a021..f4517f58d4fc1 100644 --- a/plan/logical_plans.go +++ b/plan/logical_plans.go @@ -157,8 +157,7 @@ type Selection struct { // scanController mode. // The usefulTbl is used during execution phase. usefulPkName model.CIStr - UsefulIndices []*model.IndexInfo - UsefulTbl *statistics.Table + usefulIndices []*model.IndexInfo extractedUsefulThing bool } diff --git a/plan/physical_plan_builder.go b/plan/physical_plan_builder.go index 7f9afdf105cf5..b39cf4a45371f 100644 --- a/plan/physical_plan_builder.go +++ b/plan/physical_plan_builder.go @@ -893,7 +893,6 @@ func (p *Selection) makeScanController() *physicalPlanInfo { newSel.ScanController = true var child PhysicalPlan ds := p.children[0].(*DataSource) - newSel.UsefulTbl = ds.statisticTable if p.usefulPkName.L != "" { ts := &PhysicalTableScan{ Table: ds.tableInfo, @@ -913,25 +912,42 @@ func (p *Selection) makeScanController() *physicalPlanInfo { } child = ts } else { - is := &PhysicalIndexScan{ - Table: ds.tableInfo, - Index: p.UsefulIndices[0], - Columns: ds.Columns, - TableAsName: ds.TableAsName, - OutOfOrder: true, - DBName: ds.DBName, - physicalTableSource: physicalTableSource{client: ds.ctx.GetClient()}, - } - is.tp = Idx - is.allocator = ds.allocator - is.SetSchema(ds.schema) - is.initIDAndContext(ds.ctx) - if is.ctx.Txn() != nil { - is.readOnly = p.ctx.Txn().IsReadOnly() - } else { - is.readOnly = true + var chosenPlan *PhysicalIndexScan + sc := ds.ctx.GetSessionVars().StmtCtx + client := ds.ctx.GetClient() + for _, idx := range p.usefulIndices { + condsBackUp := make([]expression.Expression, 0, len(p.Conditions)) + for _, cond := range p.Conditions { + condsBackUp = append(condsBackUp, cond.Clone()) + } + is := &PhysicalIndexScan{ + Table: ds.tableInfo, + Index: idx, + Columns: ds.Columns, + TableAsName: ds.TableAsName, + OutOfOrder: true, + DBName: ds.DBName, + physicalTableSource: physicalTableSource{client: ds.ctx.GetClient()}, + } + is.tp = Idx + is.allocator = ds.allocator + is.SetSchema(ds.schema) + is.initIDAndContext(ds.ctx) + if is.ctx.Txn() != nil { + is.readOnly = p.ctx.Txn().IsReadOnly() + } else { + is.readOnly = true + } + is.AccessCondition, condsBackUp = DetachIndexScanConditions(condsBackUp, is) + idxConds, tblConds := DetachIndexFilterConditions(condsBackUp, idx.Columns, is.Table) + is.IndexConditionPBExpr, is.indexFilterConditions, _ = ExpressionsToPB(sc, idxConds, client) + is.TableConditionPBExpr, is.tableFilterConditions, _ = ExpressionsToPB(sc, tblConds, client) + if chosenPlan == nil || chosenPlan.accessEqualCount < is.accessEqualCount || chosenPlan.accessInAndEqCount < is.accessInAndEqCount { + chosenPlan = is + } + is.DoubleRead = IsCoveringIndex(is.Columns, is.Index.Columns, is.Table.PKIsHandle) } - child = is + child = chosenPlan } newSel.SetChildren(child) info := &physicalPlanInfo{ @@ -953,11 +969,11 @@ func (p *Selection) convert2PhysicalPlan(prop *requiredProperty) (*physicalPlanI } if !p.extractedUsefulThing { if ds, ok := p.children[0].(*DataSource); ok { - p.UsefulIndices, p.usefulPkName = p.getUsableIndicesAndPk(ds) + p.usefulIndices, p.usefulPkName = p.getUsableIndicesAndPk(ds) } p.extractedUsefulThing = true } - if p.usefulPkName.L != "" || len(p.UsefulIndices) > 0 { + if p.usefulPkName.L != "" || len(p.usefulIndices) > 0 { info = p.makeScanController() p.storePlanInfo(prop, info) return info, nil From 7251827480cca34b6578bc5aae7161746c5dfc44 Mon Sep 17 00:00:00 2001 From: Yiding Cui Date: Tue, 21 Mar 2017 19:49:16 +0800 Subject: [PATCH 16/27] tiny change --- executor/executor.go | 2 -- plan/logical_plans.go | 1 - plan/physical_plan_builder.go | 9 ++++----- plan/physical_plan_test.go | 2 +- plan/statistics_helper.go | 3 +-- 5 files changed, 6 insertions(+), 11 deletions(-) diff --git a/executor/executor.go b/executor/executor.go index 18a907b8b7558..242d58d9250b9 100644 --- a/executor/executor.go +++ b/executor/executor.go @@ -27,7 +27,6 @@ import ( "github.com/pingcap/tidb/model" "github.com/pingcap/tidb/mysql" "github.com/pingcap/tidb/plan" - "github.com/pingcap/tidb/plan/statistics" "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/tablecodec" "github.com/pingcap/tidb/terror" @@ -512,7 +511,6 @@ type SelectionExec struct { controllerInit bool Conditions []expression.Expression usableIndices []*model.IndexInfo - statsTbl *statistics.Table } // Schema implements the Executor Schema interface. diff --git a/plan/logical_plans.go b/plan/logical_plans.go index f4517f58d4fc1..1349a680a7210 100644 --- a/plan/logical_plans.go +++ b/plan/logical_plans.go @@ -155,7 +155,6 @@ type Selection struct { // Since one selection may call convert2PhysicalScan many times. We extract the PkName and indices // used for scanController only once and store them to judge whether this selection can convert to // scanController mode. - // The usefulTbl is used during execution phase. usefulPkName model.CIStr usefulIndices []*model.IndexInfo extractedUsefulThing bool diff --git a/plan/physical_plan_builder.go b/plan/physical_plan_builder.go index b39cf4a45371f..d543059ceee16 100644 --- a/plan/physical_plan_builder.go +++ b/plan/physical_plan_builder.go @@ -162,7 +162,7 @@ func (p *DataSource) convert2IndexScan(prop *requiredProperty, index *model.Inde } log.Warn("truncate error in buildIndexRange") } - rowCount, err = is.GetRowCountByIndexRanges(sc, statsTbl) + rowCount, err = is.getRowCountByIndexRanges(sc, statsTbl) if err != nil { return nil, errors.Trace(err) } @@ -175,12 +175,11 @@ func (p *DataSource) convert2IndexScan(prop *requiredProperty, index *model.Inde rb := rangeBuilder{sc: p.ctx.GetSessionVars().StmtCtx} is.Ranges = rb.buildIndexRanges(fullRange, types.NewFieldType(mysql.TypeNull)) } - is.DoubleRead = !IsCoveringIndex(is.Columns, is.Index.Columns, is.Table.PKIsHandle) + is.DoubleRead = !isCoveringIndex(is.Columns, is.Index.Columns, is.Table.PKIsHandle) return resultPlan.matchProperty(prop, &physicalPlanInfo{count: rowCount}), nil } -// IsCoveringIndex will check whether the pk and index can cover all the column. -func IsCoveringIndex(columns []*model.ColumnInfo, indexColumns []*model.IndexColumn, pkIsHandle bool) bool { +func isCoveringIndex(columns []*model.ColumnInfo, indexColumns []*model.IndexColumn, pkIsHandle bool) bool { for _, colInfo := range columns { if pkIsHandle && mysql.HasPriKeyFlag(colInfo.Flag) { continue @@ -945,7 +944,7 @@ func (p *Selection) makeScanController() *physicalPlanInfo { if chosenPlan == nil || chosenPlan.accessEqualCount < is.accessEqualCount || chosenPlan.accessInAndEqCount < is.accessInAndEqCount { chosenPlan = is } - is.DoubleRead = IsCoveringIndex(is.Columns, is.Index.Columns, is.Table.PKIsHandle) + is.DoubleRead = isCoveringIndex(is.Columns, is.Index.Columns, is.Table.PKIsHandle) } child = chosenPlan } diff --git a/plan/physical_plan_test.go b/plan/physical_plan_test.go index 846fa317f2148..eea745295e4a9 100644 --- a/plan/physical_plan_test.go +++ b/plan/physical_plan_test.go @@ -693,7 +693,7 @@ func (s *testPlanSuite) TestCoveringIndex(c *C) { icl := ca.indexLens[i] indexCols = append(indexCols, &model.IndexColumn{Name: model.NewCIStr(icn), Length: icl}) } - covering := IsCoveringIndex(columns, indexCols, pkIsHandle) + covering := isCoveringIndex(columns, indexCols, pkIsHandle) c.Assert(covering, Equals, ca.isCovering) } } diff --git a/plan/statistics_helper.go b/plan/statistics_helper.go index 58e7782c4a324..06c932aac32d3 100644 --- a/plan/statistics_helper.go +++ b/plan/statistics_helper.go @@ -24,8 +24,7 @@ import ( "github.com/pingcap/tidb/util/types" ) -// GetRowCountByIndexRanges will return the row count calculated from ranges. -func (is *PhysicalIndexScan) GetRowCountByIndexRanges(sc *variable.StatementContext, statsTbl *statistics.Table) (uint64, error) { +func (is *PhysicalIndexScan) getRowCountByIndexRanges(sc *variable.StatementContext, statsTbl *statistics.Table) (uint64, error) { var offset int for i := range is.Table.Indices { if is.Table.Indices[i].Name.L == is.Index.Name.L { From e45f669c31736e87b09b1d78aa0da1197dd8daea Mon Sep 17 00:00:00 2001 From: Yiding Cui Date: Wed, 22 Mar 2017 14:12:45 +0800 Subject: [PATCH 17/27] address some comments --- executor/executor.go | 5 ++--- plan/physical_plan_builder.go | 8 ++++---- 2 files changed, 6 insertions(+), 7 deletions(-) diff --git a/executor/executor.go b/executor/executor.go index 242d58d9250b9..c96395bcb1a1c 100644 --- a/executor/executor.go +++ b/executor/executor.go @@ -468,10 +468,10 @@ func substituteCorCol2Constant(expr expression.Expression) (expression.Expressio newArgs := make([]expression.Expression, 0, len(x.GetArgs())) for _, arg := range x.GetArgs() { newArg, err := substituteCorCol2Constant(arg) - _, ok := newArg.(*expression.Constant) if err != nil { return nil, errors.Trace(err) } + _, ok := newArg.(*expression.Constant) newArgs = append(newArgs, newArg) allConstant = allConstant && ok } @@ -510,7 +510,6 @@ type SelectionExec struct { scanController bool controllerInit bool Conditions []expression.Expression - usableIndices []*model.IndexInfo } // Schema implements the Executor Schema interface. @@ -552,7 +551,7 @@ func (e *SelectionExec) initController() error { } x.where = x.indexPlan.TableConditionPBExpr default: - return errors.New("Error type of PhysicalPlan") + return errors.Errorf("Error type of Executor: %T", x) } return nil } diff --git a/plan/physical_plan_builder.go b/plan/physical_plan_builder.go index d543059ceee16..b00d9a738d0dc 100644 --- a/plan/physical_plan_builder.go +++ b/plan/physical_plan_builder.go @@ -822,14 +822,14 @@ func substituteCorCol2Constant(cond expression.Expression) expression.Expression // checking whether this index or pk is contained in one condition that has correlated column, // and whether this condition can be used as an access condition. func (p *Selection) getUsableIndicesAndPk(ds *DataSource) ([]*model.IndexInfo, model.CIStr) { - indices, _ := availableIndices(ds.indexHints, ds.tableInfo) + indices, includeTableScan := availableIndices(ds.indexHints, ds.tableInfo) var usableIdxs []*model.IndexInfo var newConds []expression.Expression for _, expr := range p.Conditions { - cond := pushDownNot(expr.Clone(), false, nil) - if !cond.IsCorrelated() { + if !expr.IsCorrelated() { continue } + cond := pushDownNot(expr.Clone(), false, nil) newConds = append(newConds, substituteCorCol2Constant(cond)) } for _, idx := range indices { @@ -857,7 +857,7 @@ func (p *Selection) getUsableIndicesAndPk(ds *DataSource) ([]*model.IndexInfo, m } } var pkName model.CIStr - if ds.tableInfo.PKIsHandle { + if ds.tableInfo.PKIsHandle && includeTableScan { var pkCol *expression.Column for i, col := range ds.Columns { if mysql.HasPriKeyFlag(col.Flag) { From 4e0012bd21cfc7be27f1d6dd01bc6b5302447fd7 Mon Sep 17 00:00:00 2001 From: Yiding Cui Date: Wed, 22 Mar 2017 14:21:47 +0800 Subject: [PATCH 18/27] add test --- executor/executor_test.go | 13 +++++++++++++ 1 file changed, 13 insertions(+) diff --git a/executor/executor_test.go b/executor/executor_test.go index 97ce1dadadd95..bba9e1415d1bc 100644 --- a/executor/executor_test.go +++ b/executor/executor_test.go @@ -1280,3 +1280,16 @@ func (s *testSuite) TestHistoryRead(c *C) { tk.MustExec("set @@tidb_snapshot = ''") tk.MustQuery("select * from history_read order by a").Check(testkit.Rows("2 ", "4 ", "8 8", "9 9")) } + +func (s *testSuite) TestScanControlSelection(c *C) { + defer func() { + s.cleanEnv(c) + testleak.AfterTest(c)() + }() + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(a int primary key, b int, c int, index idx_b(b))") + tk.MustExec("insert into t values (1, 1, 1), (2, 1, 1), (3, 1, 2), (4, 2, 3)") + tk.MustQuery("select (select count(1) k from t s where s.b = t1.c) from t t1").Check(testkit.Rows("3", "3", "1", "0")) +} From f0ec2e06b6e8ec36d4ea00cbaab936cdc676d6db Mon Sep 17 00:00:00 2001 From: Yiding Cui Date: Wed, 22 Mar 2017 14:54:07 +0800 Subject: [PATCH 19/27] remove redundant function --- executor/executor.go | 39 +--------------------------- expression/column.go | 49 +++++++++++++++++++++++++++++++++++ plan/physical_plan_builder.go | 42 ++---------------------------- 3 files changed, 52 insertions(+), 78 deletions(-) diff --git a/executor/executor.go b/executor/executor.go index c96395bcb1a1c..bc9a436557d77 100644 --- a/executor/executor.go +++ b/executor/executor.go @@ -461,43 +461,6 @@ func (e *TableDualExec) Close() error { return nil } -func substituteCorCol2Constant(expr expression.Expression) (expression.Expression, error) { - switch x := expr.(type) { - case *expression.ScalarFunction: - allConstant := true - newArgs := make([]expression.Expression, 0, len(x.GetArgs())) - for _, arg := range x.GetArgs() { - newArg, err := substituteCorCol2Constant(arg) - if err != nil { - return nil, errors.Trace(err) - } - _, ok := newArg.(*expression.Constant) - newArgs = append(newArgs, newArg) - allConstant = allConstant && ok - } - if allConstant { - val, err := x.Eval(nil) - if err != nil { - return nil, errors.Trace(err) - } - return &expression.Constant{Value: val}, nil - } - var newSf expression.Expression - if x.FuncName.L == ast.Cast { - newSf = expression.NewCastFunc(x.RetType, newArgs[0], x.GetCtx()) - } else { - newSf, _ = expression.NewFunction(x.GetCtx(), x.FuncName.L, x.GetType(), newArgs...) - } - return newSf, nil - case *expression.CorrelatedColumn: - return &expression.Constant{Value: *x.Data, RetType: x.GetType()}, nil - case *expression.Constant: - return x.Clone(), nil - default: - return x.Clone(), nil - } -} - // SelectionExec represents a filter executor. type SelectionExec struct { Src Executor @@ -524,7 +487,7 @@ func (e *SelectionExec) initController() error { client := e.ctx.GetClient() newConds := make([]expression.Expression, 0, len(e.Conditions)) for _, cond := range e.Conditions { - newCond, err := substituteCorCol2Constant(cond.Clone()) + newCond, err := expression.SubstituteCorCol2Constant(cond.Clone(), true) if err != nil { return errors.Trace(err) } diff --git a/expression/column.go b/expression/column.go index f121b2afd3541..9c4f666153772 100644 --- a/expression/column.go +++ b/expression/column.go @@ -17,7 +17,9 @@ import ( "bytes" "fmt" + "github.com/juju/errors" "github.com/ngaut/log" + "github.com/pingcap/tidb/ast" "github.com/pingcap/tidb/context" "github.com/pingcap/tidb/model" "github.com/pingcap/tidb/util/codec" @@ -166,3 +168,50 @@ func Column2Exprs(cols []*Column) []Expression { } return result } + +// SubstituteCorCol2Constant will substitute correlated column to constant value which it contains. +// If the args of one scalar function are all constant, we will substitute it to constant. +// If it's called in plan phase, correlated column will change to expression.One. +// If in executor phase, correlated column will change to value it contains. +func SubstituteCorCol2Constant(expr Expression, eval bool) (Expression, error) { + switch x := expr.(type) { + case *ScalarFunction: + allConstant := true + newArgs := make([]Expression, 0, len(x.GetArgs())) + for _, arg := range x.GetArgs() { + newArg, err := SubstituteCorCol2Constant(arg, eval) + if err != nil { + return nil, errors.Trace(err) + } + _, ok := newArg.(*Constant) + newArgs = append(newArgs, newArg) + allConstant = allConstant && ok + } + if allConstant { + if !eval { + return One, nil + } + val, err := x.Eval(nil) + if err != nil { + return nil, errors.Trace(err) + } + return &Constant{Value: val}, nil + } + var newSf Expression + if x.FuncName.L == ast.Cast { + newSf = NewCastFunc(x.RetType, newArgs[0], x.GetCtx()) + } else { + newSf, _ = NewFunction(x.GetCtx(), x.FuncName.L, x.GetType(), newArgs...) + } + return newSf, nil + case *CorrelatedColumn: + if !eval { + return One, nil + } + return &Constant{Value: *x.Data, RetType: x.GetType()}, nil + case *Constant: + return x.Clone(), nil + default: + return x.Clone(), nil + } +} diff --git a/plan/physical_plan_builder.go b/plan/physical_plan_builder.go index b00d9a738d0dc..43862cef0ba10 100644 --- a/plan/physical_plan_builder.go +++ b/plan/physical_plan_builder.go @@ -18,7 +18,6 @@ import ( "github.com/juju/errors" "github.com/ngaut/log" - "github.com/pingcap/tidb/ast" "github.com/pingcap/tidb/context" "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/infoschema" @@ -785,39 +784,6 @@ func (p *Union) convert2PhysicalPlan(prop *requiredProperty) (*physicalPlanInfo, return info, nil } -// The first return value is the new expression, the second is a bool value tell whether the below expression is all constant. -// The Second is used for simplify the scalar function. -// If the args of one scalar function are all constant, we will substitute it to constant. -func substituteCorCol2Constant(cond expression.Expression) expression.Expression { - switch x := cond.(type) { - case *expression.ScalarFunction: - newArgs := make([]expression.Expression, 0, len(x.GetArgs())) - allConstant := true - for _, arg := range x.GetArgs() { - newArg := substituteCorCol2Constant(arg) - _, ok := newArg.(*expression.Constant) - allConstant = allConstant && ok - newArgs = append(newArgs, newArg) - } - if allConstant { - return expression.One - } - var newSf expression.Expression - if x.FuncName.L == ast.Cast { - newSf = expression.NewCastFunc(x.RetType, newArgs[0], x.GetCtx()) - } else { - newSf, _ = expression.NewFunction(x.GetCtx(), x.FuncName.L, x.GetType(), newArgs...) - } - return newSf - case *expression.CorrelatedColumn: - return expression.One - case *expression.Constant: - return x.Clone() - default: - return x.Clone() - } -} - // getUsableIndicesAndPk will simply check whether the pk or one index could used in this situation by // checking whether this index or pk is contained in one condition that has correlated column, // and whether this condition can be used as an access condition. @@ -830,7 +796,8 @@ func (p *Selection) getUsableIndicesAndPk(ds *DataSource) ([]*model.IndexInfo, m continue } cond := pushDownNot(expr.Clone(), false, nil) - newConds = append(newConds, substituteCorCol2Constant(cond)) + newCond, _ := expression.SubstituteCorCol2Constant(cond, false) + newConds = append(newConds, newCond) } for _, idx := range indices { // TODO: Currently we don't consider composite index. @@ -912,8 +879,6 @@ func (p *Selection) makeScanController() *physicalPlanInfo { child = ts } else { var chosenPlan *PhysicalIndexScan - sc := ds.ctx.GetSessionVars().StmtCtx - client := ds.ctx.GetClient() for _, idx := range p.usefulIndices { condsBackUp := make([]expression.Expression, 0, len(p.Conditions)) for _, cond := range p.Conditions { @@ -938,9 +903,6 @@ func (p *Selection) makeScanController() *physicalPlanInfo { is.readOnly = true } is.AccessCondition, condsBackUp = DetachIndexScanConditions(condsBackUp, is) - idxConds, tblConds := DetachIndexFilterConditions(condsBackUp, idx.Columns, is.Table) - is.IndexConditionPBExpr, is.indexFilterConditions, _ = ExpressionsToPB(sc, idxConds, client) - is.TableConditionPBExpr, is.tableFilterConditions, _ = ExpressionsToPB(sc, tblConds, client) if chosenPlan == nil || chosenPlan.accessEqualCount < is.accessEqualCount || chosenPlan.accessInAndEqCount < is.accessInAndEqCount { chosenPlan = is } From 02db63cba280bd676efeac97b7848bbd5d21950e Mon Sep 17 00:00:00 2001 From: Yiding Cui Date: Wed, 22 Mar 2017 15:03:40 +0800 Subject: [PATCH 20/27] move function --- executor/executor.go | 2 +- expression/column.go | 49 ----------------------------------- expression/util.go | 41 +++++++++++++++++++++++++++++ plan/physical_plan_builder.go | 8 ++++-- 4 files changed, 48 insertions(+), 52 deletions(-) diff --git a/executor/executor.go b/executor/executor.go index bc9a436557d77..5f4a2dcbf460f 100644 --- a/executor/executor.go +++ b/executor/executor.go @@ -487,7 +487,7 @@ func (e *SelectionExec) initController() error { client := e.ctx.GetClient() newConds := make([]expression.Expression, 0, len(e.Conditions)) for _, cond := range e.Conditions { - newCond, err := expression.SubstituteCorCol2Constant(cond.Clone(), true) + newCond, err := expression.SubstituteCorCol2Constant(cond.Clone()) if err != nil { return errors.Trace(err) } diff --git a/expression/column.go b/expression/column.go index 9c4f666153772..f121b2afd3541 100644 --- a/expression/column.go +++ b/expression/column.go @@ -17,9 +17,7 @@ import ( "bytes" "fmt" - "github.com/juju/errors" "github.com/ngaut/log" - "github.com/pingcap/tidb/ast" "github.com/pingcap/tidb/context" "github.com/pingcap/tidb/model" "github.com/pingcap/tidb/util/codec" @@ -168,50 +166,3 @@ func Column2Exprs(cols []*Column) []Expression { } return result } - -// SubstituteCorCol2Constant will substitute correlated column to constant value which it contains. -// If the args of one scalar function are all constant, we will substitute it to constant. -// If it's called in plan phase, correlated column will change to expression.One. -// If in executor phase, correlated column will change to value it contains. -func SubstituteCorCol2Constant(expr Expression, eval bool) (Expression, error) { - switch x := expr.(type) { - case *ScalarFunction: - allConstant := true - newArgs := make([]Expression, 0, len(x.GetArgs())) - for _, arg := range x.GetArgs() { - newArg, err := SubstituteCorCol2Constant(arg, eval) - if err != nil { - return nil, errors.Trace(err) - } - _, ok := newArg.(*Constant) - newArgs = append(newArgs, newArg) - allConstant = allConstant && ok - } - if allConstant { - if !eval { - return One, nil - } - val, err := x.Eval(nil) - if err != nil { - return nil, errors.Trace(err) - } - return &Constant{Value: val}, nil - } - var newSf Expression - if x.FuncName.L == ast.Cast { - newSf = NewCastFunc(x.RetType, newArgs[0], x.GetCtx()) - } else { - newSf, _ = NewFunction(x.GetCtx(), x.FuncName.L, x.GetType(), newArgs...) - } - return newSf, nil - case *CorrelatedColumn: - if !eval { - return One, nil - } - return &Constant{Value: *x.Data, RetType: x.GetType()}, nil - case *Constant: - return x.Clone(), nil - default: - return x.Clone(), nil - } -} diff --git a/expression/util.go b/expression/util.go index 8d23ecaad538b..a6c1065ce4253 100644 --- a/expression/util.go +++ b/expression/util.go @@ -175,3 +175,44 @@ func (d *distinctChecker) Check(values []interface{}) (bool, error) { d.existingKeys[key] = true return true, nil } + +// SubstituteCorCol2Constant will substitute correlated column to constant value which it contains. +// If the args of one scalar function are all constant, we will substitute it to constant. +// If it's called in plan phase, correlated column will change to expression.One. +// If in executor phase, correlated column will change to value it contains. +func SubstituteCorCol2Constant(expr Expression) (Expression, error) { + switch x := expr.(type) { + case *ScalarFunction: + allConstant := true + newArgs := make([]Expression, 0, len(x.GetArgs())) + for _, arg := range x.GetArgs() { + newArg, err := SubstituteCorCol2Constant(arg) + if err != nil { + return nil, errors.Trace(err) + } + _, ok := newArg.(*Constant) + newArgs = append(newArgs, newArg) + allConstant = allConstant && ok + } + if allConstant { + val, err := x.Eval(nil) + if err != nil { + return nil, errors.Trace(err) + } + return &Constant{Value: val}, nil + } + var newSf Expression + if x.FuncName.L == ast.Cast { + newSf = NewCastFunc(x.RetType, newArgs[0], x.GetCtx()) + } else { + newSf, _ = NewFunction(x.GetCtx(), x.FuncName.L, x.GetType(), newArgs...) + } + return newSf, nil + case *CorrelatedColumn: + return &Constant{Value: *x.Data, RetType: x.GetType()}, nil + case *Constant: + return x.Clone(), nil + default: + return x.Clone(), nil + } +} diff --git a/plan/physical_plan_builder.go b/plan/physical_plan_builder.go index 43862cef0ba10..cd623ba51f5b2 100644 --- a/plan/physical_plan_builder.go +++ b/plan/physical_plan_builder.go @@ -795,8 +795,12 @@ func (p *Selection) getUsableIndicesAndPk(ds *DataSource) ([]*model.IndexInfo, m if !expr.IsCorrelated() { continue } - cond := pushDownNot(expr.Clone(), false, nil) - newCond, _ := expression.SubstituteCorCol2Constant(cond, false) + cond := pushDownNot(expr, false, nil) + corCols := extractCorColumns(cond) + for _, col := range corCols { + *col.Data = expression.One.Value + } + newCond, _ := expression.SubstituteCorCol2Constant(cond) newConds = append(newConds, newCond) } for _, idx := range indices { From c791ff709d684ba7b8b6adb7368d4cbbe4f4d474 Mon Sep 17 00:00:00 2001 From: Yiding Cui Date: Wed, 22 Mar 2017 16:10:49 +0800 Subject: [PATCH 21/27] move code --- plan/decorrelate.go | 81 ++++++++++++++++++++++++ plan/logical_plans.go | 8 +-- plan/physical_plan_builder.go | 115 +++++++++------------------------- 3 files changed, 114 insertions(+), 90 deletions(-) diff --git a/plan/decorrelate.go b/plan/decorrelate.go index ef40bfaa69de5..ddc82c7563894 100644 --- a/plan/decorrelate.go +++ b/plan/decorrelate.go @@ -17,6 +17,8 @@ import ( "github.com/pingcap/tidb/ast" "github.com/pingcap/tidb/context" "github.com/pingcap/tidb/expression" + "github.com/pingcap/tidb/model" + "github.com/pingcap/tidb/mysql" "github.com/pingcap/tidb/util/types" ) @@ -158,6 +160,11 @@ func (s *decorrelateSolver) optimize(p LogicalPlan, _ context.Context, _ *idAllo } } } + if sel, ok := p.(*Selection); ok { + if ds, ok := p.(*DataSource); ok { + sel.canControlScan = sel.hasUsableIndicesAndPk(ds) + } + } newChildren := make([]Plan, 0, len(p.Children())) for _, child := range p.Children() { np, _ := s.optimize(child.(LogicalPlan), nil, nil) @@ -167,3 +174,77 @@ func (s *decorrelateSolver) optimize(p LogicalPlan, _ context.Context, _ *idAllo p.SetChildren(newChildren...) return p, nil } + +// hasUsableIndicesAndPk will simply check whether the pk or one index could used in this situation by +// checking whether this index or pk is contained in one condition that has correlated column, +// and whether this condition can be used as an access condition. +func (p *Selection) hasUsableIndicesAndPk(ds *DataSource) bool { + indices, includeTableScan := availableIndices(ds.indexHints, ds.tableInfo) + var usableIdxs []*model.IndexInfo + var newConds []expression.Expression + for _, expr := range p.Conditions { + if !expr.IsCorrelated() { + continue + } + cond := pushDownNot(expr, false, nil) + corCols := extractCorColumns(cond) + for _, col := range corCols { + *col.Data = expression.One.Value + } + newCond, _ := expression.SubstituteCorCol2Constant(cond) + newConds = append(newConds, newCond) + } + for _, idx := range indices { + // TODO: Currently we don't consider composite index. + if len(idx.Columns) > 1 { + continue + } + checker := &conditionChecker{ + idx: idx, + columnOffset: 0, + length: idx.Columns[0].Length, + } + // This idx column should occur in one condition which contains both column and correlated column. + // And conditionChecker.check(this condition) should be true. + var usable bool + for _, cond := range newConds { + // If one cond is ok, then this index is useful. + if checker.check(cond) { + usable = true + break + } + } + if usable { + usableIdxs = append(usableIdxs, idx) + } + } + var pkName model.CIStr + if ds.tableInfo.PKIsHandle && includeTableScan { + var pkCol *expression.Column + for i, col := range ds.Columns { + if mysql.HasPriKeyFlag(col.Flag) { + pkCol = ds.schema.Columns[i] + break + } + } + if pkCol == nil { + return len(usableIdxs) > 0 + } + checker := conditionChecker{ + pkName: pkCol.ColName, + length: types.UnspecifiedLength, + } + // Pk should satisfies the same property as the index. + var usable bool + for _, cond := range newConds { + if checker.check(cond) { + usable = true + break + } + } + if usable { + pkName = pkCol.ColName + } + } + return len(usableIdxs) > 0 || pkName.L != "" +} diff --git a/plan/logical_plans.go b/plan/logical_plans.go index 1349a680a7210..79c55a2bb6705 100644 --- a/plan/logical_plans.go +++ b/plan/logical_plans.go @@ -152,12 +152,8 @@ type Selection struct { // and tblConditions to control the below plan. ScanController bool - // Since one selection may call convert2PhysicalScan many times. We extract the PkName and indices - // used for scanController only once and store them to judge whether this selection can convert to - // scanController mode. - usefulPkName model.CIStr - usefulIndices []*model.IndexInfo - extractedUsefulThing bool + // We will check this at decorrelate phase. + canControlScan bool } func (p *Selection) extractCorrelatedCols() []*expression.CorrelatedColumn { diff --git a/plan/physical_plan_builder.go b/plan/physical_plan_builder.go index cd623ba51f5b2..5452008d17030 100644 --- a/plan/physical_plan_builder.go +++ b/plan/physical_plan_builder.go @@ -784,86 +784,22 @@ func (p *Union) convert2PhysicalPlan(prop *requiredProperty) (*physicalPlanInfo, return info, nil } -// getUsableIndicesAndPk will simply check whether the pk or one index could used in this situation by -// checking whether this index or pk is contained in one condition that has correlated column, -// and whether this condition can be used as an access condition. -func (p *Selection) getUsableIndicesAndPk(ds *DataSource) ([]*model.IndexInfo, model.CIStr) { +func (p *Selection) makeScanController() *physicalPlanInfo { + newSel := *p + newSel.ScanController = true + var child PhysicalPlan + ds := p.children[0].(*DataSource) indices, includeTableScan := availableIndices(ds.indexHints, ds.tableInfo) - var usableIdxs []*model.IndexInfo - var newConds []expression.Expression - for _, expr := range p.Conditions { - if !expr.IsCorrelated() { - continue - } - cond := pushDownNot(expr, false, nil) - corCols := extractCorColumns(cond) - for _, col := range corCols { - *col.Data = expression.One.Value - } - newCond, _ := expression.SubstituteCorCol2Constant(cond) - newConds = append(newConds, newCond) - } - for _, idx := range indices { - // TODO: Currently we don't consider composite index. - if len(idx.Columns) > 1 { - continue - } - checker := &conditionChecker{ - idx: idx, - columnOffset: 0, - length: idx.Columns[0].Length, - } - // This idx column should occur in one condition which contains both column and correlated column. - // And conditionChecker.check(this condition) should be true. - var usable bool - for _, cond := range newConds { - // If one cond is ok, then this index is useful. - if checker.check(cond) { - usable = true - break - } - } - if usable { - usableIdxs = append(usableIdxs, idx) - } - } - var pkName model.CIStr + var pkCol *expression.Column if ds.tableInfo.PKIsHandle && includeTableScan { - var pkCol *expression.Column for i, col := range ds.Columns { if mysql.HasPriKeyFlag(col.Flag) { pkCol = ds.schema.Columns[i] break } } - if pkCol == nil { - return usableIdxs, pkName - } - checker := conditionChecker{ - pkName: pkCol.ColName, - length: types.UnspecifiedLength, - } - // Pk should satisfies the same property as the index. - var usable bool - for _, cond := range newConds { - if checker.check(cond) { - usable = true - break - } - } - if usable { - pkName = pkCol.ColName - } } - return usableIdxs, pkName -} - -func (p *Selection) makeScanController() *physicalPlanInfo { - newSel := *p - newSel.ScanController = true - var child PhysicalPlan - ds := p.children[0].(*DataSource) - if p.usefulPkName.L != "" { + if pkCol != nil { ts := &PhysicalTableScan{ Table: ds.tableInfo, Columns: ds.Columns, @@ -883,9 +819,21 @@ func (p *Selection) makeScanController() *physicalPlanInfo { child = ts } else { var chosenPlan *PhysicalIndexScan - for _, idx := range p.usefulIndices { - condsBackUp := make([]expression.Expression, 0, len(p.Conditions)) - for _, cond := range p.Conditions { + var corColConds []expression.Expression + for _, expr := range p.Conditions { + if !expr.IsCorrelated() { + continue + } + cond, _ := expression.SubstituteCorCol2Constant(expr) + corColConds = append(corColConds, cond) + } + for _, idx := range indices { + // TODO: Currently we don't consider composite index. + if len(idx.Columns) > 1 { + continue + } + condsBackUp := make([]expression.Expression, 0, len(corColConds)) + for _, cond := range corColConds { condsBackUp = append(condsBackUp, cond.Clone()) } is := &PhysicalIndexScan{ @@ -906,11 +854,16 @@ func (p *Selection) makeScanController() *physicalPlanInfo { } else { is.readOnly = true } - is.AccessCondition, condsBackUp = DetachIndexScanConditions(condsBackUp, is) - if chosenPlan == nil || chosenPlan.accessEqualCount < is.accessEqualCount || chosenPlan.accessInAndEqCount < is.accessInAndEqCount { + accessConds, _ := DetachIndexScanConditions(condsBackUp, is) + if len(accessConds) == 0 { + continue + } + better := chosenPlan == nil || chosenPlan.accessEqualCount < is.accessEqualCount + better = better || (chosenPlan.accessEqualCount == is.accessEqualCount && chosenPlan.accessInAndEqCount < is.accessInAndEqCount) + if better { chosenPlan = is } - is.DoubleRead = isCoveringIndex(is.Columns, is.Index.Columns, is.Table.PKIsHandle) + is.DoubleRead = !isCoveringIndex(is.Columns, is.Index.Columns, is.Table.PKIsHandle) } child = chosenPlan } @@ -932,13 +885,7 @@ func (p *Selection) convert2PhysicalPlan(prop *requiredProperty) (*physicalPlanI if info != nil { return info, nil } - if !p.extractedUsefulThing { - if ds, ok := p.children[0].(*DataSource); ok { - p.usefulIndices, p.usefulPkName = p.getUsableIndicesAndPk(ds) - } - p.extractedUsefulThing = true - } - if p.usefulPkName.L != "" || len(p.usefulIndices) > 0 { + if p.canControlScan { info = p.makeScanController() p.storePlanInfo(prop, info) return info, nil From 8674c3f24999beec5b03b2da6c77f62b06300c02 Mon Sep 17 00:00:00 2001 From: Yiding Cui Date: Wed, 22 Mar 2017 19:04:14 +0800 Subject: [PATCH 22/27] typo error && add explain --- executor/explain_test.go | 74 ++++++++++++++++++++++++++++++++++++++++ plan/decorrelate.go | 2 +- plan/physical_plans.go | 3 +- 3 files changed, 77 insertions(+), 2 deletions(-) diff --git a/executor/explain_test.go b/executor/explain_test.go index eafce877782fe..7933c842a1319 100644 --- a/executor/explain_test.go +++ b/executor/explain_test.go @@ -440,6 +440,80 @@ func (s *testSuite) TestExplain(c *C) { "gt(test.t1.c2, 1)" ] } +}`, + }, + }, + { + "select (select count(1) k from t1 s where s.c1 = t1.c1 having k != 0) from t1", + []string{ + "TableScan_12", "TableScan_13", "Selection_4", "StreamAgg_15", "Selection_10", "Apply_16", "Projection_2", + }, + []string{ + "Apply_16", "Selection_4", "StreamAgg_15", "Selection_10", "Apply_16", "Projection_2", "", + }, + []string{ + `{ + "db": "test", + "table": "t1", + "desc": false, + "keep order": false, + "push down info": { + "limit": 0, + "access conditions": null, + "index filter conditions": null, + "table filter conditions": null + } +}`, + `{ + "db": "test", + "table": "t1", + "desc": false, + "keep order": false, + "push down info": { + "limit": 0, + "access conditions": null, + "index filter conditions": null, + "table filter conditions": null + } +}`, + `{ + "condition": [ + "eq(s.c1, test.t1.c1)" + ], + "scanController": true, + "child": "TableScan_13" +}`, + `{ + "AggFuncs": [ + "count(1)" + ], + "GroupByItems": null, + "child": "Selection_4" +}`, + `{ + "condition": [ + "ne(aggregation_5_col_0, 0)" + ], + "scanController": false, + "child": "StreamAgg_15" +}`, + `{ + "innerPlan": "Selection_10", + "outerPlan": "TableScan_12", + "join": { + "eqCond": null, + "leftCond": null, + "rightCond": null, + "otherCond": null, + "leftPlan": "TableScan_12", + "rightPlan": "Selection_10" + } +}`, + `{ + "exprs": [ + "k" + ], + "child": "Apply_16" }`, }, }, diff --git a/plan/decorrelate.go b/plan/decorrelate.go index ddc82c7563894..55e4b38f2745a 100644 --- a/plan/decorrelate.go +++ b/plan/decorrelate.go @@ -161,7 +161,7 @@ func (s *decorrelateSolver) optimize(p LogicalPlan, _ context.Context, _ *idAllo } } if sel, ok := p.(*Selection); ok { - if ds, ok := p.(*DataSource); ok { + if ds, ok := p.Children()[0].(*DataSource); ok { sel.canControlScan = sel.hasUsableIndicesAndPk(ds) } } diff --git a/plan/physical_plans.go b/plan/physical_plans.go index 23a02971a618e..73ad70fc79da2 100644 --- a/plan/physical_plans.go +++ b/plan/physical_plans.go @@ -647,7 +647,8 @@ func (p *Selection) MarshalJSON() ([]byte, error) { buffer := bytes.NewBufferString("{") buffer.WriteString(fmt.Sprintf(""+ " \"condition\": %s,\n"+ - " \"child\": \"%s\"\n}", conds, p.children[0].ID())) + " \"scanController\": %v,"+ + " \"child\": \"%s\"\n}", conds, p.ScanController, p.children[0].ID())) return buffer.Bytes(), nil } From 3fd04355a5889ee5ff203aa575503ea83323f6c7 Mon Sep 17 00:00:00 2001 From: Yiding Cui Date: Thu, 23 Mar 2017 15:54:20 +0800 Subject: [PATCH 23/27] remove outdate comment --- expression/util.go | 2 -- 1 file changed, 2 deletions(-) diff --git a/expression/util.go b/expression/util.go index a6c1065ce4253..86a11818438a0 100644 --- a/expression/util.go +++ b/expression/util.go @@ -178,8 +178,6 @@ func (d *distinctChecker) Check(values []interface{}) (bool, error) { // SubstituteCorCol2Constant will substitute correlated column to constant value which it contains. // If the args of one scalar function are all constant, we will substitute it to constant. -// If it's called in plan phase, correlated column will change to expression.One. -// If in executor phase, correlated column will change to value it contains. func SubstituteCorCol2Constant(expr Expression) (Expression, error) { switch x := expr.(type) { case *ScalarFunction: From 8f29d178ca7b9e3fd20b59f5fe6c69df412b8f4f Mon Sep 17 00:00:00 2001 From: Yiding Cui Date: Fri, 24 Mar 2017 16:31:10 +0800 Subject: [PATCH 24/27] add tests and some tiny change --- expression/util.go | 2 -- expression/util_test.go | 21 +++++++++++++ plan/decorrelate.go | 60 ++++++++++++++++++-------------------- plan/physical_plan_test.go | 48 ++++++++++++++++++++++++++++++ 4 files changed, 97 insertions(+), 34 deletions(-) diff --git a/expression/util.go b/expression/util.go index 86a11818438a0..e3bb275947bd3 100644 --- a/expression/util.go +++ b/expression/util.go @@ -208,8 +208,6 @@ func SubstituteCorCol2Constant(expr Expression) (Expression, error) { return newSf, nil case *CorrelatedColumn: return &Constant{Value: *x.Data, RetType: x.GetType()}, nil - case *Constant: - return x.Clone(), nil default: return x.Clone(), nil } diff --git a/expression/util_test.go b/expression/util_test.go index e2beda5e51524..35cb93e9ed0d2 100644 --- a/expression/util_test.go +++ b/expression/util_test.go @@ -15,7 +15,11 @@ package expression import ( "github.com/pingcap/check" + "github.com/pingcap/tidb/ast" + "github.com/pingcap/tidb/mysql" + "github.com/pingcap/tidb/util/mock" "github.com/pingcap/tidb/util/testleak" + "github.com/pingcap/tidb/util/types" ) var _ = check.Suite(&testUtilSuite{}) @@ -43,3 +47,20 @@ func (s *testUtilSuite) TestDistinct(c *check.C) { c.Assert(d, check.Equals, t.expect) } } + +func (s *testUtilSuite) TestSubstituteCorCol2Constant(c *check.C) { + defer testleak.AfterTest(c)() + ctx := mock.NewContext() + corCol1 := &CorrelatedColumn{Data: &One.Value} + corCol2 := &CorrelatedColumn{Data: &One.Value} + cast := NewCastFunc(types.NewFieldType(mysql.TypeLonglong), corCol1, ctx) + plus := newFunction(ast.Plus, cast, corCol2) + plus2 := newFunction(ast.Plus, plus, One) + ans := &Constant{Value: types.NewIntDatum(3)} + ret, err := SubstituteCorCol2Constant(plus2) + c.Assert(err, check.IsNil) + c.Assert(ret.Equal(ans, ctx), check.IsTrue) + col1 := &Column{Index: 1} + newCol, err := SubstituteCorCol2Constant(col1) + c.Assert(newCol.Equal(col1, ctx), check.IsTrue) +} diff --git a/plan/decorrelate.go b/plan/decorrelate.go index 55e4b38f2745a..ebe84b02d27ba 100644 --- a/plan/decorrelate.go +++ b/plan/decorrelate.go @@ -17,7 +17,6 @@ import ( "github.com/pingcap/tidb/ast" "github.com/pingcap/tidb/context" "github.com/pingcap/tidb/expression" - "github.com/pingcap/tidb/model" "github.com/pingcap/tidb/mysql" "github.com/pingcap/tidb/util/types" ) @@ -180,7 +179,6 @@ func (s *decorrelateSolver) optimize(p LogicalPlan, _ context.Context, _ *idAllo // and whether this condition can be used as an access condition. func (p *Selection) hasUsableIndicesAndPk(ds *DataSource) bool { indices, includeTableScan := availableIndices(ds.indexHints, ds.tableInfo) - var usableIdxs []*model.IndexInfo var newConds []expression.Expression for _, expr := range p.Conditions { if !expr.IsCorrelated() { @@ -194,6 +192,32 @@ func (p *Selection) hasUsableIndicesAndPk(ds *DataSource) bool { newCond, _ := expression.SubstituteCorCol2Constant(cond) newConds = append(newConds, newCond) } + if ds.tableInfo.PKIsHandle && includeTableScan { + var pkCol *expression.Column + for i, col := range ds.Columns { + if mysql.HasPriKeyFlag(col.Flag) { + pkCol = ds.schema.Columns[i] + break + } + } + if pkCol != nil { + checker := conditionChecker{ + pkName: pkCol.ColName, + length: types.UnspecifiedLength, + } + // Pk should satisfies the same property as the index. + var usable bool + for _, cond := range newConds { + if checker.check(cond) { + usable = true + break + } + } + if usable { + return true + } + } + } for _, idx := range indices { // TODO: Currently we don't consider composite index. if len(idx.Columns) > 1 { @@ -215,36 +239,8 @@ func (p *Selection) hasUsableIndicesAndPk(ds *DataSource) bool { } } if usable { - usableIdxs = append(usableIdxs, idx) - } - } - var pkName model.CIStr - if ds.tableInfo.PKIsHandle && includeTableScan { - var pkCol *expression.Column - for i, col := range ds.Columns { - if mysql.HasPriKeyFlag(col.Flag) { - pkCol = ds.schema.Columns[i] - break - } - } - if pkCol == nil { - return len(usableIdxs) > 0 - } - checker := conditionChecker{ - pkName: pkCol.ColName, - length: types.UnspecifiedLength, - } - // Pk should satisfies the same property as the index. - var usable bool - for _, cond := range newConds { - if checker.check(cond) { - usable = true - break - } - } - if usable { - pkName = pkCol.ColName + return true } } - return len(usableIdxs) > 0 || pkName.L != "" + return false } diff --git a/plan/physical_plan_test.go b/plan/physical_plan_test.go index eea745295e4a9..6c07c02664210 100644 --- a/plan/physical_plan_test.go +++ b/plan/physical_plan_test.go @@ -1002,3 +1002,51 @@ func (s *testPlanSuite) TestRangeBuilder(c *C) { c.Assert(got, Equals, ca.resultStr, Commentf("different for expr %s", ca.exprStr)) } } + +func (s *testPlanSuite) TestScanController(c *C) { + defer testleak.AfterTest(c)() + cases := []struct { + sql string + ans string + }{ + { + sql: "select (select count(1) k from t s where s.a = t.a having k != 0) from t", + ans: "Apply{Table(t)->Table(t)->Selection->StreamAgg}->Projection", + }, + { + sql: "select (select count(1) k from t s where s.b = t.b having k != 0) from t", + ans: "Apply{Table(t)->Table(t)->Cache->Selection->StreamAgg}->Projection", + }, + } + for _, ca := range cases { + comment := Commentf("for %s", ca.sql) + stmt, err := s.ParseOneStmt(ca.sql, "", "") + c.Assert(err, IsNil, comment) + ast.SetFlag(stmt) + + is, err := mockResolve(stmt) + c.Assert(err, IsNil) + + builder := &planBuilder{ + allocator: new(idAllocator), + ctx: mockContext(), + colMapper: make(map[*ast.ColumnNameExpr]int), + is: is, + } + p := builder.build(stmt) + c.Assert(builder.err, IsNil) + lp := p.(LogicalPlan) + _, lp, err = lp.PredicatePushDown(nil) + c.Assert(err, IsNil) + lp.PruneColumns(lp.Schema().Columns) + dSolver := &decorrelateSolver{} + lp, err = dSolver.optimize(lp, mockContext(), new(idAllocator)) + c.Assert(err, IsNil) + lp.ResolveIndicesAndCorCols() + info, err := lp.convert2PhysicalPlan(&requiredProperty{}) + pp := info.p + pp = EliminateProjection(pp) + addCachePlan(pp, builder.allocator) + c.Assert(ToString(pp), Equals, ca.ans, Commentf("for %s", ca.sql)) + } +} From 088e9e6e48a92a3be9cd5ed281f775ec3ed9a550 Mon Sep 17 00:00:00 2001 From: Yiding Cui Date: Fri, 24 Mar 2017 16:57:24 +0800 Subject: [PATCH 25/27] tiny change --- plan/decorrelate.go | 14 ++------------ 1 file changed, 2 insertions(+), 12 deletions(-) diff --git a/plan/decorrelate.go b/plan/decorrelate.go index ebe84b02d27ba..63137759b8a03 100644 --- a/plan/decorrelate.go +++ b/plan/decorrelate.go @@ -206,16 +206,11 @@ func (p *Selection) hasUsableIndicesAndPk(ds *DataSource) bool { length: types.UnspecifiedLength, } // Pk should satisfies the same property as the index. - var usable bool for _, cond := range newConds { if checker.check(cond) { - usable = true - break + return true } } - if usable { - return true - } } } for _, idx := range indices { @@ -230,17 +225,12 @@ func (p *Selection) hasUsableIndicesAndPk(ds *DataSource) bool { } // This idx column should occur in one condition which contains both column and correlated column. // And conditionChecker.check(this condition) should be true. - var usable bool for _, cond := range newConds { // If one cond is ok, then this index is useful. if checker.check(cond) { - usable = true - break + return true } } - if usable { - return true - } } return false } From 6351036e42892be90c7e9b1727f71e48cd7d93b3 Mon Sep 17 00:00:00 2001 From: Yiding Cui Date: Fri, 24 Mar 2017 19:09:51 +0800 Subject: [PATCH 26/27] add test --- plan/physical_plan_test.go | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/plan/physical_plan_test.go b/plan/physical_plan_test.go index 6c07c02664210..c158fc516cead 100644 --- a/plan/physical_plan_test.go +++ b/plan/physical_plan_test.go @@ -1017,6 +1017,10 @@ func (s *testPlanSuite) TestScanController(c *C) { sql: "select (select count(1) k from t s where s.b = t.b having k != 0) from t", ans: "Apply{Table(t)->Table(t)->Cache->Selection->StreamAgg}->Projection", }, + { + sql: "select (select count(1) k from t s where s.f = t.f having k != 0) from t", + ans: "Apply{Table(t)->Index(t.f)[]->Selection->StreamAgg}->Projection", + }, } for _, ca := range cases { comment := Commentf("for %s", ca.sql) From 97998fcf2d2ba0e5f72dbd2e933077e34ba1b61c Mon Sep 17 00:00:00 2001 From: Yiding Cui Date: Mon, 27 Mar 2017 19:18:19 +0800 Subject: [PATCH 27/27] address comments --- plan/decorrelate.go | 66 ++--------------------------------- plan/physical_plan_builder.go | 56 ++++++++++++++++++++++------- 2 files changed, 46 insertions(+), 76 deletions(-) diff --git a/plan/decorrelate.go b/plan/decorrelate.go index 63137759b8a03..a8914598b1a36 100644 --- a/plan/decorrelate.go +++ b/plan/decorrelate.go @@ -17,7 +17,6 @@ import ( "github.com/pingcap/tidb/ast" "github.com/pingcap/tidb/context" "github.com/pingcap/tidb/expression" - "github.com/pingcap/tidb/mysql" "github.com/pingcap/tidb/util/types" ) @@ -160,8 +159,8 @@ func (s *decorrelateSolver) optimize(p LogicalPlan, _ context.Context, _ *idAllo } } if sel, ok := p.(*Selection); ok { - if ds, ok := p.Children()[0].(*DataSource); ok { - sel.canControlScan = sel.hasUsableIndicesAndPk(ds) + if _, ok := p.Children()[0].(*DataSource); ok { + _, sel.canControlScan = sel.makeScanController(true) } } newChildren := make([]Plan, 0, len(p.Children())) @@ -173,64 +172,3 @@ func (s *decorrelateSolver) optimize(p LogicalPlan, _ context.Context, _ *idAllo p.SetChildren(newChildren...) return p, nil } - -// hasUsableIndicesAndPk will simply check whether the pk or one index could used in this situation by -// checking whether this index or pk is contained in one condition that has correlated column, -// and whether this condition can be used as an access condition. -func (p *Selection) hasUsableIndicesAndPk(ds *DataSource) bool { - indices, includeTableScan := availableIndices(ds.indexHints, ds.tableInfo) - var newConds []expression.Expression - for _, expr := range p.Conditions { - if !expr.IsCorrelated() { - continue - } - cond := pushDownNot(expr, false, nil) - corCols := extractCorColumns(cond) - for _, col := range corCols { - *col.Data = expression.One.Value - } - newCond, _ := expression.SubstituteCorCol2Constant(cond) - newConds = append(newConds, newCond) - } - if ds.tableInfo.PKIsHandle && includeTableScan { - var pkCol *expression.Column - for i, col := range ds.Columns { - if mysql.HasPriKeyFlag(col.Flag) { - pkCol = ds.schema.Columns[i] - break - } - } - if pkCol != nil { - checker := conditionChecker{ - pkName: pkCol.ColName, - length: types.UnspecifiedLength, - } - // Pk should satisfies the same property as the index. - for _, cond := range newConds { - if checker.check(cond) { - return true - } - } - } - } - for _, idx := range indices { - // TODO: Currently we don't consider composite index. - if len(idx.Columns) > 1 { - continue - } - checker := &conditionChecker{ - idx: idx, - columnOffset: 0, - length: idx.Columns[0].Length, - } - // This idx column should occur in one condition which contains both column and correlated column. - // And conditionChecker.check(this condition) should be true. - for _, cond := range newConds { - // If one cond is ok, then this index is useful. - if checker.check(cond) { - return true - } - } - } - return false -} diff --git a/plan/physical_plan_builder.go b/plan/physical_plan_builder.go index 22af1232486cf..5a17b4bc10b81 100644 --- a/plan/physical_plan_builder.go +++ b/plan/physical_plan_builder.go @@ -982,13 +982,29 @@ func (p *Union) convert2PhysicalPlan(prop *requiredProperty) (*physicalPlanInfo, return info, nil } -func (p *Selection) makeScanController() *physicalPlanInfo { - newSel := *p - newSel.ScanController = true - var child PhysicalPlan +// makeScanController will try to build a selection that controls the below scan's filter condition, +// and return a physicalPlanInfo. If the onlyJudge is true, it will only check whether this selection +// can become a scan controller without building the physical plan. +func (p *Selection) makeScanController(onlyJudge bool) (*physicalPlanInfo, bool) { + var ( + child PhysicalPlan + corColConds []expression.Expression + pkCol *expression.Column + ) ds := p.children[0].(*DataSource) indices, includeTableScan := availableIndices(ds.indexHints, ds.tableInfo) - var pkCol *expression.Column + for _, expr := range p.Conditions { + if !expr.IsCorrelated() { + continue + } + cond := pushDownNot(expr, false, nil) + corCols := extractCorColumns(cond) + for _, col := range corCols { + *col.Data = expression.One.Value + } + newCond, _ := expression.SubstituteCorCol2Constant(cond) + corColConds = append(corColConds, newCond) + } if ds.tableInfo.PKIsHandle && includeTableScan { for i, col := range ds.Columns { if mysql.HasPriKeyFlag(col.Flag) { @@ -998,6 +1014,19 @@ func (p *Selection) makeScanController() *physicalPlanInfo { } } if pkCol != nil { + if onlyJudge { + checker := conditionChecker{ + pkName: pkCol.ColName, + length: types.UnspecifiedLength, + } + // Pk should satisfies the same property as the index. + for _, cond := range corColConds { + if checker.check(cond) { + return nil, true + } + } + return nil, false + } ts := &PhysicalTableScan{ Table: ds.tableInfo, Columns: ds.Columns, @@ -1017,7 +1046,6 @@ func (p *Selection) makeScanController() *physicalPlanInfo { child = ts } else { var chosenPlan *PhysicalIndexScan - var corColConds []expression.Expression for _, expr := range p.Conditions { if !expr.IsCorrelated() { continue @@ -1026,10 +1054,6 @@ func (p *Selection) makeScanController() *physicalPlanInfo { corColConds = append(corColConds, cond) } for _, idx := range indices { - // TODO: Currently we don't consider composite index. - if len(idx.Columns) > 1 { - continue - } condsBackUp := make([]expression.Expression, 0, len(corColConds)) for _, cond := range corColConds { condsBackUp = append(condsBackUp, cond.Clone()) @@ -1056,6 +1080,9 @@ func (p *Selection) makeScanController() *physicalPlanInfo { if len(accessConds) == 0 { continue } + if onlyJudge { + return nil, true + } better := chosenPlan == nil || chosenPlan.accessEqualCount < is.accessEqualCount better = better || (chosenPlan.accessEqualCount == is.accessEqualCount && chosenPlan.accessInAndEqCount < is.accessInAndEqCount) if better { @@ -1063,15 +1090,20 @@ func (p *Selection) makeScanController() *physicalPlanInfo { } is.DoubleRead = !isCoveringIndex(is.Columns, is.Index.Columns, is.Table.PKIsHandle) } + if chosenPlan == nil && onlyJudge { + return nil, false + } child = chosenPlan } + newSel := *p + newSel.ScanController = true newSel.SetChildren(child) info := &physicalPlanInfo{ p: &newSel, count: uint64(ds.statisticTable.Count), } info.cost = float64(info.count) * selectionFactor - return info + return info, true } // convert2PhysicalPlan implements the LogicalPlan convert2PhysicalPlan interface. @@ -1084,7 +1116,7 @@ func (p *Selection) convert2PhysicalPlan(prop *requiredProperty) (*physicalPlanI return info, nil } if p.canControlScan { - info = p.makeScanController() + info, _ = p.makeScanController(false) p.storePlanInfo(prop, info) return info, nil }