From e04b3ef6bd7771012d430b4fb0ceefeaa161c2b0 Mon Sep 17 00:00:00 2001 From: lawyerphx Date: Mon, 8 Jun 2020 01:25:27 -0700 Subject: [PATCH 01/37] make find_best_task return the count of plans. --- planner/core/find_best_task.go | 95 ++++++++++++++++++----------- planner/core/find_best_task_test.go | 18 +++--- planner/core/optimizer.go | 2 +- planner/core/plan.go | 2 +- 4 files changed, 70 insertions(+), 47 deletions(-) diff --git a/planner/core/find_best_task.go b/planner/core/find_best_task.go index c0c38f9ae9725..ccc3f1e83772d 100644 --- a/planner/core/find_best_task.go +++ b/planner/core/find_best_task.go @@ -103,49 +103,54 @@ func GetPropByOrderByItemsContainScalarFunc(items []*util.ByItems) (*property.Ph return &property.PhysicalProperty{Items: propItems}, true, onlyColumn } -func (p *LogicalTableDual) findBestTask(prop *property.PhysicalProperty) (task, error) { +func (p *LogicalTableDual) findBestTask(prop *property.PhysicalProperty) (task, int64, error) { // If the required property is not empty and the row count > 1, // we cannot ensure this required property. // But if the row count is 0 or 1, we don't need to care about the property. if !prop.IsEmpty() && p.RowCount > 1 { - return invalidTask, nil + return invalidTask, 0, nil } dual := PhysicalTableDual{ RowCount: p.RowCount, }.Init(p.ctx, p.stats, p.blockOffset) dual.SetSchema(p.schema) - return &rootTask{p: dual}, nil + return &rootTask{p: dual}, 1, nil } -func (p *LogicalShow) findBestTask(prop *property.PhysicalProperty) (task, error) { +func (p *LogicalShow) findBestTask(prop *property.PhysicalProperty) (task, int64, error) { if !prop.IsEmpty() { - return invalidTask, nil + return invalidTask, 0, nil } pShow := PhysicalShow{ShowContents: p.ShowContents}.Init(p.ctx) pShow.SetSchema(p.schema) - return &rootTask{p: pShow}, nil + return &rootTask{p: pShow}, 1, nil } -func (p *LogicalShowDDLJobs) findBestTask(prop *property.PhysicalProperty) (task, error) { +func (p *LogicalShowDDLJobs) findBestTask(prop *property.PhysicalProperty) (task, int64, error) { if !prop.IsEmpty() { - return invalidTask, nil + return invalidTask, 0, nil } pShow := PhysicalShowDDLJobs{JobNumber: p.JobNumber}.Init(p.ctx) pShow.SetSchema(p.schema) - return &rootTask{p: pShow}, nil + return &rootTask{p: pShow}, 1, nil } -func (p *baseLogicalPlan) enumeratePhysicalPlans4Task(physicalPlans []PhysicalPlan, prop *property.PhysicalProperty) (task, error) { +func (p *baseLogicalPlan) enumeratePhysicalPlans4Task(physicalPlans []PhysicalPlan, prop *property.PhysicalProperty) (task, int64, error) { var bestTask task = invalidTask + var curCntPlan int64 = 1 + var cntPlan int64 = 0 childTasks := make([]task, 0, len(p.children)) for _, pp := range physicalPlans { // find best child tasks firstly. childTasks = childTasks[:0] - for i, child := range p.children { - childTask, err := child.findBestTask(pp.GetChildReqProps(i)) + // curCntPlan record the number of possible plan for pp + curCntPlan = 1 + for j, child := range p.children { + childTask, cnt, err := child.findBestTask(pp.GetChildReqProps(j)) if err != nil { - return nil, err + return nil, 0, err } + curCntPlan = curCntPlan * cnt if childTask != nil && childTask.invalid() { break } @@ -160,45 +165,49 @@ func (p *baseLogicalPlan) enumeratePhysicalPlans4Task(physicalPlans []PhysicalPl // combine best child tasks with parent physical plan. curTask := pp.attach2Task(childTasks...) - // enforce curTask property + // enforce curTask property, set curCntPlan to 1 since the Task is enforced. if prop.Enforced { curTask = enforceProperty(prop, curTask, p.basePlan.ctx) + curCntPlan = 1 } // optimize by shuffle executor to running in parallel manner. if prop.IsEmpty() { + // TODO: using SQL hint `use_plan(x)` should avoid random here. curTask = optimizeByShuffle(pp, curTask, p.basePlan.ctx) } // get the most efficient one. + cntPlan += curCntPlan if curTask.cost() < bestTask.cost() || (bestTask.invalid() && !curTask.invalid()) { bestTask = curTask } } - return bestTask, nil + return bestTask, cntPlan, nil } // findBestTask implements LogicalPlan interface. -func (p *baseLogicalPlan) findBestTask(prop *property.PhysicalProperty) (bestTask task, err error) { +func (p *baseLogicalPlan) findBestTask(prop *property.PhysicalProperty) (bestTask task, cntPlan int64, err error) { // If p is an inner plan in an IndexJoin, the IndexJoin will generate an inner plan by itself, // and set inner child prop nil, so here we do nothing. if prop == nil { - return nil, nil + return nil, 0, nil } // Look up the task with this prop in the task map. // It's used to reduce double counting. bestTask = p.getTask(prop) if bestTask != nil { - return bestTask, nil + return bestTask, 1, nil } if prop.TaskTp != property.RootTaskType { // Currently all plan cannot totally push down. p.storeTask(prop, invalidTask) - return invalidTask, nil + return invalidTask, 0, nil } bestTask = invalidTask + cntPlan = 0 // prop should be read only because its cached hashcode might be not consistent // when it is changed. So we clone a new one for the temporary changes. newProp := prop.Clone() @@ -207,6 +216,7 @@ func (p *baseLogicalPlan) findBestTask(prop *property.PhysicalProperty) (bestTas var hintWorksWithProp bool // Maybe the plan can satisfy the required property, // so we try to get the task without the enforced sort first. + // TODO: from PHX will it introduce some random? plansFitsProp, hintWorksWithProp = p.self.exhaustPhysicalPlans(newProp) if !hintWorksWithProp && !newProp.IsEmpty() { // If there is a hint in the plan and the hint cannot satisfy the property, @@ -238,25 +248,29 @@ func (p *baseLogicalPlan) findBestTask(prop *property.PhysicalProperty) (bestTas } newProp.Enforced = false - if bestTask, err = p.enumeratePhysicalPlans4Task(plansFitsProp, newProp); err != nil { - return nil, err + var cnt int64 = 0 + if bestTask, cnt, err = p.enumeratePhysicalPlans4Task(plansFitsProp, newProp); err != nil { + return nil, 0, err } + cntPlan += cnt + cnt = 0 newProp.Enforced = true - curTask, err := p.enumeratePhysicalPlans4Task(plansNeedEnforce, newProp) + curTask, cnt, err := p.enumeratePhysicalPlans4Task(plansNeedEnforce, newProp) if err != nil { - return nil, err + return nil, 0, err } + cntPlan += cnt if curTask.cost() < bestTask.cost() || (bestTask.invalid() && !curTask.invalid()) { bestTask = curTask } p.storeTask(prop, bestTask) - return bestTask, nil + return bestTask, cntPlan, nil } -func (p *LogicalMemTable) findBestTask(prop *property.PhysicalProperty) (t task, err error) { +func (p *LogicalMemTable) findBestTask(prop *property.PhysicalProperty) (t task, cntPlan int64, err error) { if !prop.IsEmpty() { - return invalidTask, nil + return invalidTask, 0, nil } memTable := PhysicalMemTable{ DBName: p.DBName, @@ -266,7 +280,7 @@ func (p *LogicalMemTable) findBestTask(prop *property.PhysicalProperty) (t task, QueryTimeRange: p.QueryTimeRange, }.Init(p.ctx, p.stats, p.blockOffset) memTable.SetSchema(p.schema) - return &rootTask{p: memTable}, nil + return &rootTask{p: memTable}, 1, nil } // tryToGetDualTask will check if the push down predicate has false constant. If so, it will return table dual. @@ -441,30 +455,33 @@ func (ds *DataSource) skylinePruning(prop *property.PhysicalProperty) []*candida // findBestTask implements the PhysicalPlan interface. // It will enumerate all the available indices and choose a plan with least cost. -func (ds *DataSource) findBestTask(prop *property.PhysicalProperty) (t task, err error) { +func (ds *DataSource) findBestTask(prop *property.PhysicalProperty) (t task, cntPlan int64, err error) { // If ds is an inner plan in an IndexJoin, the IndexJoin will generate an inner plan by itself, // and set inner child prop nil, so here we do nothing. if prop == nil { - return nil, nil + return nil, 0, nil } t = ds.getTask(prop) if t != nil { + cntPlan = 1 return } + var cnt int64 = 0 // If prop.enforced is true, the prop.cols need to be set nil for ds.findBestTask. // Before function return, reset it for enforcing task prop and storing map. oldPropCols := prop.Items if prop.Enforced { // First, get the bestTask without enforced prop prop.Enforced = false - t, err = ds.findBestTask(prop) + t, cnt, err = ds.findBestTask(prop) if err != nil { - return nil, err + return nil, 0, err } prop.Enforced = true if t != invalidTask { ds.storeTask(prop, t) + cntPlan = cnt return } // Next, get the bestTask with enforced prop @@ -483,19 +500,21 @@ func (ds *DataSource) findBestTask(prop *property.PhysicalProperty) (t task, err t, err = ds.tryToGetDualTask() if err != nil || t != nil { - return t, err + return t, 1, err } t = invalidTask candidates := ds.skylinePruning(prop) + cntPlan = 0 for _, candidate := range candidates { path := candidate.path if path.PartialIndexPaths != nil { idxMergeTask, err := ds.convertToIndexMergeScan(prop, candidate) if err != nil { - return nil, err + return nil, 0, err } + cntPlan += 1 if idxMergeTask.cost() < t.cost() { t = idxMergeTask } @@ -505,9 +524,10 @@ func (ds *DataSource) findBestTask(prop *property.PhysicalProperty) (t task, err if len(path.Ranges) == 0 && !ds.ctx.GetSessionVars().StmtCtx.UseCache { dual := PhysicalTableDual{}.Init(ds.ctx, ds.stats, ds.blockOffset) dual.SetSchema(ds.schema) + cntPlan += 1 return &rootTask{ p: dual, - }, nil + }, cntPlan, nil } canConvertPointGet := (!ds.isPartition && len(path.Ranges) > 0) || (ds.isPartition && len(path.Ranges) == 1) canConvertPointGet = canConvertPointGet && candidate.path.StoreType != kv.TiFlash @@ -532,6 +552,7 @@ func (ds *DataSource) findBestTask(prop *property.PhysicalProperty) (t task, err } else { pointGetTask = ds.convertToBatchPointGet(prop, candidate) } + cntPlan += 1 if pointGetTask.cost() < t.cost() { t = pointGetTask continue @@ -547,8 +568,9 @@ func (ds *DataSource) findBestTask(prop *property.PhysicalProperty) (t task, err } tblTask, err := ds.convertToTableScan(prop, candidate) if err != nil { - return nil, err + return nil, 0, err } + cntPlan += 1 if tblTask.cost() < t.cost() { t = tblTask } @@ -560,8 +582,9 @@ func (ds *DataSource) findBestTask(prop *property.PhysicalProperty) (t task, err } idxTask, err := ds.convertToIndexScan(prop, candidate) if err != nil { - return nil, err + return nil, 0, err } + cntPlan += 1 if idxTask.cost() < t.cost() { t = idxTask } diff --git a/planner/core/find_best_task_test.go b/planner/core/find_best_task_test.go index 701376b035c17..bf082d3882be5 100644 --- a/planner/core/find_best_task_test.go +++ b/planner/core/find_best_task_test.go @@ -42,7 +42,7 @@ func (ds mockDataSource) Init(ctx sessionctx.Context) *mockDataSource { return &ds } -func (ds *mockDataSource) findBestTask(prop *property.PhysicalProperty) (task, error) { +func (ds *mockDataSource) findBestTask(prop *property.PhysicalProperty) (task, int64, error) { // It can satisfy any of the property! // Just use a TableDual for convenience. p := PhysicalTableDual{}.Init(ds.ctx, &property.StatsInfo{RowCount: 1}, 0) @@ -50,7 +50,7 @@ func (ds *mockDataSource) findBestTask(prop *property.PhysicalProperty) (task, e p: p, cst: 10000, } - return task, nil + return task, 1, nil } // mockLogicalPlan4Test is a LogicalPlan which is used for unit test. @@ -151,7 +151,7 @@ func (s *testFindBestTaskSuite) TestCostOverflow(c *C) { mockPlan.SetChildren(mockDS) // An empty property is enough for this test. prop := property.NewPhysicalProperty(property.RootTaskType, nil, false, 0, false) - t, err := mockPlan.findBestTask(prop) + t, _, err := mockPlan.findBestTask(prop) c.Assert(err, IsNil) // The cost should be overflowed, but the task shouldn't be invalid. c.Assert(t.invalid(), IsFalse) @@ -178,7 +178,7 @@ func (s *testFindBestTaskSuite) TestEnforcedProperty(c *C) { Enforced: false, } // should return invalid task because no physical plan can match this property. - task, err := mockPlan.findBestTask(prop0) + task, _, err := mockPlan.findBestTask(prop0) c.Assert(err, IsNil) c.Assert(task.invalid(), IsTrue) @@ -187,7 +187,7 @@ func (s *testFindBestTaskSuite) TestEnforcedProperty(c *C) { Enforced: true, } // should return the valid task when the property is enforced. - task, err = mockPlan.findBestTask(prop1) + task, _, err = mockPlan.findBestTask(prop1) c.Assert(err, IsNil) c.Assert(task.invalid(), IsFalse) } @@ -210,7 +210,7 @@ func (s *testFindBestTaskSuite) TestHintCannotFitProperty(c *C) { Items: items, Enforced: true, } - task, err := mockPlan0.findBestTask(prop0) + task, _, err := mockPlan0.findBestTask(prop0) c.Assert(err, IsNil) c.Assert(task.invalid(), IsFalse) _, enforcedSort := task.plan().(*PhysicalSort) @@ -226,7 +226,7 @@ func (s *testFindBestTaskSuite) TestHintCannotFitProperty(c *C) { Items: items, Enforced: false, } - task, err = mockPlan0.findBestTask(prop1) + task, _, err = mockPlan0.findBestTask(prop1) c.Assert(err, IsNil) c.Assert(task.invalid(), IsFalse) _, enforcedSort = task.plan().(*PhysicalSort) @@ -247,7 +247,7 @@ func (s *testFindBestTaskSuite) TestHintCannotFitProperty(c *C) { canGeneratePlan2: false, }.Init(ctx) mockPlan1.SetChildren(mockDS) - task, err = mockPlan1.findBestTask(prop2) + task, _, err = mockPlan1.findBestTask(prop2) c.Assert(err, IsNil) c.Assert(task.invalid(), IsFalse) c.Assert(ctx.GetSessionVars().StmtCtx.WarningCount(), Equals, uint16(1)) @@ -263,7 +263,7 @@ func (s *testFindBestTaskSuite) TestHintCannotFitProperty(c *C) { Items: items, Enforced: true, } - task, err = mockPlan1.findBestTask(prop3) + task, _, err = mockPlan1.findBestTask(prop3) c.Assert(err, IsNil) c.Assert(task.invalid(), IsFalse) c.Assert(ctx.GetSessionVars().StmtCtx.WarningCount(), Equals, uint16(1)) diff --git a/planner/core/optimizer.go b/planner/core/optimizer.go index da394fcf62cba..8c5e88aae9e1c 100644 --- a/planner/core/optimizer.go +++ b/planner/core/optimizer.go @@ -181,7 +181,7 @@ func physicalOptimize(logic LogicalPlan) (PhysicalPlan, float64, error) { ExpectedCnt: math.MaxFloat64, } - t, err := logic.findBestTask(prop) + t, _, err := logic.findBestTask(prop) if err != nil { return nil, 0, err } diff --git a/planner/core/plan.go b/planner/core/plan.go index b4a70d6dc7767..944385a0d4864 100644 --- a/planner/core/plan.go +++ b/planner/core/plan.go @@ -154,7 +154,7 @@ type LogicalPlan interface { // It is called recursively from the parent to the children to create the result physical plan. // Some logical plans will convert the children to the physical plans in different ways, and return the one // with the lowest cost. - findBestTask(prop *property.PhysicalProperty) (task, error) + findBestTask(prop *property.PhysicalProperty) (task, int64, error) // BuildKeyInfo will collect the information of unique keys into schema. // Because this method is also used in cascades planner, we cannot use From c4bfa1f257c4022c7e23f07de52a7e367f6411df Mon Sep 17 00:00:00 2001 From: lawyerphx Date: Mon, 8 Jun 2020 01:26:15 -0700 Subject: [PATCH 02/37] remove dup TODO --- planner/core/find_best_task.go | 1 - 1 file changed, 1 deletion(-) diff --git a/planner/core/find_best_task.go b/planner/core/find_best_task.go index ccc3f1e83772d..9b96ee2c6626e 100644 --- a/planner/core/find_best_task.go +++ b/planner/core/find_best_task.go @@ -216,7 +216,6 @@ func (p *baseLogicalPlan) findBestTask(prop *property.PhysicalProperty) (bestTas var hintWorksWithProp bool // Maybe the plan can satisfy the required property, // so we try to get the task without the enforced sort first. - // TODO: from PHX will it introduce some random? plansFitsProp, hintWorksWithProp = p.self.exhaustPhysicalPlans(newProp) if !hintWorksWithProp && !newProp.IsEmpty() { // If there is a hint in the plan and the hint cannot satisfy the property, From 72a566f9b0b351ffd530feed1c6fbcb1e660e92a Mon Sep 17 00:00:00 2001 From: lawyerphx Date: Mon, 8 Jun 2020 01:35:29 -0700 Subject: [PATCH 03/37] make CI happy --- planner/core/find_best_task.go | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/planner/core/find_best_task.go b/planner/core/find_best_task.go index 9b96ee2c6626e..29d2f9071e149 100644 --- a/planner/core/find_best_task.go +++ b/planner/core/find_best_task.go @@ -137,7 +137,7 @@ func (p *LogicalShowDDLJobs) findBestTask(prop *property.PhysicalProperty) (task func (p *baseLogicalPlan) enumeratePhysicalPlans4Task(physicalPlans []PhysicalPlan, prop *property.PhysicalProperty) (task, int64, error) { var bestTask task = invalidTask - var curCntPlan int64 = 1 + var curCntPlan int64 var cntPlan int64 = 0 childTasks := make([]task, 0, len(p.children)) for _, pp := range physicalPlans { @@ -247,12 +247,11 @@ func (p *baseLogicalPlan) findBestTask(prop *property.PhysicalProperty) (bestTas } newProp.Enforced = false - var cnt int64 = 0 + var cnt int64 if bestTask, cnt, err = p.enumeratePhysicalPlans4Task(plansFitsProp, newProp); err != nil { return nil, 0, err } cntPlan += cnt - cnt = 0 newProp.Enforced = true curTask, cnt, err := p.enumeratePhysicalPlans4Task(plansNeedEnforce, newProp) if err != nil { @@ -466,7 +465,7 @@ func (ds *DataSource) findBestTask(prop *property.PhysicalProperty) (t task, cnt cntPlan = 1 return } - var cnt int64 = 0 + var cnt int64 // If prop.enforced is true, the prop.cols need to be set nil for ds.findBestTask. // Before function return, reset it for enforcing task prop and storing map. oldPropCols := prop.Items From 657b97fe048b75c4e548a481f56ba4416d7a7b12 Mon Sep 17 00:00:00 2001 From: lawyerphx Date: Mon, 8 Jun 2020 01:58:45 -0700 Subject: [PATCH 04/37] handle shuffled plan. --- planner/core/find_best_task.go | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/planner/core/find_best_task.go b/planner/core/find_best_task.go index 29d2f9071e149..fc67bdb55ac02 100644 --- a/planner/core/find_best_task.go +++ b/planner/core/find_best_task.go @@ -173,8 +173,9 @@ func (p *baseLogicalPlan) enumeratePhysicalPlans4Task(physicalPlans []PhysicalPl // optimize by shuffle executor to running in parallel manner. if prop.IsEmpty() { - // TODO: using SQL hint `use_plan(x)` should avoid random here. + // currently, we regard shuffled plan as a new plan. curTask = optimizeByShuffle(pp, curTask, p.basePlan.ctx) + curCntPlan += 1 } // get the most efficient one. From 9d4af5ac2c5f89937175e876c15eb279aa36f14e Mon Sep 17 00:00:00 2001 From: lawyerphx Date: Tue, 9 Jun 2020 02:19:39 -0700 Subject: [PATCH 05/37] enable countdown. --- planner/core/find_best_task.go | 149 +++++++++++++++++++++++++++------ planner/core/optimizer.go | 2 +- planner/core/plan.go | 7 +- 3 files changed, 129 insertions(+), 29 deletions(-) diff --git a/planner/core/find_best_task.go b/planner/core/find_best_task.go index fc67bdb55ac02..3df30deada4aa 100644 --- a/planner/core/find_best_task.go +++ b/planner/core/find_best_task.go @@ -60,6 +60,33 @@ var aggFuncFactor = map[string]float64{ "default": 1.5, } +// CountDown is used in hint nth_plan() to indicate which plan to use. +type CountDown int8 + +// Sign is the default value of CountDown, indicating that optimizer needn't to force a plan. +var Sign CountDown = -1 + +// Dec minus countdown by x +func (c *CountDown) Dec(x int8) { + if *c <= 0 { + return + } + *c = CountDown(int8(*c) - x) + if *c < 0 { + *c = 0 + } +} + +// Empty indicates which the CountDown is clear now. +func (c *CountDown) Empty() bool { + return *c == 0 +} + +// IsForce indicates whether to force a plan. +func (c *CountDown) IsForce() bool { + return *c != -1 +} + // wholeTaskTypes records all possible kinds of task that a plan can return. For Agg, TopN and Limit, we will try to get // these tasks one by one. var wholeTaskTypes = [...]property.TaskType{property.CopSingleReadTaskType, property.CopDoubleReadTaskType, property.RootTaskType} @@ -103,11 +130,11 @@ func GetPropByOrderByItemsContainScalarFunc(items []*util.ByItems) (*property.Ph return &property.PhysicalProperty{Items: propItems}, true, onlyColumn } -func (p *LogicalTableDual) findBestTask(prop *property.PhysicalProperty) (task, int64, error) { +func (p *LogicalTableDual) findBestTask(prop *property.PhysicalProperty, clock *CountDown) (task, int64, error) { // If the required property is not empty and the row count > 1, // we cannot ensure this required property. // But if the row count is 0 or 1, we don't need to care about the property. - if !prop.IsEmpty() && p.RowCount > 1 { + if (!prop.IsEmpty() && p.RowCount > 1) || *clock == 0 { return invalidTask, 0, nil } dual := PhysicalTableDual{ @@ -117,8 +144,8 @@ func (p *LogicalTableDual) findBestTask(prop *property.PhysicalProperty) (task, return &rootTask{p: dual}, 1, nil } -func (p *LogicalShow) findBestTask(prop *property.PhysicalProperty) (task, int64, error) { - if !prop.IsEmpty() { +func (p *LogicalShow) findBestTask(prop *property.PhysicalProperty, clock *CountDown) (task, int64, error) { + if !prop.IsEmpty() || clock.Empty() { return invalidTask, 0, nil } pShow := PhysicalShow{ShowContents: p.ShowContents}.Init(p.ctx) @@ -126,8 +153,8 @@ func (p *LogicalShow) findBestTask(prop *property.PhysicalProperty) (task, int64 return &rootTask{p: pShow}, 1, nil } -func (p *LogicalShowDDLJobs) findBestTask(prop *property.PhysicalProperty) (task, int64, error) { - if !prop.IsEmpty() { +func (p *LogicalShowDDLJobs) findBestTask(prop *property.PhysicalProperty, clock *CountDown) (task, int64, error) { + if !prop.IsEmpty() || clock.Empty() { return invalidTask, 0, nil } pShow := PhysicalShowDDLJobs{JobNumber: p.JobNumber}.Init(p.ctx) @@ -135,18 +162,42 @@ func (p *LogicalShowDDLJobs) findBestTask(prop *property.PhysicalProperty) (task return &rootTask{p: pShow}, 1, nil } -func (p *baseLogicalPlan) enumeratePhysicalPlans4Task(physicalPlans []PhysicalPlan, prop *property.PhysicalProperty) (task, int64, error) { +// rebuildChildTasks rebuild the childTasks to make the clock_th combination. +func (p *baseLogicalPlan) rebuildChildTasks(childTasks *[]task, pp PhysicalPlan, childCnts []int64, clock int8) error { + var multAll int8 = 1 + var curClock CountDown + for _, x := range childCnts { + multAll *= int8(x) + } + for j, child := range p.children { + multAll /= int8(childCnts[j]) + curClock = CountDown((clock-1)/multAll + 1) + childTask, _, err := child.findBestTask(pp.GetChildReqProps(j), &curClock) + clock = (clock-1)%multAll + 1 + // y.Assert(curClock == 0) + if err != nil { + return err + } + // y.Assert(!(childTask != nil && childTask.invalid())) + *childTasks = append(*childTasks, childTask) + } + return nil +} + +func (p *baseLogicalPlan) enumeratePhysicalPlans4Task(physicalPlans []PhysicalPlan, prop *property.PhysicalProperty, clock *CountDown) (task, int64, error) { var bestTask task = invalidTask var curCntPlan int64 var cntPlan int64 = 0 childTasks := make([]task, 0, len(p.children)) + childCnts := make([]int64, 0, len(p.children)) for _, pp := range physicalPlans { // find best child tasks firstly. childTasks = childTasks[:0] // curCntPlan record the number of possible plan for pp curCntPlan = 1 for j, child := range p.children { - childTask, cnt, err := child.findBestTask(pp.GetChildReqProps(j)) + childTask, cnt, err := child.findBestTask(pp.GetChildReqProps(j), &Sign) + childCnts[j] = cnt if err != nil { return nil, 0, err } @@ -162,24 +213,37 @@ func (p *baseLogicalPlan) enumeratePhysicalPlans4Task(physicalPlans []PhysicalPl continue } + if clock.IsForce() && int64(*clock) <= curCntPlan { + curCntPlan = int64(*clock) + err := p.rebuildChildTasks(&childTasks, pp, childCnts, int8(*clock)) + if err != nil { + return nil, 0, err + } + } + // combine best child tasks with parent physical plan. curTask := pp.attach2Task(childTasks...) - // enforce curTask property, set curCntPlan to 1 since the Task is enforced. + // enforce curTask property if prop.Enforced { curTask = enforceProperty(prop, curTask, p.basePlan.ctx) - curCntPlan = 1 } // optimize by shuffle executor to running in parallel manner. if prop.IsEmpty() { - // currently, we regard shuffled plan as a new plan. + // currently, we don not regard shuffled plan as a new plan. curTask = optimizeByShuffle(pp, curTask, p.basePlan.ctx) - curCntPlan += 1 } // get the most efficient one. cntPlan += curCntPlan + clock.Dec(int8(curCntPlan)) + + if clock.Empty() { + bestTask = curTask + break + } + if curTask.cost() < bestTask.cost() || (bestTask.invalid() && !curTask.invalid()) { bestTask = curTask } @@ -188,7 +252,7 @@ func (p *baseLogicalPlan) enumeratePhysicalPlans4Task(physicalPlans []PhysicalPl } // findBestTask implements LogicalPlan interface. -func (p *baseLogicalPlan) findBestTask(prop *property.PhysicalProperty) (bestTask task, cntPlan int64, err error) { +func (p *baseLogicalPlan) findBestTask(prop *property.PhysicalProperty, clock *CountDown) (bestTask task, cntPlan int64, err error) { // If p is an inner plan in an IndexJoin, the IndexJoin will generate an inner plan by itself, // and set inner child prop nil, so here we do nothing. if prop == nil { @@ -198,6 +262,7 @@ func (p *baseLogicalPlan) findBestTask(prop *property.PhysicalProperty) (bestTas // It's used to reduce double counting. bestTask = p.getTask(prop) if bestTask != nil { + clock.Dec(1) return bestTask, 1, nil } @@ -249,26 +314,36 @@ func (p *baseLogicalPlan) findBestTask(prop *property.PhysicalProperty) (bestTas newProp.Enforced = false var cnt int64 - if bestTask, cnt, err = p.enumeratePhysicalPlans4Task(plansFitsProp, newProp); err != nil { + var curTask task + if bestTask, cnt, err = p.enumeratePhysicalPlans4Task(plansFitsProp, newProp, clock); err != nil { return nil, 0, err } cntPlan += cnt + if clock.Empty() { + goto END + } + newProp.Enforced = true - curTask, cnt, err := p.enumeratePhysicalPlans4Task(plansNeedEnforce, newProp) + curTask, cnt, err = p.enumeratePhysicalPlans4Task(plansNeedEnforce, newProp, clock) if err != nil { return nil, 0, err } cntPlan += cnt + if clock.Empty() { + bestTask = curTask + goto END + } if curTask.cost() < bestTask.cost() || (bestTask.invalid() && !curTask.invalid()) { bestTask = curTask } +END: p.storeTask(prop, bestTask) return bestTask, cntPlan, nil } -func (p *LogicalMemTable) findBestTask(prop *property.PhysicalProperty) (t task, cntPlan int64, err error) { - if !prop.IsEmpty() { +func (p *LogicalMemTable) findBestTask(prop *property.PhysicalProperty, clock *CountDown) (t task, cntPlan int64, err error) { + if !prop.IsEmpty() || clock.Empty() { return invalidTask, 0, nil } memTable := PhysicalMemTable{ @@ -454,7 +529,7 @@ func (ds *DataSource) skylinePruning(prop *property.PhysicalProperty) []*candida // findBestTask implements the PhysicalPlan interface. // It will enumerate all the available indices and choose a plan with least cost. -func (ds *DataSource) findBestTask(prop *property.PhysicalProperty) (t task, cntPlan int64, err error) { +func (ds *DataSource) findBestTask(prop *property.PhysicalProperty, clock *CountDown) (t task, cntPlan int64, err error) { // If ds is an inner plan in an IndexJoin, the IndexJoin will generate an inner plan by itself, // and set inner child prop nil, so here we do nothing. if prop == nil { @@ -464,6 +539,7 @@ func (ds *DataSource) findBestTask(prop *property.PhysicalProperty) (t task, cnt t = ds.getTask(prop) if t != nil { cntPlan = 1 + clock.Dec(1) return } var cnt int64 @@ -473,7 +549,7 @@ func (ds *DataSource) findBestTask(prop *property.PhysicalProperty) (t task, cnt if prop.Enforced { // First, get the bestTask without enforced prop prop.Enforced = false - t, cnt, err = ds.findBestTask(prop) + t, cnt, err = ds.findBestTask(prop, clock) if err != nil { return nil, 0, err } @@ -499,6 +575,7 @@ func (ds *DataSource) findBestTask(prop *property.PhysicalProperty) (t task, cnt t, err = ds.tryToGetDualTask() if err != nil || t != nil { + clock.Dec(1) return t, 1, err } @@ -514,16 +591,22 @@ func (ds *DataSource) findBestTask(prop *property.PhysicalProperty) (t task, cnt return nil, 0, err } cntPlan += 1 - if idxMergeTask.cost() < t.cost() { + clock.Dec(1) + if idxMergeTask.cost() < t.cost() || clock.Empty() { t = idxMergeTask } - continue + if clock.Empty() { + return t, cntPlan, nil + } else { + continue + } } // if we already know the range of the scan is empty, just return a TableDual if len(path.Ranges) == 0 && !ds.ctx.GetSessionVars().StmtCtx.UseCache { dual := PhysicalTableDual{}.Init(ds.ctx, ds.stats, ds.blockOffset) dual.SetSchema(ds.schema) cntPlan += 1 + clock.Dec(1) return &rootTask{ p: dual, }, cntPlan, nil @@ -552,9 +635,14 @@ func (ds *DataSource) findBestTask(prop *property.PhysicalProperty) (t task, cnt pointGetTask = ds.convertToBatchPointGet(prop, candidate) } cntPlan += 1 - if pointGetTask.cost() < t.cost() { + clock.Dec(1) + if pointGetTask.cost() < t.cost() || clock.Empty() { t = pointGetTask - continue + if clock.Empty() { + return + } else { + continue + } } } } @@ -570,10 +658,15 @@ func (ds *DataSource) findBestTask(prop *property.PhysicalProperty) (t task, cnt return nil, 0, err } cntPlan += 1 - if tblTask.cost() < t.cost() { + clock.Dec(1) + if tblTask.cost() < t.cost() || clock.Empty() { t = tblTask } - continue + if clock.Empty() { + return t, cntPlan, nil + } else { + continue + } } // TiFlash storage do not support index scan. if ds.preferStoreType&preferTiFlash != 0 { @@ -584,9 +677,13 @@ func (ds *DataSource) findBestTask(prop *property.PhysicalProperty) (t task, cnt return nil, 0, err } cntPlan += 1 - if idxTask.cost() < t.cost() { + clock.Dec(1) + if idxTask.cost() < t.cost() || clock.Empty() { t = idxTask } + if clock.Empty() { + return t, cntPlan, nil + } } return diff --git a/planner/core/optimizer.go b/planner/core/optimizer.go index 8c5e88aae9e1c..510859f18b7d7 100644 --- a/planner/core/optimizer.go +++ b/planner/core/optimizer.go @@ -181,7 +181,7 @@ func physicalOptimize(logic LogicalPlan) (PhysicalPlan, float64, error) { ExpectedCnt: math.MaxFloat64, } - t, _, err := logic.findBestTask(prop) + t, _, err := logic.findBestTask(prop, &Sign) if err != nil { return nil, 0, err } diff --git a/planner/core/plan.go b/planner/core/plan.go index 944385a0d4864..361a2d7fd4d95 100644 --- a/planner/core/plan.go +++ b/planner/core/plan.go @@ -153,8 +153,11 @@ type LogicalPlan interface { // findBestTask converts the logical plan to the physical plan. It's a new interface. // It is called recursively from the parent to the children to create the result physical plan. // Some logical plans will convert the children to the physical plans in different ways, and return the one - // with the lowest cost. - findBestTask(prop *property.PhysicalProperty) (task, int64, error) + // with the lowest cost and how many plans are found in this function. + // clock is a counter for planner to force a plan. + // if clock = 0, the plan generated in this function will not be considered. + // if clock = -1, then we will not force plan. + findBestTask(prop *property.PhysicalProperty, clock *CountDown) (task, int64, error) // BuildKeyInfo will collect the information of unique keys into schema. // Because this method is also used in cascades planner, we cannot use From e5b7a7be105106765bc5ee1dfce290ed26b5af15 Mon Sep 17 00:00:00 2001 From: lawyerphx Date: Tue, 9 Jun 2020 19:33:46 -0700 Subject: [PATCH 06/37] make CI happy. --- planner/core/find_best_task.go | 9 +++------ 1 file changed, 3 insertions(+), 6 deletions(-) diff --git a/planner/core/find_best_task.go b/planner/core/find_best_task.go index 3df30deada4aa..37f9eed8c17a1 100644 --- a/planner/core/find_best_task.go +++ b/planner/core/find_best_task.go @@ -597,9 +597,8 @@ func (ds *DataSource) findBestTask(prop *property.PhysicalProperty, clock *Count } if clock.Empty() { return t, cntPlan, nil - } else { - continue } + continue } // if we already know the range of the scan is empty, just return a TableDual if len(path.Ranges) == 0 && !ds.ctx.GetSessionVars().StmtCtx.UseCache { @@ -640,9 +639,8 @@ func (ds *DataSource) findBestTask(prop *property.PhysicalProperty, clock *Count t = pointGetTask if clock.Empty() { return - } else { - continue } + continue } } } @@ -664,9 +662,8 @@ func (ds *DataSource) findBestTask(prop *property.PhysicalProperty, clock *Count } if clock.Empty() { return t, cntPlan, nil - } else { - continue } + continue } // TiFlash storage do not support index scan. if ds.preferStoreType&preferTiFlash != 0 { From 82855bd2f29978d01ea7e2e89d12a3529990e8b2 Mon Sep 17 00:00:00 2001 From: lawyerphx Date: Tue, 9 Jun 2020 19:41:09 -0700 Subject: [PATCH 07/37] fix unit tests. --- planner/core/find_best_task_test.go | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/planner/core/find_best_task_test.go b/planner/core/find_best_task_test.go index bf082d3882be5..f80cc8eafcb77 100644 --- a/planner/core/find_best_task_test.go +++ b/planner/core/find_best_task_test.go @@ -42,7 +42,7 @@ func (ds mockDataSource) Init(ctx sessionctx.Context) *mockDataSource { return &ds } -func (ds *mockDataSource) findBestTask(prop *property.PhysicalProperty) (task, int64, error) { +func (ds *mockDataSource) findBestTask(prop *property.PhysicalProperty, clock *CountDown) (task, int64, error) { // It can satisfy any of the property! // Just use a TableDual for convenience. p := PhysicalTableDual{}.Init(ds.ctx, &property.StatsInfo{RowCount: 1}, 0) @@ -151,7 +151,7 @@ func (s *testFindBestTaskSuite) TestCostOverflow(c *C) { mockPlan.SetChildren(mockDS) // An empty property is enough for this test. prop := property.NewPhysicalProperty(property.RootTaskType, nil, false, 0, false) - t, _, err := mockPlan.findBestTask(prop) + t, _, err := mockPlan.findBestTask(prop, &Sign) c.Assert(err, IsNil) // The cost should be overflowed, but the task shouldn't be invalid. c.Assert(t.invalid(), IsFalse) @@ -178,7 +178,7 @@ func (s *testFindBestTaskSuite) TestEnforcedProperty(c *C) { Enforced: false, } // should return invalid task because no physical plan can match this property. - task, _, err := mockPlan.findBestTask(prop0) + task, _, err := mockPlan.findBestTask(prop0, &Sign) c.Assert(err, IsNil) c.Assert(task.invalid(), IsTrue) @@ -187,7 +187,7 @@ func (s *testFindBestTaskSuite) TestEnforcedProperty(c *C) { Enforced: true, } // should return the valid task when the property is enforced. - task, _, err = mockPlan.findBestTask(prop1) + task, _, err = mockPlan.findBestTask(prop1, &Sign) c.Assert(err, IsNil) c.Assert(task.invalid(), IsFalse) } @@ -210,7 +210,7 @@ func (s *testFindBestTaskSuite) TestHintCannotFitProperty(c *C) { Items: items, Enforced: true, } - task, _, err := mockPlan0.findBestTask(prop0) + task, _, err := mockPlan0.findBestTask(prop0, &Sign) c.Assert(err, IsNil) c.Assert(task.invalid(), IsFalse) _, enforcedSort := task.plan().(*PhysicalSort) @@ -226,7 +226,7 @@ func (s *testFindBestTaskSuite) TestHintCannotFitProperty(c *C) { Items: items, Enforced: false, } - task, _, err = mockPlan0.findBestTask(prop1) + task, _, err = mockPlan0.findBestTask(prop1, &Sign) c.Assert(err, IsNil) c.Assert(task.invalid(), IsFalse) _, enforcedSort = task.plan().(*PhysicalSort) @@ -247,7 +247,7 @@ func (s *testFindBestTaskSuite) TestHintCannotFitProperty(c *C) { canGeneratePlan2: false, }.Init(ctx) mockPlan1.SetChildren(mockDS) - task, _, err = mockPlan1.findBestTask(prop2) + task, _, err = mockPlan1.findBestTask(prop2, &Sign) c.Assert(err, IsNil) c.Assert(task.invalid(), IsFalse) c.Assert(ctx.GetSessionVars().StmtCtx.WarningCount(), Equals, uint16(1)) @@ -263,7 +263,7 @@ func (s *testFindBestTaskSuite) TestHintCannotFitProperty(c *C) { Items: items, Enforced: true, } - task, _, err = mockPlan1.findBestTask(prop3) + task, _, err = mockPlan1.findBestTask(prop3, &Sign) c.Assert(err, IsNil) c.Assert(task.invalid(), IsFalse) c.Assert(ctx.GetSessionVars().StmtCtx.WarningCount(), Equals, uint16(1)) From 213b14217f3252b8e307bf53fa8d064ab11ca505 Mon Sep 17 00:00:00 2001 From: lawyerphx Date: Tue, 9 Jun 2020 20:01:40 -0700 Subject: [PATCH 08/37] fix unit tests. --- planner/core/find_best_task.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/planner/core/find_best_task.go b/planner/core/find_best_task.go index 37f9eed8c17a1..fa8bc9afe88d9 100644 --- a/planner/core/find_best_task.go +++ b/planner/core/find_best_task.go @@ -189,7 +189,7 @@ func (p *baseLogicalPlan) enumeratePhysicalPlans4Task(physicalPlans []PhysicalPl var curCntPlan int64 var cntPlan int64 = 0 childTasks := make([]task, 0, len(p.children)) - childCnts := make([]int64, 0, len(p.children)) + childCnts := make([]int64, len(p.children)) for _, pp := range physicalPlans { // find best child tasks firstly. childTasks = childTasks[:0] From c3ecb76f64fe4d1501ef1f830a29078379802951 Mon Sep 17 00:00:00 2001 From: lawyerphx Date: Wed, 10 Jun 2020 03:26:45 -0700 Subject: [PATCH 09/37] implement hint nth_plan. --- go.mod | 2 +- go.sum | 4 +-- planner/core/common_plans.go | 2 ++ planner/core/find_best_task.go | 26 +++++++++++++++-- planner/core/find_best_task_test.go | 1 + planner/core/logical_plan_test.go | 2 +- planner/core/optimizer.go | 13 +++++++-- planner/core/plan.go | 43 ++++++++++++++++++++++++++--- planner/core/plan_test.go | 28 +++++++++++++++++++ planner/optimize.go | 26 +++++++++++++++-- sessionctx/stmtctx/stmtctx.go | 3 ++ 11 files changed, 135 insertions(+), 15 deletions(-) diff --git a/go.mod b/go.mod index 06b8590346155..09db58d88fbf6 100644 --- a/go.mod +++ b/go.mod @@ -33,7 +33,7 @@ require ( github.com/pingcap/goleveldb v0.0.0-20191226122134-f82aafb29989 github.com/pingcap/kvproto v0.0.0-20200518112156-d4aeb467de29 github.com/pingcap/log v0.0.0-20200511115504-543df19646ad - github.com/pingcap/parser v0.0.0-20200601053017-87d29a820b2d + github.com/pingcap/parser v0.0.0-20200610035536-f0bf63b41eb1 github.com/pingcap/pd/v4 v4.0.0-rc.2.0.20200520083007-2c251bd8f181 github.com/pingcap/sysutil v0.0.0-20200408114249-ed3bd6f7fdb1 github.com/pingcap/tidb-tools v4.0.0-rc.1.0.20200514040632-f76b3e428e19+incompatible diff --git a/go.sum b/go.sum index 57cbbec087348..85c6ba66ee07e 100644 --- a/go.sum +++ b/go.sum @@ -455,8 +455,8 @@ github.com/pingcap/parser v0.0.0-20200424075042-8222d8b724a4/go.mod h1:9v0Edh8Ib github.com/pingcap/parser v0.0.0-20200507022230-f3bf29096657 h1:2ceTso30kmgMeddZ4iZ6zrK8N9eFF8zmCa1hSSE1tXc= github.com/pingcap/parser v0.0.0-20200507022230-f3bf29096657/go.mod h1:9v0Edh8IbgjGYW2ArJr19E+bvL8zKahsFp+ixWeId+4= github.com/pingcap/parser v0.0.0-20200522094936-3b720a0512a6/go.mod h1:vQdbJqobJAgFyiRNNtXahpMoGWwPEuWciVEK5A20NS0= -github.com/pingcap/parser v0.0.0-20200601053017-87d29a820b2d h1:eXigbUyAJDOhAIX9Hx72SqwbfSmpMiohdVM+WV5M0QQ= -github.com/pingcap/parser v0.0.0-20200601053017-87d29a820b2d/go.mod h1:vQdbJqobJAgFyiRNNtXahpMoGWwPEuWciVEK5A20NS0= +github.com/pingcap/parser v0.0.0-20200610035536-f0bf63b41eb1 h1:PTHVXiz6ULSqEC46stmV5ha+7Uj9Qepqga4+weJmycI= +github.com/pingcap/parser v0.0.0-20200610035536-f0bf63b41eb1/go.mod h1:vQdbJqobJAgFyiRNNtXahpMoGWwPEuWciVEK5A20NS0= github.com/pingcap/pd/v4 v4.0.0-rc.1.0.20200422143320-428acd53eba2 h1:JTzYYukREvxVSKW/ncrzNjFitd8snoQ/Xz32pw8i+s8= github.com/pingcap/pd/v4 v4.0.0-rc.1.0.20200422143320-428acd53eba2/go.mod h1:s+utZtXDznOiL24VK0qGmtoHjjXNsscJx3m1n8cC56s= github.com/pingcap/pd/v4 v4.0.0-rc.2.0.20200520083007-2c251bd8f181 h1:FM+PzdoR3fmWAJx3ug+p5aOgs5aZYwFkoDL7Potdsz0= diff --git a/planner/core/common_plans.go b/planner/core/common_plans.go index 7045fffe035b9..dd5c1f5d1b5df 100644 --- a/planner/core/common_plans.go +++ b/planner/core/common_plans.go @@ -835,6 +835,8 @@ func (e *Explain) RenderResult() error { hints := GenHintsFromPhysicalPlan(e.TargetPlan) hints = append(hints, hint.ExtractTableHintsFromStmtNode(e.ExecStmt)...) e.Rows = append(e.Rows, []string{hint.RestoreOptimizerHints(hints)}) + case ast.ExplainFormatVerbose: + //TODO: from PHX add expain format = verbose. here default: return errors.Errorf("explain format '%s' is not supported now", e.Format) } diff --git a/planner/core/find_best_task.go b/planner/core/find_best_task.go index fa8bc9afe88d9..a89cf87f01958 100644 --- a/planner/core/find_best_task.go +++ b/planner/core/find_best_task.go @@ -16,6 +16,7 @@ package core import ( "math" + "github.com/pingcap/errors" "github.com/pingcap/parser/ast" "github.com/pingcap/parser/model" "github.com/pingcap/parser/mysql" @@ -141,6 +142,7 @@ func (p *LogicalTableDual) findBestTask(prop *property.PhysicalProperty, clock * RowCount: p.RowCount, }.Init(p.ctx, p.stats, p.blockOffset) dual.SetSchema(p.schema) + clock.Dec(1) return &rootTask{p: dual}, 1, nil } @@ -150,6 +152,7 @@ func (p *LogicalShow) findBestTask(prop *property.PhysicalProperty, clock *Count } pShow := PhysicalShow{ShowContents: p.ShowContents}.Init(p.ctx) pShow.SetSchema(p.schema) + clock.Dec(1) return &rootTask{p: pShow}, 1, nil } @@ -159,6 +162,7 @@ func (p *LogicalShowDDLJobs) findBestTask(prop *property.PhysicalProperty, clock } pShow := PhysicalShowDDLJobs{JobNumber: p.JobNumber}.Init(p.ctx) pShow.SetSchema(p.schema) + clock.Dec(1) return &rootTask{p: pShow}, 1, nil } @@ -169,16 +173,18 @@ func (p *baseLogicalPlan) rebuildChildTasks(childTasks *[]task, pp PhysicalPlan, for _, x := range childCnts { multAll *= int8(x) } + *childTasks = (*childTasks)[:0] for j, child := range p.children { multAll /= int8(childCnts[j]) curClock = CountDown((clock-1)/multAll + 1) childTask, _, err := child.findBestTask(pp.GetChildReqProps(j), &curClock) clock = (clock-1)%multAll + 1 - // y.Assert(curClock == 0) if err != nil { return err } - // y.Assert(!(childTask != nil && childTask.invalid())) + if curClock != 0 || (childTask != nil && childTask.invalid()) { + return errors.Errorf("CountDown clock is not handled") + } *childTasks = append(*childTasks, childTask) } return nil @@ -189,6 +195,7 @@ func (p *baseLogicalPlan) enumeratePhysicalPlans4Task(physicalPlans []PhysicalPl var curCntPlan int64 var cntPlan int64 = 0 childTasks := make([]task, 0, len(p.children)) + // childCnts : record the number of tasks founded during generating childTasks childCnts := make([]int64, len(p.children)) for _, pp := range physicalPlans { // find best child tasks firstly. @@ -214,6 +221,14 @@ func (p *baseLogicalPlan) enumeratePhysicalPlans4Task(physicalPlans []PhysicalPl } if clock.IsForce() && int64(*clock) <= curCntPlan { + // the taskMap of children nodes should be rolled back. + for _, child := range p.children { + if pp, ok := child.(*baseLogicalPlan); ok { + pp.rollBackTaskMap() + } else if pp, ok := child.(*DataSource); ok { + pp.rollBackTaskMap() + } + } curCntPlan = int64(*clock) err := p.rebuildChildTasks(&childTasks, pp, childCnts, int8(*clock)) if err != nil { @@ -255,6 +270,9 @@ func (p *baseLogicalPlan) enumeratePhysicalPlans4Task(physicalPlans []PhysicalPl func (p *baseLogicalPlan) findBestTask(prop *property.PhysicalProperty, clock *CountDown) (bestTask task, cntPlan int64, err error) { // If p is an inner plan in an IndexJoin, the IndexJoin will generate an inner plan by itself, // and set inner child prop nil, so here we do nothing. + if clock.IsForce() { + p.enableBak() + } if prop == nil { return nil, 0, nil } @@ -263,6 +281,9 @@ func (p *baseLogicalPlan) findBestTask(prop *property.PhysicalProperty, clock *C bestTask = p.getTask(prop) if bestTask != nil { clock.Dec(1) + if p.needBak { + p.storeTask(prop, bestTask) + } return bestTask, 1, nil } @@ -354,6 +375,7 @@ func (p *LogicalMemTable) findBestTask(prop *property.PhysicalProperty, clock *C QueryTimeRange: p.QueryTimeRange, }.Init(p.ctx, p.stats, p.blockOffset) memTable.SetSchema(p.schema) + clock.Dec(1) return &rootTask{p: memTable}, 1, nil } diff --git a/planner/core/find_best_task_test.go b/planner/core/find_best_task_test.go index f80cc8eafcb77..ad386b13da1a9 100644 --- a/planner/core/find_best_task_test.go +++ b/planner/core/find_best_task_test.go @@ -50,6 +50,7 @@ func (ds *mockDataSource) findBestTask(prop *property.PhysicalProperty, clock *C p: p, cst: 10000, } + clock.Dec(1) return task, 1, nil } diff --git a/planner/core/logical_plan_test.go b/planner/core/logical_plan_test.go index 08b380d109715..7217dd17bcea4 100644 --- a/planner/core/logical_plan_test.go +++ b/planner/core/logical_plan_test.go @@ -1379,7 +1379,7 @@ func (s *testPlanSuite) optimize(ctx context.Context, sql string) (PhysicalPlan, if err != nil { return nil, nil, err } - p, _, err = physicalOptimize(p.(LogicalPlan)) + p, _, err = physicalOptimize(p.(LogicalPlan), &Sign) return p.(PhysicalPlan), stmt, err } diff --git a/planner/core/optimizer.go b/planner/core/optimizer.go index 510859f18b7d7..c1995c8a4f46f 100644 --- a/planner/core/optimizer.go +++ b/planner/core/optimizer.go @@ -132,7 +132,11 @@ func DoOptimize(ctx context.Context, sctx sessionctx.Context, flag uint64, logic if !AllowCartesianProduct.Load() && existsCartesianProduct(logic) { return nil, 0, errors.Trace(ErrCartesianProductUnsupported) } - physical, cost, err := physicalOptimize(logic) + var clock = CountDown(sctx.GetSessionVars().StmtCtx.StmtHints.ForceNthPlan) + if clock == 0 { + clock = -1 + } + physical, cost, err := physicalOptimize(logic, &clock) if err != nil { return nil, 0, err } @@ -169,7 +173,7 @@ func isLogicalRuleDisabled(r logicalOptRule) bool { return disabled } -func physicalOptimize(logic LogicalPlan) (PhysicalPlan, float64, error) { +func physicalOptimize(logic LogicalPlan, clock *CountDown) (PhysicalPlan, float64, error) { if _, err := logic.recursiveDeriveStats(); err != nil { return nil, 0, err } @@ -181,10 +185,13 @@ func physicalOptimize(logic LogicalPlan) (PhysicalPlan, float64, error) { ExpectedCnt: math.MaxFloat64, } - t, _, err := logic.findBestTask(prop, &Sign) + t, _, err := logic.findBestTask(prop, clock) if err != nil { return nil, 0, err } + if *clock > 0 { + return nil, 0, ErrInternal.GenWithStackByArgs("The parameter of nth_plan() is out of range.") + } if t.invalid() { return nil, 0, ErrInternal.GenWithStackByArgs("Can't find a proper physical plan for this query") } diff --git a/planner/core/plan.go b/planner/core/plan.go index 84af6650c8e0e..49857713c98a2 100644 --- a/planner/core/plan.go +++ b/planner/core/plan.go @@ -244,7 +244,11 @@ type PhysicalPlan interface { type baseLogicalPlan struct { basePlan - taskMap map[string]task + taskMap map[string]task + // taskMapBak is a backlog stack of taskMap, used to roll back the task. + taskMapBak []string + // needBask indicates whether taskMap needs to be backed. Avoid affecting the speed of optimizer. + needBak bool self LogicalPlan maxOneRow bool children []LogicalPlan @@ -286,6 +290,32 @@ func (p *basePhysicalPlan) ExtractCorrelatedCols() []*expression.CorrelatedColum return nil } +// enableBak sets the needBak of logical plan tree to true. +func (p *baseLogicalPlan) enableBak() { + p.needBak = true + for _, child := range p.children { + if pp, ok := child.(*baseLogicalPlan); ok && !pp.needBak { + pp.enableBak() + } else if pp, ok := child.(*DataSource); ok && !pp.needBak { + pp.enableBak() + } + } +} + +func (p *baseLogicalPlan) rollBackTaskMap() { + if len(p.taskMapBak) > 0 { + p.taskMap[p.taskMapBak[len(p.taskMapBak)-1]] = nil + p.taskMapBak = p.taskMapBak[:len(p.taskMapBak)-1] + } + for _, child := range p.children { + if pp, ok := child.(*baseLogicalPlan); ok { + pp.rollBackTaskMap() + } else if pp, ok := child.(*DataSource); ok { + pp.rollBackTaskMap() + } + } +} + func (p *baseLogicalPlan) getTask(prop *property.PhysicalProperty) task { key := prop.HashCode() return p.taskMap[string(key)] @@ -293,6 +323,9 @@ func (p *baseLogicalPlan) getTask(prop *property.PhysicalProperty) task { func (p *baseLogicalPlan) storeTask(prop *property.PhysicalProperty, task task) { key := prop.HashCode() + if p.needBak { + p.taskMapBak = append(p.taskMapBak, string(key)) + } p.taskMap[string(key)] = task } @@ -349,9 +382,11 @@ func newBasePlan(ctx sessionctx.Context, tp string, offset int) basePlan { func newBaseLogicalPlan(ctx sessionctx.Context, tp string, self LogicalPlan, offset int) baseLogicalPlan { return baseLogicalPlan{ - taskMap: make(map[string]task), - basePlan: newBasePlan(ctx, tp, offset), - self: self, + taskMap: make(map[string]task), + taskMapBak: make([]string, 0, 100), + needBak: false, + basePlan: newBasePlan(ctx, tp, offset), + self: self, } } diff --git a/planner/core/plan_test.go b/planner/core/plan_test.go index fa9936b0c55f3..a2d377ecc5b0f 100644 --- a/planner/core/plan_test.go +++ b/planner/core/plan_test.go @@ -232,3 +232,31 @@ func compareStringSlice(c *C, ss1, ss2 []string) { c.Assert(s, Equals, ss2[i]) } } + +func (s *testPlanNormalize) TestNthPlanHint(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + tk.MustExec("drop table if exists tt") + tk.MustExec("create table tt (a int,b int, index(a), index(b));") + tk.MustExec("insert into tt values (1, 1), (2, 2), (3, 4)") + + tk.MustQuery("explain select /*+nth_plan(1)*/ * from tt where a=1 and b=1;").Check(testkit.Rows( + "TableReader_18 0.00 root data:Selection_17", + "└─Selection_17 0.00 cop[tikv] eq(test.tt.a, 1), eq(test.tt.b, 1)", + " └─TableFullScan_16 3.00 cop[tikv] table:tt keep order:false, stats:pseudo")) + + tk.MustQuery("explain select /*+nth_plan(2)*/ * from tt where a=1 and b=1;").Check(testkit.Rows( + "IndexLookUp_22 0.00 root ", + "├─IndexRangeScan_19(Build) 0.00 cop[tikv] table:tt, index:a(a) range:[1,1], keep order:false, stats:pseudo", + "└─Selection_21(Probe) 0.00 cop[tikv] eq(test.tt.b, 1)", + " └─TableRowIDScan_20 0.00 cop[tikv] table:tt keep order:false, stats:pseudo")) + + tk.MustQuery("explain select /*+nth_plan(3)*/ * from tt where a=1 and b=1;").Check(testkit.Rows( + "IndexLookUp_26 0.00 root ", + "├─IndexRangeScan_23(Build) 0.00 cop[tikv] table:tt, index:b(b) range:[1,1], keep order:false, stats:pseudo", + "└─Selection_25(Probe) 0.00 cop[tikv] eq(test.tt.a, 1)", + " └─TableRowIDScan_24 0.00 cop[tikv] table:tt keep order:false, stats:pseudo")) + + tk.MustQuery("explain select /*+nth_plan(4)*/ * from tt where a=1 and b=1;").Check(testkit.Rows( + "ERROR 1815 (HY000): Internal : The parameter of nth_plan() is out of range.")) +} diff --git a/planner/optimize.go b/planner/optimize.go index bf8684dfda7f2..facee61190f4d 100644 --- a/planner/optimize.go +++ b/planner/optimize.go @@ -317,8 +317,8 @@ func handleStmtHints(hints []*ast.TableOptimizerHint) (stmtHints stmtctx.StmtHin if len(hints) == 0 { return } - var memoryQuotaHint, useToJAHint, useCascadesHint, maxExecutionTime *ast.TableOptimizerHint - var memoryQuotaHintCnt, useToJAHintCnt, useCascadesHintCnt, noIndexMergeHintCnt, readReplicaHintCnt, maxExecutionTimeCnt int + var memoryQuotaHint, useToJAHint, useCascadesHint, maxExecutionTime, forceNthPlan *ast.TableOptimizerHint + var memoryQuotaHintCnt, useToJAHintCnt, useCascadesHintCnt, noIndexMergeHintCnt, readReplicaHintCnt, maxExecutionTimeCnt, forceNthPlanCnt int for _, hint := range hints { switch hint.HintName.L { case "memory_quota": @@ -337,6 +337,9 @@ func handleStmtHints(hints []*ast.TableOptimizerHint) (stmtHints stmtctx.StmtHin case "max_execution_time": maxExecutionTimeCnt++ maxExecutionTime = hint + case "nth_plan": + forceNthPlanCnt++ + forceNthPlan = hint } } // Handle MEMORY_QUOTA @@ -402,6 +405,25 @@ func handleStmtHints(hints []*ast.TableOptimizerHint) (stmtHints stmtctx.StmtHin stmtHints.HasMaxExecutionTime = true stmtHints.MaxExecutionTime = maxExecutionTime.HintData.(uint64) } + // Handle NTH_PLAN + if forceNthPlanCnt != 0 { + if forceNthPlanCnt > 1 { + warn := errors.Errorf("NTH_PLAN() is defined more than once, only the last definition takes effect: NTH_PLAN(%v)", forceNthPlan.HintData.(uint64)) + warns = append(warns, warn) + } + stmtHints.ForceNthPlan = forceNthPlan.HintData.(int64) + if stmtHints.ForceNthPlan < 1 { + stmtHints.ForceNthPlan = 1 + warn := errors.Errorf("the hintdata for NTH_PLAN() is too small, set to 1 automatically.") + warns = append(warns, warn) + } else if stmtHints.ForceNthPlan > 100 { + stmtHints.ForceNthPlan = 100 + warn := errors.Errorf("the hintdata for NTH_PLAN() is too big, set to 100 automatically.") + warns = append(warns, warn) + } + } else { + stmtHints.ForceNthPlan = -1 + } return } diff --git a/sessionctx/stmtctx/stmtctx.go b/sessionctx/stmtctx/stmtctx.go index 36033c6c020cf..18fd57eea7fbe 100644 --- a/sessionctx/stmtctx/stmtctx.go +++ b/sessionctx/stmtctx/stmtctx.go @@ -165,6 +165,9 @@ type StmtHints struct { NoIndexMergeHint bool // EnableCascadesPlanner is use cascades planner for a single query only. EnableCascadesPlanner bool + // ForceNthPlan indicates the countDown number for finding physical plan. + // -1 for disable + ForceNthPlan int64 // Hint flags HasAllowInSubqToJoinAndAggHint bool From bb518ee5785a0c13c224de9e0c22afadb9559a61 Mon Sep 17 00:00:00 2001 From: lawyerphx Date: Wed, 10 Jun 2020 03:44:24 -0700 Subject: [PATCH 10/37] fix unit test. --- planner/core/plan_test.go | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/planner/core/plan_test.go b/planner/core/plan_test.go index a2d377ecc5b0f..8fdf186b584f5 100644 --- a/planner/core/plan_test.go +++ b/planner/core/plan_test.go @@ -241,9 +241,9 @@ func (s *testPlanNormalize) TestNthPlanHint(c *C) { tk.MustExec("insert into tt values (1, 1), (2, 2), (3, 4)") tk.MustQuery("explain select /*+nth_plan(1)*/ * from tt where a=1 and b=1;").Check(testkit.Rows( - "TableReader_18 0.00 root data:Selection_17", - "└─Selection_17 0.00 cop[tikv] eq(test.tt.a, 1), eq(test.tt.b, 1)", - " └─TableFullScan_16 3.00 cop[tikv] table:tt keep order:false, stats:pseudo")) + "TableReader_18 0.01 root data:Selection_17", + "└─Selection_17 0.01 cop[tikv] eq(test.tt.a, 1), eq(test.tt.b, 1)", + " └─TableFullScan_16 10000.00 cop[tikv] table:tt keep order:false, stats:pseudo")) tk.MustQuery("explain select /*+nth_plan(2)*/ * from tt where a=1 and b=1;").Check(testkit.Rows( "IndexLookUp_22 0.00 root ", @@ -257,6 +257,6 @@ func (s *testPlanNormalize) TestNthPlanHint(c *C) { "└─Selection_25(Probe) 0.00 cop[tikv] eq(test.tt.a, 1)", " └─TableRowIDScan_24 0.00 cop[tikv] table:tt keep order:false, stats:pseudo")) - tk.MustQuery("explain select /*+nth_plan(4)*/ * from tt where a=1 and b=1;").Check(testkit.Rows( - "ERROR 1815 (HY000): Internal : The parameter of nth_plan() is out of range.")) + _, err := tk.Exec("explain select /*+nth_plan(4)*/ * from tt where a=1 and b=1;") + c.Assert(err, NotNil) } From ab00bfa2007f737614a37fa5e6fd4d23e1775696 Mon Sep 17 00:00:00 2001 From: lawyerphx Date: Wed, 10 Jun 2020 03:46:04 -0700 Subject: [PATCH 11/37] fix unit test. --- planner/core/plan_test.go | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/planner/core/plan_test.go b/planner/core/plan_test.go index 8fdf186b584f5..0b2cc9d4c3b2a 100644 --- a/planner/core/plan_test.go +++ b/planner/core/plan_test.go @@ -257,6 +257,13 @@ func (s *testPlanNormalize) TestNthPlanHint(c *C) { "└─Selection_25(Probe) 0.00 cop[tikv] eq(test.tt.a, 1)", " └─TableRowIDScan_24 0.00 cop[tikv] table:tt keep order:false, stats:pseudo")) + // with the same param in nth_plan, we should get the same plan. + tk.MustQuery("explain select /*+nth_plan(2)*/ * from tt where a=1 and b=1;").Check(testkit.Rows( + "IndexLookUp_22 0.00 root ", + "├─IndexRangeScan_19(Build) 0.00 cop[tikv] table:tt, index:a(a) range:[1,1], keep order:false, stats:pseudo", + "└─Selection_21(Probe) 0.00 cop[tikv] eq(test.tt.b, 1)", + " └─TableRowIDScan_20 0.00 cop[tikv] table:tt keep order:false, stats:pseudo")) + _, err := tk.Exec("explain select /*+nth_plan(4)*/ * from tt where a=1 and b=1;") c.Assert(err, NotNil) } From 7c0d44690392dac125345b70af1c1d3de779c17d Mon Sep 17 00:00:00 2001 From: lawyerphx Date: Wed, 10 Jun 2020 03:52:51 -0700 Subject: [PATCH 12/37] update the unit tests to keep consist with master. --- planner/core/plan_test.go | 24 ++++++++++++------------ 1 file changed, 12 insertions(+), 12 deletions(-) diff --git a/planner/core/plan_test.go b/planner/core/plan_test.go index 0b2cc9d4c3b2a..eecfef80d5775 100644 --- a/planner/core/plan_test.go +++ b/planner/core/plan_test.go @@ -246,23 +246,23 @@ func (s *testPlanNormalize) TestNthPlanHint(c *C) { " └─TableFullScan_16 10000.00 cop[tikv] table:tt keep order:false, stats:pseudo")) tk.MustQuery("explain select /*+nth_plan(2)*/ * from tt where a=1 and b=1;").Check(testkit.Rows( - "IndexLookUp_22 0.00 root ", - "├─IndexRangeScan_19(Build) 0.00 cop[tikv] table:tt, index:a(a) range:[1,1], keep order:false, stats:pseudo", - "└─Selection_21(Probe) 0.00 cop[tikv] eq(test.tt.b, 1)", - " └─TableRowIDScan_20 0.00 cop[tikv] table:tt keep order:false, stats:pseudo")) + "IndexLookUp_22 0.01 root ", + "├─IndexRangeScan_19(Build) 10.00 cop[tikv] table:tt, index:a(a) range:[1,1], keep order:false, stats:pseudo", + "└─Selection_21(Probe) 0.01 cop[tikv] eq(test.tt.b, 1)", + " └─TableRowIDScan_20 10.00 cop[tikv] table:tt keep order:false, stats:pseudo")) tk.MustQuery("explain select /*+nth_plan(3)*/ * from tt where a=1 and b=1;").Check(testkit.Rows( - "IndexLookUp_26 0.00 root ", - "├─IndexRangeScan_23(Build) 0.00 cop[tikv] table:tt, index:b(b) range:[1,1], keep order:false, stats:pseudo", - "└─Selection_25(Probe) 0.00 cop[tikv] eq(test.tt.a, 1)", - " └─TableRowIDScan_24 0.00 cop[tikv] table:tt keep order:false, stats:pseudo")) + "IndexLookUp_26 0.01 root ", + "├─IndexRangeScan_23(Build) 10.00 cop[tikv] table:tt, index:b(b) range:[1,1], keep order:false, stats:pseudo", + "└─Selection_25(Probe) 0.01 cop[tikv] eq(test.tt.a, 1)", + " └─TableRowIDScan_24 10.00 cop[tikv] table:tt keep order:false, stats:pseudo")) // with the same param in nth_plan, we should get the same plan. tk.MustQuery("explain select /*+nth_plan(2)*/ * from tt where a=1 and b=1;").Check(testkit.Rows( - "IndexLookUp_22 0.00 root ", - "├─IndexRangeScan_19(Build) 0.00 cop[tikv] table:tt, index:a(a) range:[1,1], keep order:false, stats:pseudo", - "└─Selection_21(Probe) 0.00 cop[tikv] eq(test.tt.b, 1)", - " └─TableRowIDScan_20 0.00 cop[tikv] table:tt keep order:false, stats:pseudo")) + "IndexLookUp_22 0.01 root ", + "├─IndexRangeScan_19(Build) 10.00 cop[tikv] table:tt, index:a(a) range:[1,1], keep order:false, stats:pseudo", + "└─Selection_21(Probe) 0.01 cop[tikv] eq(test.tt.b, 1)", + " └─TableRowIDScan_20 10.00 cop[tikv] table:tt keep order:false, stats:pseudo")) _, err := tk.Exec("explain select /*+nth_plan(4)*/ * from tt where a=1 and b=1;") c.Assert(err, NotNil) From 0d3f50a7765cea657b99ea6a2aa70f30abe3c1b3 Mon Sep 17 00:00:00 2001 From: lawyerphx Date: Wed, 10 Jun 2020 19:55:00 -0700 Subject: [PATCH 13/37] add some comments to help review. --- planner/core/find_best_task.go | 3 +++ planner/core/plan.go | 6 +++++- 2 files changed, 8 insertions(+), 1 deletion(-) diff --git a/planner/core/find_best_task.go b/planner/core/find_best_task.go index 52dd224257dae..e633b80531716 100644 --- a/planner/core/find_best_task.go +++ b/planner/core/find_best_task.go @@ -283,6 +283,7 @@ func (p *baseLogicalPlan) findBestTask(prop *property.PhysicalProperty, clock *C if bestTask != nil { clock.Dec(1) if p.needBak { + // Ensure that a key will be pushed into taskMapBak every time we call the function. p.storeTask(prop, bestTask) } return bestTask, 1, nil @@ -563,6 +564,8 @@ func (ds *DataSource) findBestTask(prop *property.PhysicalProperty, clock *Count if t != nil { cntPlan = 1 clock.Dec(1) + // Ensure that a key will be pushed into taskMapBak every time we call the function. + ds.storeTask(prop, t) return } var cnt int64 diff --git a/planner/core/plan.go b/planner/core/plan.go index 49857713c98a2..fe39f51950d94 100644 --- a/planner/core/plan.go +++ b/planner/core/plan.go @@ -247,7 +247,7 @@ type baseLogicalPlan struct { taskMap map[string]task // taskMapBak is a backlog stack of taskMap, used to roll back the task. taskMapBak []string - // needBask indicates whether taskMap needs to be backed. Avoid affecting the speed of optimizer. + // needBask indicates whether taskMap needs to be backed. Avoid affecting the speed of the optimizer. needBak bool self LogicalPlan maxOneRow bool @@ -302,12 +302,14 @@ func (p *baseLogicalPlan) enableBak() { } } +// rollBackTaskMap roll back the taskMap with the record in taskMapBak func (p *baseLogicalPlan) rollBackTaskMap() { if len(p.taskMapBak) > 0 { p.taskMap[p.taskMapBak[len(p.taskMapBak)-1]] = nil p.taskMapBak = p.taskMapBak[:len(p.taskMapBak)-1] } for _, child := range p.children { + // only baseLogicalPlan and DataSource will be affected. if pp, ok := child.(*baseLogicalPlan); ok { pp.rollBackTaskMap() } else if pp, ok := child.(*DataSource); ok { @@ -324,6 +326,8 @@ func (p *baseLogicalPlan) getTask(prop *property.PhysicalProperty) task { func (p *baseLogicalPlan) storeTask(prop *property.PhysicalProperty, task task) { key := prop.HashCode() if p.needBak { + // when we storeTask, we have taskMap[key] is nil. + // so to roll back, we need only store the key and set taskMap[key] to nil. p.taskMapBak = append(p.taskMapBak, string(key)) } p.taskMap[string(key)] = task From be7a57e7d33fb0e91f47a75e795249a69d068693 Mon Sep 17 00:00:00 2001 From: lawyerphx Date: Wed, 10 Jun 2020 20:52:58 -0700 Subject: [PATCH 14/37] refine unit tests --- planner/core/find_best_task.go | 4 +++- planner/core/plan_test.go | 26 ++++++++++++++++++++++++++ 2 files changed, 29 insertions(+), 1 deletion(-) diff --git a/planner/core/find_best_task.go b/planner/core/find_best_task.go index e633b80531716..dbc59a8d5e5af 100644 --- a/planner/core/find_best_task.go +++ b/planner/core/find_best_task.go @@ -565,7 +565,9 @@ func (ds *DataSource) findBestTask(prop *property.PhysicalProperty, clock *Count cntPlan = 1 clock.Dec(1) // Ensure that a key will be pushed into taskMapBak every time we call the function. - ds.storeTask(prop, t) + if ds.needBak { + ds.storeTask(prop, t) + } return } var cnt int64 diff --git a/planner/core/plan_test.go b/planner/core/plan_test.go index eecfef80d5775..eb5bd7c9960ff 100644 --- a/planner/core/plan_test.go +++ b/planner/core/plan_test.go @@ -240,6 +240,7 @@ func (s *testPlanNormalize) TestNthPlanHint(c *C) { tk.MustExec("create table tt (a int,b int, index(a), index(b));") tk.MustExec("insert into tt values (1, 1), (2, 2), (3, 4)") + // test the explained plans. tk.MustQuery("explain select /*+nth_plan(1)*/ * from tt where a=1 and b=1;").Check(testkit.Rows( "TableReader_18 0.01 root data:Selection_17", "└─Selection_17 0.01 cop[tikv] eq(test.tt.a, 1), eq(test.tt.b, 1)", @@ -266,4 +267,29 @@ func (s *testPlanNormalize) TestNthPlanHint(c *C) { _, err := tk.Exec("explain select /*+nth_plan(4)*/ * from tt where a=1 and b=1;") c.Assert(err, NotNil) + + // test hints for nth_plan(x) + tk.MustExec("drop table if exists t") + tk.MustExec("create table t (a int, b int, c int, index(a), index(b), index(a,b))") + tk.MustQuery("explain format='hint' select * from t where a=1 and b=1").Check(testkit.Rows( + "use_index(@`sel_1` `test`.`t` `a_2`)")) + tk.MustQuery("explain format='hint' select * /*+ nth_plan(1) */ from t where a=1 and b=1").Check(testkit.Rows( + "use_index(@`sel_1` `test`.`t` ), nth_plan(1)")) + tk.MustQuery("explain format='hint' select * /*+ nth_plan(2) */ from t where a=1 and b=1").Check(testkit.Rows( + "use_index(@`sel_1` `test`.`t` `a_2`), nth_plan(2)")) + + _, err = tk.Exec("explain format='hint' select * /*+ nth_plan(3) */ from t where a=1 and b=1") + c.Assert(err, NotNil) + + // test the correctness of generated plans. + tk.MustQuery("select /*+ nth_plan(1) */ * from t where a=1 and b=1;").Check(testkit.Rows( + "1 1 1")) + tk.MustQuery("select /*+ nth_plan(2) */ * from t where a=1 and b=1;").Check(testkit.Rows( + "1 1 1")) + tk.MustQuery("select /*+ nth_plan(1) */ * from tt where a=1 and b=1;").Check(testkit.Rows( + "1 1")) + tk.MustQuery("select /*+ nth_plan(2) */ * from tt where a=1 and b=1;").Check(testkit.Rows( + "1 1")) + tk.MustQuery("select /*+ nth_plan(3) */ * from tt where a=1 and b=1;").Check(testkit.Rows( + "1 1")) } From e411f6a98a434cba501dd186de2df47250eebd68 Mon Sep 17 00:00:00 2001 From: lawyerphx Date: Wed, 10 Jun 2020 20:54:49 -0700 Subject: [PATCH 15/37] refine unit tests --- planner/core/plan_test.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/planner/core/plan_test.go b/planner/core/plan_test.go index eb5bd7c9960ff..2b11b7311592e 100644 --- a/planner/core/plan_test.go +++ b/planner/core/plan_test.go @@ -272,7 +272,7 @@ func (s *testPlanNormalize) TestNthPlanHint(c *C) { tk.MustExec("drop table if exists t") tk.MustExec("create table t (a int, b int, c int, index(a), index(b), index(a,b))") tk.MustQuery("explain format='hint' select * from t where a=1 and b=1").Check(testkit.Rows( - "use_index(@`sel_1` `test`.`t` `a_2`)")) + "use_index(@`sel_1` `test`.`t` `a_2`)")) tk.MustQuery("explain format='hint' select * /*+ nth_plan(1) */ from t where a=1 and b=1").Check(testkit.Rows( "use_index(@`sel_1` `test`.`t` ), nth_plan(1)")) tk.MustQuery("explain format='hint' select * /*+ nth_plan(2) */ from t where a=1 and b=1").Check(testkit.Rows( From 3ba32fc20d87b0a312cf6d5e5094be8dfa40230f Mon Sep 17 00:00:00 2001 From: lawyerphx Date: Wed, 10 Jun 2020 21:52:59 -0700 Subject: [PATCH 16/37] correct warnings --- planner/core/plan_test.go | 14 +++++++++++--- planner/optimize.go | 2 +- 2 files changed, 12 insertions(+), 4 deletions(-) diff --git a/planner/core/plan_test.go b/planner/core/plan_test.go index 2b11b7311592e..01d45ca866878 100644 --- a/planner/core/plan_test.go +++ b/planner/core/plan_test.go @@ -273,15 +273,23 @@ func (s *testPlanNormalize) TestNthPlanHint(c *C) { tk.MustExec("create table t (a int, b int, c int, index(a), index(b), index(a,b))") tk.MustQuery("explain format='hint' select * from t where a=1 and b=1").Check(testkit.Rows( "use_index(@`sel_1` `test`.`t` `a_2`)")) - tk.MustQuery("explain format='hint' select * /*+ nth_plan(1) */ from t where a=1 and b=1").Check(testkit.Rows( + tk.MustQuery("explain format='hint' select /*+ nth_plan(1) */ * from t where a=1 and b=1").Check(testkit.Rows( "use_index(@`sel_1` `test`.`t` ), nth_plan(1)")) - tk.MustQuery("explain format='hint' select * /*+ nth_plan(2) */ from t where a=1 and b=1").Check(testkit.Rows( + tk.MustQuery("explain format='hint' select /*+ nth_plan(2) */ * from t where a=1 and b=1").Check(testkit.Rows( "use_index(@`sel_1` `test`.`t` `a_2`), nth_plan(2)")) - _, err = tk.Exec("explain format='hint' select * /*+ nth_plan(3) */ from t where a=1 and b=1") + _, err = tk.Exec("explain format='hint' select /*+ nth_plan(3) */ * from t where a=1 and b=1") c.Assert(err, NotNil) + // test warning for multiply hints + tk.MustQuery("explain format='hint' select /*+ nth_plan(1) nth_plan(2) */ * from t where a=1 and b=1").Check(testkit.Rows( + "use_index(@`sel_1` `test`.`t` `a_2`), nth_plan(1), nth_plan(2)")) + tk.MustQuery("show warnings").Check(testkit.Rows( + "Warning 1105 NTH_PLAN() is defined more than once, only the last definition takes effect: NTH_PLAN(2)", + "Warning 1105 NTH_PLAN() is defined more than once, only the last definition takes effect: NTH_PLAN(2)")) + // test the correctness of generated plans. + tk.MustExec("insert into t values (1,1,1)") tk.MustQuery("select /*+ nth_plan(1) */ * from t where a=1 and b=1;").Check(testkit.Rows( "1 1 1")) tk.MustQuery("select /*+ nth_plan(2) */ * from t where a=1 and b=1;").Check(testkit.Rows( diff --git a/planner/optimize.go b/planner/optimize.go index d7b3b70f43a3b..6d392161829fd 100644 --- a/planner/optimize.go +++ b/planner/optimize.go @@ -411,7 +411,7 @@ func handleStmtHints(hints []*ast.TableOptimizerHint) (stmtHints stmtctx.StmtHin // Handle NTH_PLAN if forceNthPlanCnt != 0 { if forceNthPlanCnt > 1 { - warn := errors.Errorf("NTH_PLAN() is defined more than once, only the last definition takes effect: NTH_PLAN(%v)", forceNthPlan.HintData.(uint64)) + warn := errors.Errorf("NTH_PLAN() is defined more than once, only the last definition takes effect: NTH_PLAN(%v)", forceNthPlan.HintData.(int64)) warns = append(warns, warn) } stmtHints.ForceNthPlan = forceNthPlan.HintData.(int64) From bc82b526045aebad4f2143c7c69a0238eff7317d Mon Sep 17 00:00:00 2001 From: lawyerphx Date: Thu, 11 Jun 2020 06:10:36 -0700 Subject: [PATCH 17/37] refine TaskMap rollback. --- planner/core/find_best_task.go | 18 ++++---- planner/core/optimizer.go | 2 + planner/core/plan.go | 76 ++++++++++++++++++++++++++-------- 3 files changed, 68 insertions(+), 28 deletions(-) diff --git a/planner/core/find_best_task.go b/planner/core/find_best_task.go index dbc59a8d5e5af..6525d5af32fbf 100644 --- a/planner/core/find_best_task.go +++ b/planner/core/find_best_task.go @@ -168,7 +168,12 @@ func (p *LogicalShowDDLJobs) findBestTask(prop *property.PhysicalProperty, clock } // rebuildChildTasks rebuild the childTasks to make the clock_th combination. -func (p *baseLogicalPlan) rebuildChildTasks(childTasks *[]task, pp PhysicalPlan, childCnts []int64, clock int8) error { +func (p *baseLogicalPlan) rebuildChildTasks(childTasks *[]task, pp PhysicalPlan, childCnts []int64, clock int8, TS int64) error { + // the taskMap of children nodes should be rolled back. + for _, child := range p.children { + child.rollBackTaskMap(TS) + } + var multAll int8 = 1 var curClock CountDown for _, x := range childCnts { @@ -203,6 +208,7 @@ func (p *baseLogicalPlan) enumeratePhysicalPlans4Task(physicalPlans []PhysicalPl childTasks = childTasks[:0] // curCntPlan record the number of possible plan for pp curCntPlan = 1 + TimeStampNow := p.GetBakTimeStamp() for j, child := range p.children { childTask, cnt, err := child.findBestTask(pp.GetChildReqProps(j), &Sign) childCnts[j] = cnt @@ -222,16 +228,8 @@ func (p *baseLogicalPlan) enumeratePhysicalPlans4Task(physicalPlans []PhysicalPl } if clock.IsForce() && int64(*clock) <= curCntPlan { - // the taskMap of children nodes should be rolled back. - for _, child := range p.children { - if pp, ok := child.(*baseLogicalPlan); ok { - pp.rollBackTaskMap() - } else if pp, ok := child.(*DataSource); ok { - pp.rollBackTaskMap() - } - } curCntPlan = int64(*clock) - err := p.rebuildChildTasks(&childTasks, pp, childCnts, int8(*clock)) + err := p.rebuildChildTasks(&childTasks, pp, childCnts, int8(*clock), TimeStampNow) if err != nil { return nil, 0, err } diff --git a/planner/core/optimizer.go b/planner/core/optimizer.go index c1995c8a4f46f..04c46ffb35060 100644 --- a/planner/core/optimizer.go +++ b/planner/core/optimizer.go @@ -185,6 +185,8 @@ func physicalOptimize(logic LogicalPlan, clock *CountDown) (PhysicalPlan, float6 ExpectedCnt: math.MaxFloat64, } + // clear the timestamp for taskMap rollback + ClearBakTimeStamp() t, _, err := logic.findBestTask(prop, clock) if err != nil { return nil, 0, err diff --git a/planner/core/plan.go b/planner/core/plan.go index fe39f51950d94..d0f6558de7bf3 100644 --- a/planner/core/plan.go +++ b/planner/core/plan.go @@ -200,6 +200,12 @@ type LogicalPlan interface { // SetChild sets the ith child for the plan. SetChild(i int, child LogicalPlan) + + // enableBak sets the needBak of logical plan tree to true. + enableBak() + + // rollBackTaskMap roll back all taskMap's logs after TimeStamp TS + rollBackTaskMap(TS int64) } // PhysicalPlan is a tree of the physical operators. @@ -245,8 +251,10 @@ type baseLogicalPlan struct { basePlan taskMap map[string]task - // taskMapBak is a backlog stack of taskMap, used to roll back the task. - taskMapBak []string + // taskMapBak and taskMapBakID form a backlog stack of taskMap, used to roll back the task. + // taskMapBakTS stores the TimeStamps of logs. + taskMapBak []string + taskMapBakTS []int64 // needBask indicates whether taskMap needs to be backed. Avoid affecting the speed of the optimizer. needBak bool self LogicalPlan @@ -290,31 +298,55 @@ func (p *basePhysicalPlan) ExtractCorrelatedCols() []*expression.CorrelatedColum return nil } -// enableBak sets the needBak of logical plan tree to true. +var BakTimeStamp int64 = 0 + +func ClearBakTimeStamp() { + BakTimeStamp = 0 +} + +func (p *baseLogicalPlan) GetBakTimeStamp() int64 { + BakTimeStamp++ + return BakTimeStamp +} + func (p *baseLogicalPlan) enableBak() { + if p.needBak { + return + } p.needBak = true for _, child := range p.children { - if pp, ok := child.(*baseLogicalPlan); ok && !pp.needBak { - pp.enableBak() - } else if pp, ok := child.(*DataSource); ok && !pp.needBak { - pp.enableBak() - } + child.enableBak() } } -// rollBackTaskMap roll back the taskMap with the record in taskMapBak -func (p *baseLogicalPlan) rollBackTaskMap() { +func (p *baseLogicalPlan) rollBackTaskMap(TS int64) { + if !p.needBak { + return + } if len(p.taskMapBak) > 0 { - p.taskMap[p.taskMapBak[len(p.taskMapBak)-1]] = nil - p.taskMapBak = p.taskMapBak[:len(p.taskMapBak)-1] + // rollback all the logs with TimeStamp TS + N := len(p.taskMapBak) + for i := 0; i < N; i++ { + cur := p.taskMapBak[i] + if p.taskMapBakTS[i] < TS { + continue + } + + // remove the i_th log + p.taskMapBak = append(p.taskMapBak[:i], p.taskMapBak[i+1:]...) + p.taskMapBakTS = append(p.taskMapBakTS[:i], p.taskMapBakTS[i+1:]...) + i-- + N-- + + // if cur is not an invalid log, then roll back. + if cur != "" { + p.taskMap[cur] = nil + } + } } for _, child := range p.children { // only baseLogicalPlan and DataSource will be affected. - if pp, ok := child.(*baseLogicalPlan); ok { - pp.rollBackTaskMap() - } else if pp, ok := child.(*DataSource); ok { - pp.rollBackTaskMap() - } + child.rollBackTaskMap(TS) } } @@ -328,7 +360,15 @@ func (p *baseLogicalPlan) storeTask(prop *property.PhysicalProperty, task task) if p.needBak { // when we storeTask, we have taskMap[key] is nil. // so to roll back, we need only store the key and set taskMap[key] to nil. - p.taskMapBak = append(p.taskMapBak, string(key)) + // empty string for useless change. + TS := p.GetBakTimeStamp() + if p.taskMap[string(key)] != nil { + p.taskMapBakTS = append(p.taskMapBakTS, TS) + p.taskMapBak = append(p.taskMapBak, "") + } else { + p.taskMapBakTS = append(p.taskMapBakTS, TS) + p.taskMapBak = append(p.taskMapBak, string(key)) + } } p.taskMap[string(key)] = task } From f9a8f7139c1e4d998fcb844d04c266f1857e1fcc Mon Sep 17 00:00:00 2001 From: lawyerphx Date: Thu, 11 Jun 2020 06:15:03 -0700 Subject: [PATCH 18/37] make CI happy. --- planner/core/optimizer.go | 2 +- planner/core/plan.go | 10 +++++----- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/planner/core/optimizer.go b/planner/core/optimizer.go index 04c46ffb35060..699d4a88e9fb0 100644 --- a/planner/core/optimizer.go +++ b/planner/core/optimizer.go @@ -186,7 +186,7 @@ func physicalOptimize(logic LogicalPlan, clock *CountDown) (PhysicalPlan, float6 } // clear the timestamp for taskMap rollback - ClearBakTimeStamp() + clearBakTimeStamp() t, _, err := logic.findBestTask(prop, clock) if err != nil { return nil, 0, err diff --git a/planner/core/plan.go b/planner/core/plan.go index d0f6558de7bf3..5ebbef9c27b2c 100644 --- a/planner/core/plan.go +++ b/planner/core/plan.go @@ -298,15 +298,15 @@ func (p *basePhysicalPlan) ExtractCorrelatedCols() []*expression.CorrelatedColum return nil } -var BakTimeStamp int64 = 0 +var bakTimeStamp int64 = 0 -func ClearBakTimeStamp() { - BakTimeStamp = 0 +func clearBakTimeStamp() { + bakTimeStamp = 0 } func (p *baseLogicalPlan) GetBakTimeStamp() int64 { - BakTimeStamp++ - return BakTimeStamp + bakTimeStamp++ + return bakTimeStamp } func (p *baseLogicalPlan) enableBak() { From 441afd4e27d2cbd6224f2e4fccb551f85e40b54a Mon Sep 17 00:00:00 2001 From: lawyerphx Date: Thu, 11 Jun 2020 06:39:38 -0700 Subject: [PATCH 19/37] avoid data race. --- planner/core/find_best_task.go | 3 ++- planner/core/optimizer.go | 2 -- planner/core/plan.go | 20 +++++++------------- 3 files changed, 9 insertions(+), 16 deletions(-) diff --git a/planner/core/find_best_task.go b/planner/core/find_best_task.go index 6525d5af32fbf..43ce06c4304d2 100644 --- a/planner/core/find_best_task.go +++ b/planner/core/find_best_task.go @@ -15,6 +15,7 @@ package core import ( "math" + "time" "github.com/pingcap/errors" "github.com/pingcap/parser/ast" @@ -168,7 +169,7 @@ func (p *LogicalShowDDLJobs) findBestTask(prop *property.PhysicalProperty, clock } // rebuildChildTasks rebuild the childTasks to make the clock_th combination. -func (p *baseLogicalPlan) rebuildChildTasks(childTasks *[]task, pp PhysicalPlan, childCnts []int64, clock int8, TS int64) error { +func (p *baseLogicalPlan) rebuildChildTasks(childTasks *[]task, pp PhysicalPlan, childCnts []int64, clock int8, TS time.Time) error { // the taskMap of children nodes should be rolled back. for _, child := range p.children { child.rollBackTaskMap(TS) diff --git a/planner/core/optimizer.go b/planner/core/optimizer.go index 699d4a88e9fb0..c1995c8a4f46f 100644 --- a/planner/core/optimizer.go +++ b/planner/core/optimizer.go @@ -185,8 +185,6 @@ func physicalOptimize(logic LogicalPlan, clock *CountDown) (PhysicalPlan, float6 ExpectedCnt: math.MaxFloat64, } - // clear the timestamp for taskMap rollback - clearBakTimeStamp() t, _, err := logic.findBestTask(prop, clock) if err != nil { return nil, 0, err diff --git a/planner/core/plan.go b/planner/core/plan.go index 5ebbef9c27b2c..f4400fa98761c 100644 --- a/planner/core/plan.go +++ b/planner/core/plan.go @@ -17,6 +17,7 @@ import ( "fmt" "math" "strconv" + "time" "github.com/cznic/mathutil" "github.com/pingcap/tidb/expression" @@ -205,7 +206,7 @@ type LogicalPlan interface { enableBak() // rollBackTaskMap roll back all taskMap's logs after TimeStamp TS - rollBackTaskMap(TS int64) + rollBackTaskMap(TS time.Time) } // PhysicalPlan is a tree of the physical operators. @@ -254,7 +255,7 @@ type baseLogicalPlan struct { // taskMapBak and taskMapBakID form a backlog stack of taskMap, used to roll back the task. // taskMapBakTS stores the TimeStamps of logs. taskMapBak []string - taskMapBakTS []int64 + taskMapBakTS []time.Time // needBask indicates whether taskMap needs to be backed. Avoid affecting the speed of the optimizer. needBak bool self LogicalPlan @@ -298,15 +299,8 @@ func (p *basePhysicalPlan) ExtractCorrelatedCols() []*expression.CorrelatedColum return nil } -var bakTimeStamp int64 = 0 - -func clearBakTimeStamp() { - bakTimeStamp = 0 -} - -func (p *baseLogicalPlan) GetBakTimeStamp() int64 { - bakTimeStamp++ - return bakTimeStamp +func (p *baseLogicalPlan) GetBakTimeStamp() time.Time { + return time.Now() } func (p *baseLogicalPlan) enableBak() { @@ -319,7 +313,7 @@ func (p *baseLogicalPlan) enableBak() { } } -func (p *baseLogicalPlan) rollBackTaskMap(TS int64) { +func (p *baseLogicalPlan) rollBackTaskMap(TS time.Time) { if !p.needBak { return } @@ -328,7 +322,7 @@ func (p *baseLogicalPlan) rollBackTaskMap(TS int64) { N := len(p.taskMapBak) for i := 0; i < N; i++ { cur := p.taskMapBak[i] - if p.taskMapBakTS[i] < TS { + if p.taskMapBakTS[i].String() < TS.String() { continue } From 77d4236827c5344142aefc27516dd8970ad45979 Mon Sep 17 00:00:00 2001 From: lawyerphx Date: Mon, 15 Jun 2020 21:03:08 -0700 Subject: [PATCH 20/37] handle invalid plan --- planner/core/find_best_task.go | 29 ++++++++++++++++++++--------- 1 file changed, 20 insertions(+), 9 deletions(-) diff --git a/planner/core/find_best_task.go b/planner/core/find_best_task.go index 43ce06c4304d2..facf545c4fe04 100644 --- a/planner/core/find_best_task.go +++ b/planner/core/find_best_task.go @@ -189,9 +189,12 @@ func (p *baseLogicalPlan) rebuildChildTasks(childTasks *[]task, pp PhysicalPlan, if err != nil { return err } - if curClock != 0 || (childTask != nil && childTask.invalid()) { + if curClock != 0 { return errors.Errorf("CountDown clock is not handled") } + if childTask != nil && childTask.invalid() { + return errors.Errorf("The current plan is invalid, please skip this plan.") + } *childTasks = append(*childTasks, childTask) } return nil @@ -617,8 +620,10 @@ func (ds *DataSource) findBestTask(prop *property.PhysicalProperty, clock *Count if err != nil { return nil, 0, err } - cntPlan += 1 - clock.Dec(1) + if !idxMergeTask.invalid() { + cntPlan += 1 + clock.Dec(1) + } if idxMergeTask.cost() < t.cost() || clock.Empty() { t = idxMergeTask } @@ -660,8 +665,10 @@ func (ds *DataSource) findBestTask(prop *property.PhysicalProperty, clock *Count } else { pointGetTask = ds.convertToBatchPointGet(prop, candidate) } - cntPlan += 1 - clock.Dec(1) + if !pointGetTask.invalid() { + cntPlan += 1 + clock.Dec(1) + } if pointGetTask.cost() < t.cost() || clock.Empty() { t = pointGetTask if clock.Empty() { @@ -682,8 +689,10 @@ func (ds *DataSource) findBestTask(prop *property.PhysicalProperty, clock *Count if err != nil { return nil, 0, err } - cntPlan += 1 - clock.Dec(1) + if !tblTask.invalid() { + cntPlan += 1 + clock.Dec(1) + } if tblTask.cost() < t.cost() || clock.Empty() { t = tblTask } @@ -700,8 +709,10 @@ func (ds *DataSource) findBestTask(prop *property.PhysicalProperty, clock *Count if err != nil { return nil, 0, err } - cntPlan += 1 - clock.Dec(1) + if !idxTask.invalid() { + cntPlan += 1 + clock.Dec(1) + } if idxTask.cost() < t.cost() || clock.Empty() { t = idxTask } From 72a7e689477461a276ea160e044449893898ca1b Mon Sep 17 00:00:00 2001 From: lawyerphx Date: Tue, 16 Jun 2020 16:58:34 +0800 Subject: [PATCH 21/37] Update find_best_task.go --- planner/core/find_best_task.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/planner/core/find_best_task.go b/planner/core/find_best_task.go index 5fa0e46b405e5..d156902b7474d 100644 --- a/planner/core/find_best_task.go +++ b/planner/core/find_best_task.go @@ -80,7 +80,7 @@ func (c *CountDown) Dec(x int8) { } } -// Empty indicates which the CountDown is clear now. +// Empty indicates whether the CountDown is clear now. func (c *CountDown) Empty() bool { return *c == 0 } From af965e5820b5c83ec2cad73d89dfe42c0efe954e Mon Sep 17 00:00:00 2001 From: lawyerphx Date: Tue, 16 Jun 2020 02:09:59 -0700 Subject: [PATCH 22/37] avoid int8 exceed. --- planner/core/find_best_task.go | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/planner/core/find_best_task.go b/planner/core/find_best_task.go index facf545c4fe04..4baf37831fed8 100644 --- a/planner/core/find_best_task.go +++ b/planner/core/find_best_task.go @@ -169,20 +169,20 @@ func (p *LogicalShowDDLJobs) findBestTask(prop *property.PhysicalProperty, clock } // rebuildChildTasks rebuild the childTasks to make the clock_th combination. -func (p *baseLogicalPlan) rebuildChildTasks(childTasks *[]task, pp PhysicalPlan, childCnts []int64, clock int8, TS time.Time) error { +func (p *baseLogicalPlan) rebuildChildTasks(childTasks *[]task, pp PhysicalPlan, childCnts []int64, clock int64, TS time.Time) error { // the taskMap of children nodes should be rolled back. for _, child := range p.children { child.rollBackTaskMap(TS) } - var multAll int8 = 1 + var multAll int64 = 1 var curClock CountDown for _, x := range childCnts { - multAll *= int8(x) + multAll *= x } *childTasks = (*childTasks)[:0] for j, child := range p.children { - multAll /= int8(childCnts[j]) + multAll /= childCnts[j] curClock = CountDown((clock-1)/multAll + 1) childTask, _, err := child.findBestTask(pp.GetChildReqProps(j), &curClock) clock = (clock-1)%multAll + 1 @@ -233,7 +233,7 @@ func (p *baseLogicalPlan) enumeratePhysicalPlans4Task(physicalPlans []PhysicalPl if clock.IsForce() && int64(*clock) <= curCntPlan { curCntPlan = int64(*clock) - err := p.rebuildChildTasks(&childTasks, pp, childCnts, int8(*clock), TimeStampNow) + err := p.rebuildChildTasks(&childTasks, pp, childCnts, int64(*clock), TimeStampNow) if err != nil { return nil, 0, err } From 5b5ab30ce49adc12b506764ced2e86712b7cc94e Mon Sep 17 00:00:00 2001 From: lawyerphx Date: Tue, 16 Jun 2020 03:43:57 -0700 Subject: [PATCH 23/37] address comments --- planner/core/common_plans.go | 1 + planner/core/find_best_task.go | 8 +++----- planner/core/optimizer.go | 1 + planner/core/plan.go | 26 +++++--------------------- sessionctx/variable/session.go | 4 ++++ sessionctx/variable/tidb_vars.go | 1 + 6 files changed, 15 insertions(+), 26 deletions(-) diff --git a/planner/core/common_plans.go b/planner/core/common_plans.go index dd5c1f5d1b5df..b363bd137a679 100644 --- a/planner/core/common_plans.go +++ b/planner/core/common_plans.go @@ -837,6 +837,7 @@ func (e *Explain) RenderResult() error { e.Rows = append(e.Rows, []string{hint.RestoreOptimizerHints(hints)}) case ast.ExplainFormatVerbose: //TODO: from PHX add expain format = verbose. here + return errors.Errorf("explain format '%s' is not supported now", e.Format) default: return errors.Errorf("explain format '%s' is not supported now", e.Format) } diff --git a/planner/core/find_best_task.go b/planner/core/find_best_task.go index e473d8fcdbfc0..40947a975a706 100644 --- a/planner/core/find_best_task.go +++ b/planner/core/find_best_task.go @@ -231,6 +231,7 @@ func (p *baseLogicalPlan) enumeratePhysicalPlans4Task(physicalPlans []PhysicalPl continue } + // If the aim plan can be found in this physicalPlan(pp), rebuild childTasks to make the corresponding combination. if clock.IsForce() && int64(*clock) <= curCntPlan { curCntPlan = int64(*clock) err := p.rebuildChildTasks(&childTasks, pp, childCnts, int64(*clock), TimeStampNow) @@ -273,9 +274,6 @@ func (p *baseLogicalPlan) enumeratePhysicalPlans4Task(physicalPlans []PhysicalPl func (p *baseLogicalPlan) findBestTask(prop *property.PhysicalProperty, clock *CountDown) (bestTask task, cntPlan int64, err error) { // If p is an inner plan in an IndexJoin, the IndexJoin will generate an inner plan by itself, // and set inner child prop nil, so here we do nothing. - if clock.IsForce() { - p.enableBak() - } if prop == nil { return nil, 0, nil } @@ -284,7 +282,7 @@ func (p *baseLogicalPlan) findBestTask(prop *property.PhysicalProperty, clock *C bestTask = p.getTask(prop) if bestTask != nil { clock.Dec(1) - if p.needBak { + if p.ctx.GetSessionVars().TaskMapNeedBackUp { // Ensure that a key will be pushed into taskMapBak every time we call the function. p.storeTask(prop, bestTask) } @@ -567,7 +565,7 @@ func (ds *DataSource) findBestTask(prop *property.PhysicalProperty, clock *Count cntPlan = 1 clock.Dec(1) // Ensure that a key will be pushed into taskMapBak every time we call the function. - if ds.needBak { + if ds.ctx.GetSessionVars().TaskMapNeedBackUp { ds.storeTask(prop, t) } return diff --git a/planner/core/optimizer.go b/planner/core/optimizer.go index c1995c8a4f46f..95664f2ef19e6 100644 --- a/planner/core/optimizer.go +++ b/planner/core/optimizer.go @@ -185,6 +185,7 @@ func physicalOptimize(logic LogicalPlan, clock *CountDown) (PhysicalPlan, float6 ExpectedCnt: math.MaxFloat64, } + logic.SCtx().GetSessionVars().TaskMapNeedBackUp = clock.IsForce() t, _, err := logic.findBestTask(prop, clock) if err != nil { return nil, 0, err diff --git a/planner/core/plan.go b/planner/core/plan.go index f4400fa98761c..fb1e3b56ee7a6 100644 --- a/planner/core/plan.go +++ b/planner/core/plan.go @@ -202,9 +202,6 @@ type LogicalPlan interface { // SetChild sets the ith child for the plan. SetChild(i int, child LogicalPlan) - // enableBak sets the needBak of logical plan tree to true. - enableBak() - // rollBackTaskMap roll back all taskMap's logs after TimeStamp TS rollBackTaskMap(TS time.Time) } @@ -256,11 +253,9 @@ type baseLogicalPlan struct { // taskMapBakTS stores the TimeStamps of logs. taskMapBak []string taskMapBakTS []time.Time - // needBask indicates whether taskMap needs to be backed. Avoid affecting the speed of the optimizer. - needBak bool - self LogicalPlan - maxOneRow bool - children []LogicalPlan + self LogicalPlan + maxOneRow bool + children []LogicalPlan } func (p *baseLogicalPlan) MaxOneRow() bool { @@ -303,18 +298,8 @@ func (p *baseLogicalPlan) GetBakTimeStamp() time.Time { return time.Now() } -func (p *baseLogicalPlan) enableBak() { - if p.needBak { - return - } - p.needBak = true - for _, child := range p.children { - child.enableBak() - } -} - func (p *baseLogicalPlan) rollBackTaskMap(TS time.Time) { - if !p.needBak { + if !p.ctx.GetSessionVars().TaskMapNeedBackUp { return } if len(p.taskMapBak) > 0 { @@ -351,7 +336,7 @@ func (p *baseLogicalPlan) getTask(prop *property.PhysicalProperty) task { func (p *baseLogicalPlan) storeTask(prop *property.PhysicalProperty, task task) { key := prop.HashCode() - if p.needBak { + if p.ctx.GetSessionVars().TaskMapNeedBackUp { // when we storeTask, we have taskMap[key] is nil. // so to roll back, we need only store the key and set taskMap[key] to nil. // empty string for useless change. @@ -422,7 +407,6 @@ func newBaseLogicalPlan(ctx sessionctx.Context, tp string, self LogicalPlan, off return baseLogicalPlan{ taskMap: make(map[string]task), taskMapBak: make([]string, 0, 100), - needBak: false, basePlan: newBasePlan(ctx, tp, offset), self: self, } diff --git a/sessionctx/variable/session.go b/sessionctx/variable/session.go index a057f5b65adea..99de65e68a41e 100644 --- a/sessionctx/variable/session.go +++ b/sessionctx/variable/session.go @@ -643,6 +643,9 @@ type SessionVars struct { // EnableSlowLogMasking indicates that whether masking the query data when log slow query. EnableSlowLogMasking bool + + // TaskMapNeedBackUp indicates that whether we need to back up taskMap during physical optimizing. + TaskMapNeedBackUp bool } // PreparedParams contains the parameters of the current prepared statement when executing it. @@ -734,6 +737,7 @@ func NewSessionVars() *SessionVars { AllowAutoRandExplicitInsert: DefTiDBAllowAutoRandExplicitInsert, EnableClusteredIndex: DefTiDBEnableClusteredIndex, EnableSlowLogMasking: DefTiDBSlowLogMasking, + TaskMapNeedBackUp: DefTaskMapNeedBackUp, } vars.KVVars = kv.NewVariables(&vars.Killed) vars.Concurrency = Concurrency{ diff --git a/sessionctx/variable/tidb_vars.go b/sessionctx/variable/tidb_vars.go index ba7122c3df1a2..028dacaa55488 100644 --- a/sessionctx/variable/tidb_vars.go +++ b/sessionctx/variable/tidb_vars.go @@ -499,6 +499,7 @@ const ( DefTiDBAllowAutoRandExplicitInsert = false DefTiDBEnableClusteredIndex = false DefTiDBSlowLogMasking = false + DefTaskMapNeedBackUp = false ) // Process global variables. From 86712e03948c982502f02e1d9fb3c782a6b77a98 Mon Sep 17 00:00:00 2001 From: lawyerphx Date: Thu, 18 Jun 2020 17:25:11 +0800 Subject: [PATCH 24/37] Update planner/core/plan.go Co-authored-by: Kenan Yao --- planner/core/plan.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/planner/core/plan.go b/planner/core/plan.go index fb1e3b56ee7a6..2046790acdcab 100644 --- a/planner/core/plan.go +++ b/planner/core/plan.go @@ -317,7 +317,7 @@ func (p *baseLogicalPlan) rollBackTaskMap(TS time.Time) { i-- N-- - // if cur is not an invalid log, then roll back. + // if cur is a valid log, then roll back. if cur != "" { p.taskMap[cur] = nil } From efc37539eb3d2fcc85724167da29a2897ed7d6b7 Mon Sep 17 00:00:00 2001 From: lawyerphx Date: Thu, 18 Jun 2020 17:28:33 +0800 Subject: [PATCH 25/37] Update planner/core/find_best_task.go Co-authored-by: Kenan Yao --- planner/core/find_best_task.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/planner/core/find_best_task.go b/planner/core/find_best_task.go index ca1b645d1a390..391110f67186e 100644 --- a/planner/core/find_best_task.go +++ b/planner/core/find_best_task.go @@ -174,7 +174,7 @@ func (p *baseLogicalPlan) rebuildChildTasks(childTasks *[]task, pp PhysicalPlan, child.rollBackTaskMap(TS) } - var multAll int64 = 1 + multAll := int64(1) var curClock CountDown for _, x := range childCnts { multAll *= x From ea3ad84cfc1d2d6acfbaad362d17176b19a439e8 Mon Sep 17 00:00:00 2001 From: lawyerphx Date: Thu, 18 Jun 2020 17:55:25 +0800 Subject: [PATCH 26/37] Update planner/core/find_best_task.go Co-authored-by: Kenan Yao --- planner/core/find_best_task.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/planner/core/find_best_task.go b/planner/core/find_best_task.go index 391110f67186e..0e27fa1e154e2 100644 --- a/planner/core/find_best_task.go +++ b/planner/core/find_best_task.go @@ -230,7 +230,7 @@ func (p *baseLogicalPlan) enumeratePhysicalPlans4Task(physicalPlans []PhysicalPl continue } - // If the aim plan can be found in this physicalPlan(pp), rebuild childTasks to make the corresponding combination. + // If the target plan can be found in this physicalPlan(pp), rebuild childTasks to build the corresponding combination. if clock.IsForce() && int64(*clock) <= curCntPlan { curCntPlan = int64(*clock) err := p.rebuildChildTasks(&childTasks, pp, childCnts, int64(*clock), TimeStampNow) From 9670dc0dca2fe3da9a7ee22d150e1df4ee6ec339 Mon Sep 17 00:00:00 2001 From: lawyerphx Date: Thu, 18 Jun 2020 03:51:58 -0700 Subject: [PATCH 27/37] address comments --- planner/core/find_best_task.go | 19 +++++-------------- planner/core/optimizer.go | 5 ++--- planner/core/plan.go | 23 +++++++++++------------ planner/core/plan_test.go | 10 ++++++---- planner/optimize.go | 8 ++++---- sessionctx/stmtctx/stmtctx.go | 5 +++++ sessionctx/variable/session.go | 4 ---- sessionctx/variable/tidb_vars.go | 1 - 8 files changed, 33 insertions(+), 42 deletions(-) diff --git a/planner/core/find_best_task.go b/planner/core/find_best_task.go index ca1b645d1a390..caee0e67701df 100644 --- a/planner/core/find_best_task.go +++ b/planner/core/find_best_task.go @@ -136,7 +136,7 @@ func (p *LogicalTableDual) findBestTask(prop *property.PhysicalProperty, clock * // If the required property is not empty and the row count > 1, // we cannot ensure this required property. // But if the row count is 0 or 1, we don't need to care about the property. - if (!prop.IsEmpty() && p.RowCount > 1) || *clock == 0 { + if (!prop.IsEmpty() && p.RowCount > 1) || clock.Empty() { return invalidTask, 0, nil } dual := PhysicalTableDual{ @@ -201,11 +201,10 @@ func (p *baseLogicalPlan) rebuildChildTasks(childTasks *[]task, pp PhysicalPlan, func (p *baseLogicalPlan) enumeratePhysicalPlans4Task(physicalPlans []PhysicalPlan, prop *property.PhysicalProperty, clock *CountDown) (task, int64, error) { var bestTask task = invalidTask - var curCntPlan int64 - var cntPlan int64 = 0 + var curCntPlan, cntPlan int64 childTasks := make([]task, 0, len(p.children)) - // childCnts : record the number of tasks founded during generating childTasks childCnts := make([]int64, len(p.children)) + cntPlan = 0 for _, pp := range physicalPlans { // find best child tasks firstly. childTasks = childTasks[:0] @@ -274,17 +273,13 @@ func (p *baseLogicalPlan) findBestTask(prop *property.PhysicalProperty, clock *C // If p is an inner plan in an IndexJoin, the IndexJoin will generate an inner plan by itself, // and set inner child prop nil, so here we do nothing. if prop == nil { - return nil, 0, nil + return nil, 1, nil } // Look up the task with this prop in the task map. // It's used to reduce double counting. bestTask = p.getTask(prop) if bestTask != nil { clock.Dec(1) - if p.ctx.GetSessionVars().TaskMapNeedBackUp { - // Ensure that a key will be pushed into taskMapBak every time we call the function. - p.storeTask(prop, bestTask) - } return bestTask, 1, nil } @@ -556,17 +551,13 @@ func (ds *DataSource) findBestTask(prop *property.PhysicalProperty, clock *Count // If ds is an inner plan in an IndexJoin, the IndexJoin will generate an inner plan by itself, // and set inner child prop nil, so here we do nothing. if prop == nil { - return nil, 0, nil + return nil, 1, nil } t = ds.getTask(prop) if t != nil { cntPlan = 1 clock.Dec(1) - // Ensure that a key will be pushed into taskMapBak every time we call the function. - if ds.ctx.GetSessionVars().TaskMapNeedBackUp { - ds.storeTask(prop, t) - } return } var cnt int64 diff --git a/planner/core/optimizer.go b/planner/core/optimizer.go index 95664f2ef19e6..27ffb31a9a2b3 100644 --- a/planner/core/optimizer.go +++ b/planner/core/optimizer.go @@ -132,7 +132,7 @@ func DoOptimize(ctx context.Context, sctx sessionctx.Context, flag uint64, logic if !AllowCartesianProduct.Load() && existsCartesianProduct(logic) { return nil, 0, errors.Trace(ErrCartesianProductUnsupported) } - var clock = CountDown(sctx.GetSessionVars().StmtCtx.StmtHints.ForceNthPlan) + clock := CountDown(sctx.GetSessionVars().StmtCtx.StmtHints.ForceNthPlan) if clock == 0 { clock = -1 } @@ -185,13 +185,12 @@ func physicalOptimize(logic LogicalPlan, clock *CountDown) (PhysicalPlan, float6 ExpectedCnt: math.MaxFloat64, } - logic.SCtx().GetSessionVars().TaskMapNeedBackUp = clock.IsForce() t, _, err := logic.findBestTask(prop, clock) if err != nil { return nil, 0, err } if *clock > 0 { - return nil, 0, ErrInternal.GenWithStackByArgs("The parameter of nth_plan() is out of range.") + logic.SCtx().GetSessionVars().StmtCtx.AppendWarning(errors.Errorf("The parameter of nth_plan() is out of range.")) } if t.invalid() { return nil, 0, ErrInternal.GenWithStackByArgs("Can't find a proper physical plan for this query") diff --git a/planner/core/plan.go b/planner/core/plan.go index 2046790acdcab..2c57a21fc3686 100644 --- a/planner/core/plan.go +++ b/planner/core/plan.go @@ -156,6 +156,7 @@ type LogicalPlan interface { // Some logical plans will convert the children to the physical plans in different ways, and return the one // with the lowest cost and how many plans are found in this function. // clock is a counter for planner to force a plan. + // if clock > 0, the clock_th plan generated in this function will be returned. // if clock = 0, the plan generated in this function will not be considered. // if clock = -1, then we will not force plan. findBestTask(prop *property.PhysicalProperty, clock *CountDown) (task, int64, error) @@ -249,9 +250,9 @@ type baseLogicalPlan struct { basePlan taskMap map[string]task - // taskMapBak and taskMapBakID form a backlog stack of taskMap, used to roll back the task. + // taskMapBak forms a backlog stack of taskMap, used to roll back the taskMap. + taskMapBak []string // taskMapBakTS stores the TimeStamps of logs. - taskMapBak []string taskMapBakTS []time.Time self LogicalPlan maxOneRow bool @@ -299,7 +300,7 @@ func (p *baseLogicalPlan) GetBakTimeStamp() time.Time { } func (p *baseLogicalPlan) rollBackTaskMap(TS time.Time) { - if !p.ctx.GetSessionVars().TaskMapNeedBackUp { + if !p.ctx.GetSessionVars().StmtCtx.StmtHints.TaskMapNeedBackUp() { return } if len(p.taskMapBak) > 0 { @@ -324,7 +325,6 @@ func (p *baseLogicalPlan) rollBackTaskMap(TS time.Time) { } } for _, child := range p.children { - // only baseLogicalPlan and DataSource will be affected. child.rollBackTaskMap(TS) } } @@ -336,10 +336,8 @@ func (p *baseLogicalPlan) getTask(prop *property.PhysicalProperty) task { func (p *baseLogicalPlan) storeTask(prop *property.PhysicalProperty, task task) { key := prop.HashCode() - if p.ctx.GetSessionVars().TaskMapNeedBackUp { - // when we storeTask, we have taskMap[key] is nil. - // so to roll back, we need only store the key and set taskMap[key] to nil. - // empty string for useless change. + if p.ctx.GetSessionVars().StmtCtx.StmtHints.TaskMapNeedBackUp() { + // Empty string for useless change. TS := p.GetBakTimeStamp() if p.taskMap[string(key)] != nil { p.taskMapBakTS = append(p.taskMapBakTS, TS) @@ -405,10 +403,11 @@ func newBasePlan(ctx sessionctx.Context, tp string, offset int) basePlan { func newBaseLogicalPlan(ctx sessionctx.Context, tp string, self LogicalPlan, offset int) baseLogicalPlan { return baseLogicalPlan{ - taskMap: make(map[string]task), - taskMapBak: make([]string, 0, 100), - basePlan: newBasePlan(ctx, tp, offset), - self: self, + taskMap: make(map[string]task), + taskMapBak: make([]string, 0, 10), + taskMapBakTS: make([]time.Time, 0, 10), + basePlan: newBasePlan(ctx, tp, offset), + self: self, } } diff --git a/planner/core/plan_test.go b/planner/core/plan_test.go index 01d45ca866878..0c4249c43815a 100644 --- a/planner/core/plan_test.go +++ b/planner/core/plan_test.go @@ -265,8 +265,9 @@ func (s *testPlanNormalize) TestNthPlanHint(c *C) { "└─Selection_21(Probe) 0.01 cop[tikv] eq(test.tt.b, 1)", " └─TableRowIDScan_20 10.00 cop[tikv] table:tt keep order:false, stats:pseudo")) - _, err := tk.Exec("explain select /*+nth_plan(4)*/ * from tt where a=1 and b=1;") - c.Assert(err, NotNil) + tk.MustExec("explain select /*+nth_plan(4)*/ * from tt where a=1 and b=1;") + tk.MustQuery("show warnings").Check(testkit.Rows( + "Warning 1105 The parameter of nth_plan() is out of range.")) // test hints for nth_plan(x) tk.MustExec("drop table if exists t") @@ -278,8 +279,9 @@ func (s *testPlanNormalize) TestNthPlanHint(c *C) { tk.MustQuery("explain format='hint' select /*+ nth_plan(2) */ * from t where a=1 and b=1").Check(testkit.Rows( "use_index(@`sel_1` `test`.`t` `a_2`), nth_plan(2)")) - _, err = tk.Exec("explain format='hint' select /*+ nth_plan(3) */ * from t where a=1 and b=1") - c.Assert(err, NotNil) + tk.MustExec("explain format='hint' select /*+ nth_plan(3) */ * from t where a=1 and b=1") + tk.MustQuery("show warnings").Check(testkit.Rows( + "Warning 1105 The parameter of nth_plan() is out of range.")) // test warning for multiply hints tk.MustQuery("explain format='hint' select /*+ nth_plan(1) nth_plan(2) */ * from t where a=1 and b=1").Check(testkit.Rows( diff --git a/planner/optimize.go b/planner/optimize.go index a45b0375b7218..eae8d4a9262cc 100644 --- a/planner/optimize.go +++ b/planner/optimize.go @@ -416,12 +416,12 @@ func handleStmtHints(hints []*ast.TableOptimizerHint) (stmtHints stmtctx.StmtHin } stmtHints.ForceNthPlan = forceNthPlan.HintData.(int64) if stmtHints.ForceNthPlan < 1 { - stmtHints.ForceNthPlan = 1 - warn := errors.Errorf("the hintdata for NTH_PLAN() is too small, set to 1 automatically.") + stmtHints.ForceNthPlan = -1 + warn := errors.Errorf("the hintdata for NTH_PLAN() is too small, hint ignored.") warns = append(warns, warn) } else if stmtHints.ForceNthPlan > 100 { - stmtHints.ForceNthPlan = 100 - warn := errors.Errorf("the hintdata for NTH_PLAN() is too big, set to 100 automatically.") + stmtHints.ForceNthPlan = -1 + warn := errors.Errorf("the hintdata for NTH_PLAN() is too big, hint ignored.") warns = append(warns, warn) } } else { diff --git a/sessionctx/stmtctx/stmtctx.go b/sessionctx/stmtctx/stmtctx.go index 178c736bb66c6..c70a3d10dbc63 100644 --- a/sessionctx/stmtctx/stmtctx.go +++ b/sessionctx/stmtctx/stmtctx.go @@ -178,6 +178,11 @@ type StmtHints struct { HasEnableCascadesPlannerHint bool } +// TaskMapNeedBackUp indicates that whether we need to back up taskMap during physical optimizing. +func (sh *StmtHints) TaskMapNeedBackUp() bool { + return sh.ForceNthPlan != -1 +} + // GetNowTsCached getter for nowTs, if not set get now time and cache it func (sc *StatementContext) GetNowTsCached() time.Time { if !sc.stmtTimeCached { diff --git a/sessionctx/variable/session.go b/sessionctx/variable/session.go index 6eb6290821ef7..95b142ffa70fa 100644 --- a/sessionctx/variable/session.go +++ b/sessionctx/variable/session.go @@ -643,9 +643,6 @@ type SessionVars struct { // EnableSlowLogMasking indicates that whether masking the query data when log slow query. EnableSlowLogMasking bool - - // TaskMapNeedBackUp indicates that whether we need to back up taskMap during physical optimizing. - TaskMapNeedBackUp bool } // PreparedParams contains the parameters of the current prepared statement when executing it. @@ -737,7 +734,6 @@ func NewSessionVars() *SessionVars { AllowAutoRandExplicitInsert: DefTiDBAllowAutoRandExplicitInsert, EnableClusteredIndex: DefTiDBEnableClusteredIndex, EnableSlowLogMasking: DefTiDBSlowLogMasking, - TaskMapNeedBackUp: DefTaskMapNeedBackUp, } vars.KVVars = kv.NewVariables(&vars.Killed) vars.Concurrency = Concurrency{ diff --git a/sessionctx/variable/tidb_vars.go b/sessionctx/variable/tidb_vars.go index babff584e2501..4a19892a563f2 100644 --- a/sessionctx/variable/tidb_vars.go +++ b/sessionctx/variable/tidb_vars.go @@ -500,7 +500,6 @@ const ( DefTiDBAllowAutoRandExplicitInsert = false DefTiDBEnableClusteredIndex = false DefTiDBSlowLogMasking = false - DefTaskMapNeedBackUp = false ) // Process global variables. From 6dc07f446eb4c33f4c6f7be73bff5a1e82103fe5 Mon Sep 17 00:00:00 2001 From: lawyerphx Date: Thu, 18 Jun 2020 04:03:52 -0700 Subject: [PATCH 28/37] refine comments. --- planner/core/common_plans.go | 2 +- planner/core/find_best_task.go | 22 +++++++++++----------- planner/core/plan.go | 18 +++++++++--------- planner/core/plan_test.go | 10 +++++----- sessionctx/stmtctx/stmtctx.go | 2 +- 5 files changed, 27 insertions(+), 27 deletions(-) diff --git a/planner/core/common_plans.go b/planner/core/common_plans.go index ff32e60d03352..78baa5c9b31b2 100644 --- a/planner/core/common_plans.go +++ b/planner/core/common_plans.go @@ -836,7 +836,7 @@ func (e *Explain) RenderResult() error { hints = append(hints, hint.ExtractTableHintsFromStmtNode(e.ExecStmt, nil)...) e.Rows = append(e.Rows, []string{hint.RestoreOptimizerHints(hints)}) case ast.ExplainFormatVerbose: - //TODO: from PHX add expain format = verbose. here + // TODO: add explain format = verbose here. return errors.Errorf("explain format '%s' is not supported now", e.Format) default: return errors.Errorf("explain format '%s' is not supported now", e.Format) diff --git a/planner/core/find_best_task.go b/planner/core/find_best_task.go index caee0e67701df..21ad9520686a1 100644 --- a/planner/core/find_best_task.go +++ b/planner/core/find_best_task.go @@ -65,10 +65,10 @@ var aggFuncFactor = map[string]float64{ // CountDown is used in hint nth_plan() to indicate which plan to use. type CountDown int8 -// Sign is the default value of CountDown, indicating that optimizer needn't to force a plan. +// Sign is the default value of CountDown, indicating that optimizer needn't force a plan. var Sign CountDown = -1 -// Dec minus countdown by x +// Dec minus CountDown value by x. func (c *CountDown) Dec(x int8) { if *c <= 0 { return @@ -167,9 +167,9 @@ func (p *LogicalShowDDLJobs) findBestTask(prop *property.PhysicalProperty, clock return &rootTask{p: pShow}, 1, nil } -// rebuildChildTasks rebuild the childTasks to make the clock_th combination. +// The function rebuildChildTasks rebuilds the childTasks to make the clock_th combination. func (p *baseLogicalPlan) rebuildChildTasks(childTasks *[]task, pp PhysicalPlan, childCnts []int64, clock int64, TS time.Time) error { - // the taskMap of children nodes should be rolled back. + // The taskMap of children nodes should be rolled back first. for _, child := range p.children { child.rollBackTaskMap(TS) } @@ -206,9 +206,9 @@ func (p *baseLogicalPlan) enumeratePhysicalPlans4Task(physicalPlans []PhysicalPl childCnts := make([]int64, len(p.children)) cntPlan = 0 for _, pp := range physicalPlans { - // find best child tasks firstly. + // Find best child tasks firstly. childTasks = childTasks[:0] - // curCntPlan record the number of possible plan for pp + // The curCntPlan records the number of possible plans for pp curCntPlan = 1 TimeStampNow := p.GetBakTimeStamp() for j, child := range p.children { @@ -238,21 +238,21 @@ func (p *baseLogicalPlan) enumeratePhysicalPlans4Task(physicalPlans []PhysicalPl } } - // combine best child tasks with parent physical plan. + // Combine best child tasks with parent physical plan. curTask := pp.attach2Task(childTasks...) - // enforce curTask property + // Enforce curTask property if prop.Enforced { curTask = enforceProperty(prop, curTask, p.basePlan.ctx) } - // optimize by shuffle executor to running in parallel manner. + // Optimize by shuffle executor to running in parallel manner. if prop.IsEmpty() { - // currently, we don not regard shuffled plan as a new plan. + // Currently, we don not regard shuffled plan as a new plan. curTask = optimizeByShuffle(pp, curTask, p.basePlan.ctx) } - // get the most efficient one. + // Get the most efficient one. cntPlan += curCntPlan clock.Dec(int8(curCntPlan)) diff --git a/planner/core/plan.go b/planner/core/plan.go index 2c57a21fc3686..b90da911c7d2e 100644 --- a/planner/core/plan.go +++ b/planner/core/plan.go @@ -154,11 +154,11 @@ type LogicalPlan interface { // findBestTask converts the logical plan to the physical plan. It's a new interface. // It is called recursively from the parent to the children to create the result physical plan. // Some logical plans will convert the children to the physical plans in different ways, and return the one - // with the lowest cost and how many plans are found in this function. + // With the lowest cost and how many plans are found in this function. // clock is a counter for planner to force a plan. - // if clock > 0, the clock_th plan generated in this function will be returned. - // if clock = 0, the plan generated in this function will not be considered. - // if clock = -1, then we will not force plan. + // If clock > 0, the clock_th plan generated in this function will be returned. + // If clock = 0, the plan generated in this function will not be considered. + // If clock = -1, then we will not force plan. findBestTask(prop *property.PhysicalProperty, clock *CountDown) (task, int64, error) // BuildKeyInfo will collect the information of unique keys into schema. @@ -203,7 +203,7 @@ type LogicalPlan interface { // SetChild sets the ith child for the plan. SetChild(i int, child LogicalPlan) - // rollBackTaskMap roll back all taskMap's logs after TimeStamp TS + // rollBackTaskMap roll back all taskMap's logs after TimeStamp TS. rollBackTaskMap(TS time.Time) } @@ -252,7 +252,7 @@ type baseLogicalPlan struct { taskMap map[string]task // taskMapBak forms a backlog stack of taskMap, used to roll back the taskMap. taskMapBak []string - // taskMapBakTS stores the TimeStamps of logs. + // taskMapBakTS stores the timestamps of logs. taskMapBakTS []time.Time self LogicalPlan maxOneRow bool @@ -304,7 +304,7 @@ func (p *baseLogicalPlan) rollBackTaskMap(TS time.Time) { return } if len(p.taskMapBak) > 0 { - // rollback all the logs with TimeStamp TS + // Rollback all the logs with TimeStamp TS. N := len(p.taskMapBak) for i := 0; i < N; i++ { cur := p.taskMapBak[i] @@ -312,13 +312,13 @@ func (p *baseLogicalPlan) rollBackTaskMap(TS time.Time) { continue } - // remove the i_th log + // Remove the i_th log. p.taskMapBak = append(p.taskMapBak[:i], p.taskMapBak[i+1:]...) p.taskMapBakTS = append(p.taskMapBakTS[:i], p.taskMapBakTS[i+1:]...) i-- N-- - // if cur is a valid log, then roll back. + // If cur is a valid log, then roll back. if cur != "" { p.taskMap[cur] = nil } diff --git a/planner/core/plan_test.go b/planner/core/plan_test.go index 0c4249c43815a..5047635dc7c32 100644 --- a/planner/core/plan_test.go +++ b/planner/core/plan_test.go @@ -240,7 +240,7 @@ func (s *testPlanNormalize) TestNthPlanHint(c *C) { tk.MustExec("create table tt (a int,b int, index(a), index(b));") tk.MustExec("insert into tt values (1, 1), (2, 2), (3, 4)") - // test the explained plans. + // Test the explained plans. tk.MustQuery("explain select /*+nth_plan(1)*/ * from tt where a=1 and b=1;").Check(testkit.Rows( "TableReader_18 0.01 root data:Selection_17", "└─Selection_17 0.01 cop[tikv] eq(test.tt.a, 1), eq(test.tt.b, 1)", @@ -258,7 +258,7 @@ func (s *testPlanNormalize) TestNthPlanHint(c *C) { "└─Selection_25(Probe) 0.01 cop[tikv] eq(test.tt.a, 1)", " └─TableRowIDScan_24 10.00 cop[tikv] table:tt keep order:false, stats:pseudo")) - // with the same param in nth_plan, we should get the same plan. + // With the same param in nth_plan, we should get the same plan. tk.MustQuery("explain select /*+nth_plan(2)*/ * from tt where a=1 and b=1;").Check(testkit.Rows( "IndexLookUp_22 0.01 root ", "├─IndexRangeScan_19(Build) 10.00 cop[tikv] table:tt, index:a(a) range:[1,1], keep order:false, stats:pseudo", @@ -269,7 +269,7 @@ func (s *testPlanNormalize) TestNthPlanHint(c *C) { tk.MustQuery("show warnings").Check(testkit.Rows( "Warning 1105 The parameter of nth_plan() is out of range.")) - // test hints for nth_plan(x) + // Test hints for nth_plan(x). tk.MustExec("drop table if exists t") tk.MustExec("create table t (a int, b int, c int, index(a), index(b), index(a,b))") tk.MustQuery("explain format='hint' select * from t where a=1 and b=1").Check(testkit.Rows( @@ -283,14 +283,14 @@ func (s *testPlanNormalize) TestNthPlanHint(c *C) { tk.MustQuery("show warnings").Check(testkit.Rows( "Warning 1105 The parameter of nth_plan() is out of range.")) - // test warning for multiply hints + // Test warning for multiply hints. tk.MustQuery("explain format='hint' select /*+ nth_plan(1) nth_plan(2) */ * from t where a=1 and b=1").Check(testkit.Rows( "use_index(@`sel_1` `test`.`t` `a_2`), nth_plan(1), nth_plan(2)")) tk.MustQuery("show warnings").Check(testkit.Rows( "Warning 1105 NTH_PLAN() is defined more than once, only the last definition takes effect: NTH_PLAN(2)", "Warning 1105 NTH_PLAN() is defined more than once, only the last definition takes effect: NTH_PLAN(2)")) - // test the correctness of generated plans. + // Test the correctness of generated plans. tk.MustExec("insert into t values (1,1,1)") tk.MustQuery("select /*+ nth_plan(1) */ * from t where a=1 and b=1;").Check(testkit.Rows( "1 1 1")) diff --git a/sessionctx/stmtctx/stmtctx.go b/sessionctx/stmtctx/stmtctx.go index c70a3d10dbc63..977e69f4ef838 100644 --- a/sessionctx/stmtctx/stmtctx.go +++ b/sessionctx/stmtctx/stmtctx.go @@ -167,7 +167,7 @@ type StmtHints struct { // EnableCascadesPlanner is use cascades planner for a single query only. EnableCascadesPlanner bool // ForceNthPlan indicates the countDown number for finding physical plan. - // -1 for disable + // -1 for disable. ForceNthPlan int64 // Hint flags From ac9444be06e196642494d929b3e005ce6edfb9bf Mon Sep 17 00:00:00 2001 From: lawyerphx Date: Thu, 18 Jun 2020 21:13:57 -0700 Subject: [PATCH 29/37] address comments from Zhang Jian. --- planner/core/common_plans.go | 3 -- planner/core/physical_plan_test.go | 41 +++++++++++++++++++++++ planner/core/plan_test.go | 25 -------------- planner/core/testdata/plan_suite_in.json | 9 +++++ planner/core/testdata/plan_suite_out.json | 21 ++++++++++++ 5 files changed, 71 insertions(+), 28 deletions(-) diff --git a/planner/core/common_plans.go b/planner/core/common_plans.go index 78baa5c9b31b2..b49ce48b10209 100644 --- a/planner/core/common_plans.go +++ b/planner/core/common_plans.go @@ -835,9 +835,6 @@ func (e *Explain) RenderResult() error { hints := GenHintsFromPhysicalPlan(e.TargetPlan) hints = append(hints, hint.ExtractTableHintsFromStmtNode(e.ExecStmt, nil)...) e.Rows = append(e.Rows, []string{hint.RestoreOptimizerHints(hints)}) - case ast.ExplainFormatVerbose: - // TODO: add explain format = verbose here. - return errors.Errorf("explain format '%s' is not supported now", e.Format) default: return errors.Errorf("explain format '%s' is not supported now", e.Format) } diff --git a/planner/core/physical_plan_test.go b/planner/core/physical_plan_test.go index d28681262225d..7b868c575dd51 100644 --- a/planner/core/physical_plan_test.go +++ b/planner/core/physical_plan_test.go @@ -1478,3 +1478,44 @@ func (s *testPlanSuite) TestHintFromDiffDatabase(c *C) { c.Assert(core.ToString(p), Equals, output[i].Plan, comment) } } + +func (s *testPlanSuite) TestNthPlanHintWithExplain(c *C) { + defer testleak.AfterTest(c)() + store, dom, err := newStoreWithBootstrap() + c.Assert(err, IsNil) + defer func() { + dom.Close() + store.Close() + }() + se, err := session.CreateSession4Test(store) + c.Assert(err, IsNil) + ctx := context.Background() + _, err = se.Execute(ctx, "use test") + c.Assert(err, IsNil) + _, err = se.Execute(ctx, `drop table if exists test.tt`) + c.Assert(err, IsNil) + _, err = se.Execute(ctx, `create table test.tt (a int,b int, index(a), index(b));`) + + _, err = se.Execute(ctx, "insert into tt values (1, 1), (2, 2), (3, 4)") + c.Assert(err, IsNil) + + var input []string + var output []struct { + SQL string + Plan string + } + is := domain.GetDomain(se).InfoSchema() + s.testData.GetTestCases(c, &input, &output) + for i, tt := range input { + comment := Commentf("case:%v sql: %s", i, tt) + stmt, err := s.ParseOneStmt(tt, "", "") + c.Assert(err, IsNil, comment) + p, _, err := planner.Optimize(ctx, se, stmt, is) + c.Assert(err, IsNil, comment) + s.testData.OnRecord(func() { + output[i].SQL = tt + output[i].Plan = core.ToString(p) + }) + c.Assert(core.ToString(p), Equals, output[i].Plan, comment) + } +} diff --git a/planner/core/plan_test.go b/planner/core/plan_test.go index 5047635dc7c32..6d767fb168775 100644 --- a/planner/core/plan_test.go +++ b/planner/core/plan_test.go @@ -240,31 +240,6 @@ func (s *testPlanNormalize) TestNthPlanHint(c *C) { tk.MustExec("create table tt (a int,b int, index(a), index(b));") tk.MustExec("insert into tt values (1, 1), (2, 2), (3, 4)") - // Test the explained plans. - tk.MustQuery("explain select /*+nth_plan(1)*/ * from tt where a=1 and b=1;").Check(testkit.Rows( - "TableReader_18 0.01 root data:Selection_17", - "└─Selection_17 0.01 cop[tikv] eq(test.tt.a, 1), eq(test.tt.b, 1)", - " └─TableFullScan_16 10000.00 cop[tikv] table:tt keep order:false, stats:pseudo")) - - tk.MustQuery("explain select /*+nth_plan(2)*/ * from tt where a=1 and b=1;").Check(testkit.Rows( - "IndexLookUp_22 0.01 root ", - "├─IndexRangeScan_19(Build) 10.00 cop[tikv] table:tt, index:a(a) range:[1,1], keep order:false, stats:pseudo", - "└─Selection_21(Probe) 0.01 cop[tikv] eq(test.tt.b, 1)", - " └─TableRowIDScan_20 10.00 cop[tikv] table:tt keep order:false, stats:pseudo")) - - tk.MustQuery("explain select /*+nth_plan(3)*/ * from tt where a=1 and b=1;").Check(testkit.Rows( - "IndexLookUp_26 0.01 root ", - "├─IndexRangeScan_23(Build) 10.00 cop[tikv] table:tt, index:b(b) range:[1,1], keep order:false, stats:pseudo", - "└─Selection_25(Probe) 0.01 cop[tikv] eq(test.tt.a, 1)", - " └─TableRowIDScan_24 10.00 cop[tikv] table:tt keep order:false, stats:pseudo")) - - // With the same param in nth_plan, we should get the same plan. - tk.MustQuery("explain select /*+nth_plan(2)*/ * from tt where a=1 and b=1;").Check(testkit.Rows( - "IndexLookUp_22 0.01 root ", - "├─IndexRangeScan_19(Build) 10.00 cop[tikv] table:tt, index:a(a) range:[1,1], keep order:false, stats:pseudo", - "└─Selection_21(Probe) 0.01 cop[tikv] eq(test.tt.b, 1)", - " └─TableRowIDScan_20 10.00 cop[tikv] table:tt keep order:false, stats:pseudo")) - tk.MustExec("explain select /*+nth_plan(4)*/ * from tt where a=1 and b=1;") tk.MustQuery("show warnings").Check(testkit.Rows( "Warning 1105 The parameter of nth_plan() is out of range.")) diff --git a/planner/core/testdata/plan_suite_in.json b/planner/core/testdata/plan_suite_in.json index 767103c4c3cec..43bceb2d1873f 100644 --- a/planner/core/testdata/plan_suite_in.json +++ b/planner/core/testdata/plan_suite_in.json @@ -632,5 +632,14 @@ "cases": [ "select /*+ inl_hash_join(test.t1) */ * from test.t2 join test.t1 on test.t2.a = test.t1.a" ] + }, + { + "name": "TestNthPlanHintWithExplain", + "cases": [ + "select /*+nth_plan(1)*/ * from test.tt where a=1 and b=1", + "select /*+nth_plan(2)*/ * from test.tt where a=1 and b=1;", + "select /*+nth_plan(3)*/ * from test.tt where a=1 and b=1;", + "select /*+nth_plan(2)*/ * from test.tt where a=1 and b=1;" + ] } ] diff --git a/planner/core/testdata/plan_suite_out.json b/planner/core/testdata/plan_suite_out.json index 2fc68eb392b16..04f3c163299bd 100644 --- a/planner/core/testdata/plan_suite_out.json +++ b/planner/core/testdata/plan_suite_out.json @@ -2075,5 +2075,26 @@ "Plan": "IndexHashJoin{IndexReader(Index(t2.idx_a)[[-inf,+inf]])->IndexReader(Index(t1.idx_a)[[NULL,+inf]]->Sel([not(isnull(test.t1.a))]))}(test.t2.a,test.t1.a)" } ] + }, + { + "Name": "TestNthPlanHintWithExplain", + "Cases": [ + { + "SQL": "select /*+nth_plan(1)*/ * from test.tt where a=1 and b=1", + "Plan": "TableReader(Table(tt)->Sel([eq(test.tt.a, 1) eq(test.tt.b, 1)]))" + }, + { + "SQL": "select /*+nth_plan(2)*/ * from test.tt where a=1 and b=1;", + "Plan": "IndexLookUp(Index(tt.a)[[1,1]], Table(tt)->Sel([eq(test.tt.b, 1)]))" + }, + { + "SQL": "select /*+nth_plan(3)*/ * from test.tt where a=1 and b=1;", + "Plan": "IndexLookUp(Index(tt.b)[[1,1]], Table(tt)->Sel([eq(test.tt.a, 1)]))" + }, + { + "SQL": "select /*+nth_plan(2)*/ * from test.tt where a=1 and b=1;", + "Plan": "IndexLookUp(Index(tt.a)[[1,1]], Table(tt)->Sel([eq(test.tt.b, 1)]))" + } + ] } ] From c9fc0b072a875f12b81e5822797753ea003a4d17 Mon Sep 17 00:00:00 2001 From: lawyerphx Date: Thu, 18 Jun 2020 21:28:30 -0700 Subject: [PATCH 30/37] make CI happy --- planner/core/physical_plan_test.go | 1 + 1 file changed, 1 insertion(+) diff --git a/planner/core/physical_plan_test.go b/planner/core/physical_plan_test.go index 7b868c575dd51..45a59863e210f 100644 --- a/planner/core/physical_plan_test.go +++ b/planner/core/physical_plan_test.go @@ -1495,6 +1495,7 @@ func (s *testPlanSuite) TestNthPlanHintWithExplain(c *C) { _, err = se.Execute(ctx, `drop table if exists test.tt`) c.Assert(err, IsNil) _, err = se.Execute(ctx, `create table test.tt (a int,b int, index(a), index(b));`) + c.Assert(err, IsNil) _, err = se.Execute(ctx, "insert into tt values (1, 1), (2, 2), (3, 4)") c.Assert(err, IsNil) From 69188e64954d3c719e1dad9e7d441daf1e7eeb8d Mon Sep 17 00:00:00 2001 From: lawyerphx Date: Thu, 18 Jun 2020 23:53:08 -0700 Subject: [PATCH 31/37] address comments: change name --- planner/core/find_best_task.go | 42 ++++++++++++++--------------- planner/core/find_best_task_test.go | 16 +++++------ planner/core/logical_plan_test.go | 2 +- planner/core/optimizer.go | 4 +-- planner/core/plan.go | 2 +- sessionctx/stmtctx/stmtctx.go | 2 +- 6 files changed, 34 insertions(+), 34 deletions(-) diff --git a/planner/core/find_best_task.go b/planner/core/find_best_task.go index 1123c51a2efda..b35bb95c3fa5f 100644 --- a/planner/core/find_best_task.go +++ b/planner/core/find_best_task.go @@ -62,30 +62,30 @@ var aggFuncFactor = map[string]float64{ "default": 1.5, } -// CountDown is used in hint nth_plan() to indicate which plan to use. -type CountDown int8 +// PlanCounterTp is used in hint nth_plan() to indicate which plan to use. +type PlanCounterTp int8 -// Sign is the default value of CountDown, indicating that optimizer needn't force a plan. -var Sign CountDown = -1 +// PlanCounterDisabled is the default value of PlanCounterTp, indicating that optimizer needn't force a plan. +var PlanCounterDisabled PlanCounterTp = -1 -// Dec minus CountDown value by x. -func (c *CountDown) Dec(x int8) { +// Dec minus PlanCounterTp value by x. +func (c *PlanCounterTp) Dec(x int8) { if *c <= 0 { return } - *c = CountDown(int8(*c) - x) + *c = PlanCounterTp(int8(*c) - x) if *c < 0 { *c = 0 } } -// Empty indicates whether the CountDown is clear now. -func (c *CountDown) Empty() bool { +// Empty indicates whether the PlanCounterTp is clear now. +func (c *PlanCounterTp) Empty() bool { return *c == 0 } // IsForce indicates whether to force a plan. -func (c *CountDown) IsForce() bool { +func (c *PlanCounterTp) IsForce() bool { return *c != -1 } @@ -132,7 +132,7 @@ func GetPropByOrderByItemsContainScalarFunc(items []*util.ByItems) (*property.Ph return &property.PhysicalProperty{Items: propItems}, true, onlyColumn } -func (p *LogicalTableDual) findBestTask(prop *property.PhysicalProperty, clock *CountDown) (task, int64, error) { +func (p *LogicalTableDual) findBestTask(prop *property.PhysicalProperty, clock *PlanCounterTp) (task, int64, error) { // If the required property is not empty and the row count > 1, // we cannot ensure this required property. // But if the row count is 0 or 1, we don't need to care about the property. @@ -147,7 +147,7 @@ func (p *LogicalTableDual) findBestTask(prop *property.PhysicalProperty, clock * return &rootTask{p: dual}, 1, nil } -func (p *LogicalShow) findBestTask(prop *property.PhysicalProperty, clock *CountDown) (task, int64, error) { +func (p *LogicalShow) findBestTask(prop *property.PhysicalProperty, clock *PlanCounterTp) (task, int64, error) { if !prop.IsEmpty() || clock.Empty() { return invalidTask, 0, nil } @@ -157,7 +157,7 @@ func (p *LogicalShow) findBestTask(prop *property.PhysicalProperty, clock *Count return &rootTask{p: pShow}, 1, nil } -func (p *LogicalShowDDLJobs) findBestTask(prop *property.PhysicalProperty, clock *CountDown) (task, int64, error) { +func (p *LogicalShowDDLJobs) findBestTask(prop *property.PhysicalProperty, clock *PlanCounterTp) (task, int64, error) { if !prop.IsEmpty() || clock.Empty() { return invalidTask, 0, nil } @@ -175,21 +175,21 @@ func (p *baseLogicalPlan) rebuildChildTasks(childTasks *[]task, pp PhysicalPlan, } multAll := int64(1) - var curClock CountDown + var curClock PlanCounterTp for _, x := range childCnts { multAll *= x } *childTasks = (*childTasks)[:0] for j, child := range p.children { multAll /= childCnts[j] - curClock = CountDown((clock-1)/multAll + 1) + curClock = PlanCounterTp((clock-1)/multAll + 1) childTask, _, err := child.findBestTask(pp.GetChildReqProps(j), &curClock) clock = (clock-1)%multAll + 1 if err != nil { return err } if curClock != 0 { - return errors.Errorf("CountDown clock is not handled") + return errors.Errorf("PlanCounterTp clock is not handled") } if childTask != nil && childTask.invalid() { return errors.Errorf("The current plan is invalid, please skip this plan.") @@ -199,7 +199,7 @@ func (p *baseLogicalPlan) rebuildChildTasks(childTasks *[]task, pp PhysicalPlan, return nil } -func (p *baseLogicalPlan) enumeratePhysicalPlans4Task(physicalPlans []PhysicalPlan, prop *property.PhysicalProperty, clock *CountDown) (task, int64, error) { +func (p *baseLogicalPlan) enumeratePhysicalPlans4Task(physicalPlans []PhysicalPlan, prop *property.PhysicalProperty, clock *PlanCounterTp) (task, int64, error) { var bestTask task = invalidTask var curCntPlan, cntPlan int64 childTasks := make([]task, 0, len(p.children)) @@ -212,7 +212,7 @@ func (p *baseLogicalPlan) enumeratePhysicalPlans4Task(physicalPlans []PhysicalPl curCntPlan = 1 TimeStampNow := p.GetBakTimeStamp() for j, child := range p.children { - childTask, cnt, err := child.findBestTask(pp.GetChildReqProps(j), &Sign) + childTask, cnt, err := child.findBestTask(pp.GetChildReqProps(j), &PlanCounterDisabled) childCnts[j] = cnt if err != nil { return nil, 0, err @@ -269,7 +269,7 @@ func (p *baseLogicalPlan) enumeratePhysicalPlans4Task(physicalPlans []PhysicalPl } // findBestTask implements LogicalPlan interface. -func (p *baseLogicalPlan) findBestTask(prop *property.PhysicalProperty, clock *CountDown) (bestTask task, cntPlan int64, err error) { +func (p *baseLogicalPlan) findBestTask(prop *property.PhysicalProperty, clock *PlanCounterTp) (bestTask task, cntPlan int64, err error) { // If p is an inner plan in an IndexJoin, the IndexJoin will generate an inner plan by itself, // and set inner child prop nil, so here we do nothing. if prop == nil { @@ -359,7 +359,7 @@ END: return bestTask, cntPlan, nil } -func (p *LogicalMemTable) findBestTask(prop *property.PhysicalProperty, clock *CountDown) (t task, cntPlan int64, err error) { +func (p *LogicalMemTable) findBestTask(prop *property.PhysicalProperty, clock *PlanCounterTp) (t task, cntPlan int64, err error) { if !prop.IsEmpty() || clock.Empty() { return invalidTask, 0, nil } @@ -547,7 +547,7 @@ func (ds *DataSource) skylinePruning(prop *property.PhysicalProperty) []*candida // findBestTask implements the PhysicalPlan interface. // It will enumerate all the available indices and choose a plan with least cost. -func (ds *DataSource) findBestTask(prop *property.PhysicalProperty, clock *CountDown) (t task, cntPlan int64, err error) { +func (ds *DataSource) findBestTask(prop *property.PhysicalProperty, clock *PlanCounterTp) (t task, cntPlan int64, err error) { // If ds is an inner plan in an IndexJoin, the IndexJoin will generate an inner plan by itself, // and set inner child prop nil, so here we do nothing. if prop == nil { diff --git a/planner/core/find_best_task_test.go b/planner/core/find_best_task_test.go index ad386b13da1a9..bac93b348b800 100644 --- a/planner/core/find_best_task_test.go +++ b/planner/core/find_best_task_test.go @@ -42,7 +42,7 @@ func (ds mockDataSource) Init(ctx sessionctx.Context) *mockDataSource { return &ds } -func (ds *mockDataSource) findBestTask(prop *property.PhysicalProperty, clock *CountDown) (task, int64, error) { +func (ds *mockDataSource) findBestTask(prop *property.PhysicalProperty, clock *PlanCounterTp) (task, int64, error) { // It can satisfy any of the property! // Just use a TableDual for convenience. p := PhysicalTableDual{}.Init(ds.ctx, &property.StatsInfo{RowCount: 1}, 0) @@ -152,7 +152,7 @@ func (s *testFindBestTaskSuite) TestCostOverflow(c *C) { mockPlan.SetChildren(mockDS) // An empty property is enough for this test. prop := property.NewPhysicalProperty(property.RootTaskType, nil, false, 0, false) - t, _, err := mockPlan.findBestTask(prop, &Sign) + t, _, err := mockPlan.findBestTask(prop, &PlanCounterDisabled) c.Assert(err, IsNil) // The cost should be overflowed, but the task shouldn't be invalid. c.Assert(t.invalid(), IsFalse) @@ -179,7 +179,7 @@ func (s *testFindBestTaskSuite) TestEnforcedProperty(c *C) { Enforced: false, } // should return invalid task because no physical plan can match this property. - task, _, err := mockPlan.findBestTask(prop0, &Sign) + task, _, err := mockPlan.findBestTask(prop0, &PlanCounterDisabled) c.Assert(err, IsNil) c.Assert(task.invalid(), IsTrue) @@ -188,7 +188,7 @@ func (s *testFindBestTaskSuite) TestEnforcedProperty(c *C) { Enforced: true, } // should return the valid task when the property is enforced. - task, _, err = mockPlan.findBestTask(prop1, &Sign) + task, _, err = mockPlan.findBestTask(prop1, &PlanCounterDisabled) c.Assert(err, IsNil) c.Assert(task.invalid(), IsFalse) } @@ -211,7 +211,7 @@ func (s *testFindBestTaskSuite) TestHintCannotFitProperty(c *C) { Items: items, Enforced: true, } - task, _, err := mockPlan0.findBestTask(prop0, &Sign) + task, _, err := mockPlan0.findBestTask(prop0, &PlanCounterDisabled) c.Assert(err, IsNil) c.Assert(task.invalid(), IsFalse) _, enforcedSort := task.plan().(*PhysicalSort) @@ -227,7 +227,7 @@ func (s *testFindBestTaskSuite) TestHintCannotFitProperty(c *C) { Items: items, Enforced: false, } - task, _, err = mockPlan0.findBestTask(prop1, &Sign) + task, _, err = mockPlan0.findBestTask(prop1, &PlanCounterDisabled) c.Assert(err, IsNil) c.Assert(task.invalid(), IsFalse) _, enforcedSort = task.plan().(*PhysicalSort) @@ -248,7 +248,7 @@ func (s *testFindBestTaskSuite) TestHintCannotFitProperty(c *C) { canGeneratePlan2: false, }.Init(ctx) mockPlan1.SetChildren(mockDS) - task, _, err = mockPlan1.findBestTask(prop2, &Sign) + task, _, err = mockPlan1.findBestTask(prop2, &PlanCounterDisabled) c.Assert(err, IsNil) c.Assert(task.invalid(), IsFalse) c.Assert(ctx.GetSessionVars().StmtCtx.WarningCount(), Equals, uint16(1)) @@ -264,7 +264,7 @@ func (s *testFindBestTaskSuite) TestHintCannotFitProperty(c *C) { Items: items, Enforced: true, } - task, _, err = mockPlan1.findBestTask(prop3, &Sign) + task, _, err = mockPlan1.findBestTask(prop3, &PlanCounterDisabled) c.Assert(err, IsNil) c.Assert(task.invalid(), IsFalse) c.Assert(ctx.GetSessionVars().StmtCtx.WarningCount(), Equals, uint16(1)) diff --git a/planner/core/logical_plan_test.go b/planner/core/logical_plan_test.go index d1df9db6d1bf8..de24367451e46 100644 --- a/planner/core/logical_plan_test.go +++ b/planner/core/logical_plan_test.go @@ -1379,7 +1379,7 @@ func (s *testPlanSuite) optimize(ctx context.Context, sql string) (PhysicalPlan, if err != nil { return nil, nil, err } - p, _, err = physicalOptimize(p.(LogicalPlan), &Sign) + p, _, err = physicalOptimize(p.(LogicalPlan), &PlanCounterDisabled) return p.(PhysicalPlan), stmt, err } diff --git a/planner/core/optimizer.go b/planner/core/optimizer.go index 27ffb31a9a2b3..c64292a5df442 100644 --- a/planner/core/optimizer.go +++ b/planner/core/optimizer.go @@ -132,7 +132,7 @@ func DoOptimize(ctx context.Context, sctx sessionctx.Context, flag uint64, logic if !AllowCartesianProduct.Load() && existsCartesianProduct(logic) { return nil, 0, errors.Trace(ErrCartesianProductUnsupported) } - clock := CountDown(sctx.GetSessionVars().StmtCtx.StmtHints.ForceNthPlan) + clock := PlanCounterTp(sctx.GetSessionVars().StmtCtx.StmtHints.ForceNthPlan) if clock == 0 { clock = -1 } @@ -173,7 +173,7 @@ func isLogicalRuleDisabled(r logicalOptRule) bool { return disabled } -func physicalOptimize(logic LogicalPlan, clock *CountDown) (PhysicalPlan, float64, error) { +func physicalOptimize(logic LogicalPlan, clock *PlanCounterTp) (PhysicalPlan, float64, error) { if _, err := logic.recursiveDeriveStats(); err != nil { return nil, 0, err } diff --git a/planner/core/plan.go b/planner/core/plan.go index b90da911c7d2e..7153f93ad444e 100644 --- a/planner/core/plan.go +++ b/planner/core/plan.go @@ -159,7 +159,7 @@ type LogicalPlan interface { // If clock > 0, the clock_th plan generated in this function will be returned. // If clock = 0, the plan generated in this function will not be considered. // If clock = -1, then we will not force plan. - findBestTask(prop *property.PhysicalProperty, clock *CountDown) (task, int64, error) + findBestTask(prop *property.PhysicalProperty, clock *PlanCounterTp) (task, int64, error) // BuildKeyInfo will collect the information of unique keys into schema. // Because this method is also used in cascades planner, we cannot use diff --git a/sessionctx/stmtctx/stmtctx.go b/sessionctx/stmtctx/stmtctx.go index 977e69f4ef838..1612dc5ac45d7 100644 --- a/sessionctx/stmtctx/stmtctx.go +++ b/sessionctx/stmtctx/stmtctx.go @@ -166,7 +166,7 @@ type StmtHints struct { NoIndexMergeHint bool // EnableCascadesPlanner is use cascades planner for a single query only. EnableCascadesPlanner bool - // ForceNthPlan indicates the countDown number for finding physical plan. + // ForceNthPlan indicates the PlanCounterTp number for finding physical plan. // -1 for disable. ForceNthPlan int64 From 411bafb3c266284635870d1384eb9b6db4967a49 Mon Sep 17 00:00:00 2001 From: lawyerphx Date: Thu, 18 Jun 2020 23:59:20 -0700 Subject: [PATCH 32/37] rename CountDown --- planner/core/find_best_task.go | 90 ++++++++++++++--------------- planner/core/find_best_task_test.go | 4 +- planner/core/optimizer.go | 14 ++--- planner/core/plan.go | 10 ++-- 4 files changed, 59 insertions(+), 59 deletions(-) diff --git a/planner/core/find_best_task.go b/planner/core/find_best_task.go index b35bb95c3fa5f..efaf86b38c3c3 100644 --- a/planner/core/find_best_task.go +++ b/planner/core/find_best_task.go @@ -132,43 +132,43 @@ func GetPropByOrderByItemsContainScalarFunc(items []*util.ByItems) (*property.Ph return &property.PhysicalProperty{Items: propItems}, true, onlyColumn } -func (p *LogicalTableDual) findBestTask(prop *property.PhysicalProperty, clock *PlanCounterTp) (task, int64, error) { +func (p *LogicalTableDual) findBestTask(prop *property.PhysicalProperty, planCounter *PlanCounterTp) (task, int64, error) { // If the required property is not empty and the row count > 1, // we cannot ensure this required property. // But if the row count is 0 or 1, we don't need to care about the property. - if (!prop.IsEmpty() && p.RowCount > 1) || clock.Empty() { + if (!prop.IsEmpty() && p.RowCount > 1) || planCounter.Empty() { return invalidTask, 0, nil } dual := PhysicalTableDual{ RowCount: p.RowCount, }.Init(p.ctx, p.stats, p.blockOffset) dual.SetSchema(p.schema) - clock.Dec(1) + planCounter.Dec(1) return &rootTask{p: dual}, 1, nil } -func (p *LogicalShow) findBestTask(prop *property.PhysicalProperty, clock *PlanCounterTp) (task, int64, error) { - if !prop.IsEmpty() || clock.Empty() { +func (p *LogicalShow) findBestTask(prop *property.PhysicalProperty, planCounter *PlanCounterTp) (task, int64, error) { + if !prop.IsEmpty() || planCounter.Empty() { return invalidTask, 0, nil } pShow := PhysicalShow{ShowContents: p.ShowContents}.Init(p.ctx) pShow.SetSchema(p.schema) - clock.Dec(1) + planCounter.Dec(1) return &rootTask{p: pShow}, 1, nil } -func (p *LogicalShowDDLJobs) findBestTask(prop *property.PhysicalProperty, clock *PlanCounterTp) (task, int64, error) { - if !prop.IsEmpty() || clock.Empty() { +func (p *LogicalShowDDLJobs) findBestTask(prop *property.PhysicalProperty, planCounter *PlanCounterTp) (task, int64, error) { + if !prop.IsEmpty() || planCounter.Empty() { return invalidTask, 0, nil } pShow := PhysicalShowDDLJobs{JobNumber: p.JobNumber}.Init(p.ctx) pShow.SetSchema(p.schema) - clock.Dec(1) + planCounter.Dec(1) return &rootTask{p: pShow}, 1, nil } // The function rebuildChildTasks rebuilds the childTasks to make the clock_th combination. -func (p *baseLogicalPlan) rebuildChildTasks(childTasks *[]task, pp PhysicalPlan, childCnts []int64, clock int64, TS time.Time) error { +func (p *baseLogicalPlan) rebuildChildTasks(childTasks *[]task, pp PhysicalPlan, childCnts []int64, planCounter int64, TS time.Time) error { // The taskMap of children nodes should be rolled back first. for _, child := range p.children { child.rollBackTaskMap(TS) @@ -182,14 +182,14 @@ func (p *baseLogicalPlan) rebuildChildTasks(childTasks *[]task, pp PhysicalPlan, *childTasks = (*childTasks)[:0] for j, child := range p.children { multAll /= childCnts[j] - curClock = PlanCounterTp((clock-1)/multAll + 1) + curClock = PlanCounterTp((planCounter-1)/multAll + 1) childTask, _, err := child.findBestTask(pp.GetChildReqProps(j), &curClock) - clock = (clock-1)%multAll + 1 + planCounter = (planCounter-1)%multAll + 1 if err != nil { return err } if curClock != 0 { - return errors.Errorf("PlanCounterTp clock is not handled") + return errors.Errorf("PlanCounterTp planCounter is not handled") } if childTask != nil && childTask.invalid() { return errors.Errorf("The current plan is invalid, please skip this plan.") @@ -199,7 +199,7 @@ func (p *baseLogicalPlan) rebuildChildTasks(childTasks *[]task, pp PhysicalPlan, return nil } -func (p *baseLogicalPlan) enumeratePhysicalPlans4Task(physicalPlans []PhysicalPlan, prop *property.PhysicalProperty, clock *PlanCounterTp) (task, int64, error) { +func (p *baseLogicalPlan) enumeratePhysicalPlans4Task(physicalPlans []PhysicalPlan, prop *property.PhysicalProperty, planCounter *PlanCounterTp) (task, int64, error) { var bestTask task = invalidTask var curCntPlan, cntPlan int64 childTasks := make([]task, 0, len(p.children)) @@ -230,9 +230,9 @@ func (p *baseLogicalPlan) enumeratePhysicalPlans4Task(physicalPlans []PhysicalPl } // If the target plan can be found in this physicalPlan(pp), rebuild childTasks to build the corresponding combination. - if clock.IsForce() && int64(*clock) <= curCntPlan { - curCntPlan = int64(*clock) - err := p.rebuildChildTasks(&childTasks, pp, childCnts, int64(*clock), TimeStampNow) + if planCounter.IsForce() && int64(*planCounter) <= curCntPlan { + curCntPlan = int64(*planCounter) + err := p.rebuildChildTasks(&childTasks, pp, childCnts, int64(*planCounter), TimeStampNow) if err != nil { return nil, 0, err } @@ -254,9 +254,9 @@ func (p *baseLogicalPlan) enumeratePhysicalPlans4Task(physicalPlans []PhysicalPl // Get the most efficient one. cntPlan += curCntPlan - clock.Dec(int8(curCntPlan)) + planCounter.Dec(int8(curCntPlan)) - if clock.Empty() { + if planCounter.Empty() { bestTask = curTask break } @@ -269,7 +269,7 @@ func (p *baseLogicalPlan) enumeratePhysicalPlans4Task(physicalPlans []PhysicalPl } // findBestTask implements LogicalPlan interface. -func (p *baseLogicalPlan) findBestTask(prop *property.PhysicalProperty, clock *PlanCounterTp) (bestTask task, cntPlan int64, err error) { +func (p *baseLogicalPlan) findBestTask(prop *property.PhysicalProperty, planCounter *PlanCounterTp) (bestTask task, cntPlan int64, err error) { // If p is an inner plan in an IndexJoin, the IndexJoin will generate an inner plan by itself, // and set inner child prop nil, so here we do nothing. if prop == nil { @@ -279,7 +279,7 @@ func (p *baseLogicalPlan) findBestTask(prop *property.PhysicalProperty, clock *P // It's used to reduce double counting. bestTask = p.getTask(prop) if bestTask != nil { - clock.Dec(1) + planCounter.Dec(1) return bestTask, 1, nil } @@ -332,21 +332,21 @@ func (p *baseLogicalPlan) findBestTask(prop *property.PhysicalProperty, clock *P newProp.Enforced = false var cnt int64 var curTask task - if bestTask, cnt, err = p.enumeratePhysicalPlans4Task(plansFitsProp, newProp, clock); err != nil { + if bestTask, cnt, err = p.enumeratePhysicalPlans4Task(plansFitsProp, newProp, planCounter); err != nil { return nil, 0, err } cntPlan += cnt - if clock.Empty() { + if planCounter.Empty() { goto END } newProp.Enforced = true - curTask, cnt, err = p.enumeratePhysicalPlans4Task(plansNeedEnforce, newProp, clock) + curTask, cnt, err = p.enumeratePhysicalPlans4Task(plansNeedEnforce, newProp, planCounter) if err != nil { return nil, 0, err } cntPlan += cnt - if clock.Empty() { + if planCounter.Empty() { bestTask = curTask goto END } @@ -359,8 +359,8 @@ END: return bestTask, cntPlan, nil } -func (p *LogicalMemTable) findBestTask(prop *property.PhysicalProperty, clock *PlanCounterTp) (t task, cntPlan int64, err error) { - if !prop.IsEmpty() || clock.Empty() { +func (p *LogicalMemTable) findBestTask(prop *property.PhysicalProperty, planCounter *PlanCounterTp) (t task, cntPlan int64, err error) { + if !prop.IsEmpty() || planCounter.Empty() { return invalidTask, 0, nil } memTable := PhysicalMemTable{ @@ -371,7 +371,7 @@ func (p *LogicalMemTable) findBestTask(prop *property.PhysicalProperty, clock *P QueryTimeRange: p.QueryTimeRange, }.Init(p.ctx, p.stats, p.blockOffset) memTable.SetSchema(p.schema) - clock.Dec(1) + planCounter.Dec(1) return &rootTask{p: memTable}, 1, nil } @@ -547,7 +547,7 @@ func (ds *DataSource) skylinePruning(prop *property.PhysicalProperty) []*candida // findBestTask implements the PhysicalPlan interface. // It will enumerate all the available indices and choose a plan with least cost. -func (ds *DataSource) findBestTask(prop *property.PhysicalProperty, clock *PlanCounterTp) (t task, cntPlan int64, err error) { +func (ds *DataSource) findBestTask(prop *property.PhysicalProperty, planCounter *PlanCounterTp) (t task, cntPlan int64, err error) { // If ds is an inner plan in an IndexJoin, the IndexJoin will generate an inner plan by itself, // and set inner child prop nil, so here we do nothing. if prop == nil { @@ -557,7 +557,7 @@ func (ds *DataSource) findBestTask(prop *property.PhysicalProperty, clock *PlanC t = ds.getTask(prop) if t != nil { cntPlan = 1 - clock.Dec(1) + planCounter.Dec(1) return } var cnt int64 @@ -567,7 +567,7 @@ func (ds *DataSource) findBestTask(prop *property.PhysicalProperty, clock *PlanC if prop.Enforced { // First, get the bestTask without enforced prop prop.Enforced = false - t, cnt, err = ds.findBestTask(prop, clock) + t, cnt, err = ds.findBestTask(prop, planCounter) if err != nil { return nil, 0, err } @@ -593,7 +593,7 @@ func (ds *DataSource) findBestTask(prop *property.PhysicalProperty, clock *PlanC t, err = ds.tryToGetDualTask() if err != nil || t != nil { - clock.Dec(1) + planCounter.Dec(1) return t, 1, err } @@ -610,12 +610,12 @@ func (ds *DataSource) findBestTask(prop *property.PhysicalProperty, clock *PlanC } if !idxMergeTask.invalid() { cntPlan += 1 - clock.Dec(1) + planCounter.Dec(1) } - if idxMergeTask.cost() < t.cost() || clock.Empty() { + if idxMergeTask.cost() < t.cost() || planCounter.Empty() { t = idxMergeTask } - if clock.Empty() { + if planCounter.Empty() { return t, cntPlan, nil } continue @@ -625,7 +625,7 @@ func (ds *DataSource) findBestTask(prop *property.PhysicalProperty, clock *PlanC dual := PhysicalTableDual{}.Init(ds.ctx, ds.stats, ds.blockOffset) dual.SetSchema(ds.schema) cntPlan += 1 - clock.Dec(1) + planCounter.Dec(1) return &rootTask{ p: dual, }, cntPlan, nil @@ -655,11 +655,11 @@ func (ds *DataSource) findBestTask(prop *property.PhysicalProperty, clock *PlanC } if !pointGetTask.invalid() { cntPlan += 1 - clock.Dec(1) + planCounter.Dec(1) } - if pointGetTask.cost() < t.cost() || clock.Empty() { + if pointGetTask.cost() < t.cost() || planCounter.Empty() { t = pointGetTask - if clock.Empty() { + if planCounter.Empty() { return } continue @@ -679,12 +679,12 @@ func (ds *DataSource) findBestTask(prop *property.PhysicalProperty, clock *PlanC } if !tblTask.invalid() { cntPlan += 1 - clock.Dec(1) + planCounter.Dec(1) } - if tblTask.cost() < t.cost() || clock.Empty() { + if tblTask.cost() < t.cost() || planCounter.Empty() { t = tblTask } - if clock.Empty() { + if planCounter.Empty() { return t, cntPlan, nil } continue @@ -699,12 +699,12 @@ func (ds *DataSource) findBestTask(prop *property.PhysicalProperty, clock *PlanC } if !idxTask.invalid() { cntPlan += 1 - clock.Dec(1) + planCounter.Dec(1) } - if idxTask.cost() < t.cost() || clock.Empty() { + if idxTask.cost() < t.cost() || planCounter.Empty() { t = idxTask } - if clock.Empty() { + if planCounter.Empty() { return t, cntPlan, nil } } diff --git a/planner/core/find_best_task_test.go b/planner/core/find_best_task_test.go index bac93b348b800..0d799c879e890 100644 --- a/planner/core/find_best_task_test.go +++ b/planner/core/find_best_task_test.go @@ -42,7 +42,7 @@ func (ds mockDataSource) Init(ctx sessionctx.Context) *mockDataSource { return &ds } -func (ds *mockDataSource) findBestTask(prop *property.PhysicalProperty, clock *PlanCounterTp) (task, int64, error) { +func (ds *mockDataSource) findBestTask(prop *property.PhysicalProperty, planCounter *PlanCounterTp) (task, int64, error) { // It can satisfy any of the property! // Just use a TableDual for convenience. p := PhysicalTableDual{}.Init(ds.ctx, &property.StatsInfo{RowCount: 1}, 0) @@ -50,7 +50,7 @@ func (ds *mockDataSource) findBestTask(prop *property.PhysicalProperty, clock *P p: p, cst: 10000, } - clock.Dec(1) + planCounter.Dec(1) return task, 1, nil } diff --git a/planner/core/optimizer.go b/planner/core/optimizer.go index c64292a5df442..e15ee0e4caf74 100644 --- a/planner/core/optimizer.go +++ b/planner/core/optimizer.go @@ -132,11 +132,11 @@ func DoOptimize(ctx context.Context, sctx sessionctx.Context, flag uint64, logic if !AllowCartesianProduct.Load() && existsCartesianProduct(logic) { return nil, 0, errors.Trace(ErrCartesianProductUnsupported) } - clock := PlanCounterTp(sctx.GetSessionVars().StmtCtx.StmtHints.ForceNthPlan) - if clock == 0 { - clock = -1 + planCounter := PlanCounterTp(sctx.GetSessionVars().StmtCtx.StmtHints.ForceNthPlan) + if planCounter == 0 { + planCounter = -1 } - physical, cost, err := physicalOptimize(logic, &clock) + physical, cost, err := physicalOptimize(logic, &planCounter) if err != nil { return nil, 0, err } @@ -173,7 +173,7 @@ func isLogicalRuleDisabled(r logicalOptRule) bool { return disabled } -func physicalOptimize(logic LogicalPlan, clock *PlanCounterTp) (PhysicalPlan, float64, error) { +func physicalOptimize(logic LogicalPlan, planCounter *PlanCounterTp) (PhysicalPlan, float64, error) { if _, err := logic.recursiveDeriveStats(); err != nil { return nil, 0, err } @@ -185,11 +185,11 @@ func physicalOptimize(logic LogicalPlan, clock *PlanCounterTp) (PhysicalPlan, fl ExpectedCnt: math.MaxFloat64, } - t, _, err := logic.findBestTask(prop, clock) + t, _, err := logic.findBestTask(prop, planCounter) if err != nil { return nil, 0, err } - if *clock > 0 { + if *planCounter > 0 { logic.SCtx().GetSessionVars().StmtCtx.AppendWarning(errors.Errorf("The parameter of nth_plan() is out of range.")) } if t.invalid() { diff --git a/planner/core/plan.go b/planner/core/plan.go index 7153f93ad444e..c03ce43f3ddb4 100644 --- a/planner/core/plan.go +++ b/planner/core/plan.go @@ -155,11 +155,11 @@ type LogicalPlan interface { // It is called recursively from the parent to the children to create the result physical plan. // Some logical plans will convert the children to the physical plans in different ways, and return the one // With the lowest cost and how many plans are found in this function. - // clock is a counter for planner to force a plan. - // If clock > 0, the clock_th plan generated in this function will be returned. - // If clock = 0, the plan generated in this function will not be considered. - // If clock = -1, then we will not force plan. - findBestTask(prop *property.PhysicalProperty, clock *PlanCounterTp) (task, int64, error) + // planCounter is a counter for planner to force a plan. + // If planCounter > 0, the clock_th plan generated in this function will be returned. + // If planCounter = 0, the plan generated in this function will not be considered. + // If planCounter = -1, then we will not force plan. + findBestTask(prop *property.PhysicalProperty, planCounter *PlanCounterTp) (task, int64, error) // BuildKeyInfo will collect the information of unique keys into schema. // Because this method is also used in cascades planner, we cannot use From 1b2677ac667463949e9cd1f5b685dfc95e4dfac6 Mon Sep 17 00:00:00 2001 From: lawyerphx Date: Sun, 28 Jun 2020 00:21:10 -0700 Subject: [PATCH 33/37] address comments --- planner/core/find_best_task.go | 3 +-- planner/core/optimizer.go | 1 + planner/core/plan.go | 16 ++++++++-------- sessionctx/stmtctx/stmtctx.go | 1 + 4 files changed, 11 insertions(+), 10 deletions(-) diff --git a/planner/core/find_best_task.go b/planner/core/find_best_task.go index efaf86b38c3c3..7efcbec12ed7a 100644 --- a/planner/core/find_best_task.go +++ b/planner/core/find_best_task.go @@ -15,7 +15,6 @@ package core import ( "math" - "time" "github.com/pingcap/errors" "github.com/pingcap/parser/ast" @@ -168,7 +167,7 @@ func (p *LogicalShowDDLJobs) findBestTask(prop *property.PhysicalProperty, planC } // The function rebuildChildTasks rebuilds the childTasks to make the clock_th combination. -func (p *baseLogicalPlan) rebuildChildTasks(childTasks *[]task, pp PhysicalPlan, childCnts []int64, planCounter int64, TS time.Time) error { +func (p *baseLogicalPlan) rebuildChildTasks(childTasks *[]task, pp PhysicalPlan, childCnts []int64, planCounter int64, TS uint64) error { // The taskMap of children nodes should be rolled back first. for _, child := range p.children { child.rollBackTaskMap(TS) diff --git a/planner/core/optimizer.go b/planner/core/optimizer.go index e15ee0e4caf74..f4e666bad4b21 100644 --- a/planner/core/optimizer.go +++ b/planner/core/optimizer.go @@ -185,6 +185,7 @@ func physicalOptimize(logic LogicalPlan, planCounter *PlanCounterTp) (PhysicalPl ExpectedCnt: math.MaxFloat64, } + logic.SCtx().GetSessionVars().StmtCtx.TaskMapBakTS = 0 t, _, err := logic.findBestTask(prop, planCounter) if err != nil { return nil, 0, err diff --git a/planner/core/plan.go b/planner/core/plan.go index c03ce43f3ddb4..392fc0619a9b8 100644 --- a/planner/core/plan.go +++ b/planner/core/plan.go @@ -17,7 +17,6 @@ import ( "fmt" "math" "strconv" - "time" "github.com/cznic/mathutil" "github.com/pingcap/tidb/expression" @@ -204,7 +203,7 @@ type LogicalPlan interface { SetChild(i int, child LogicalPlan) // rollBackTaskMap roll back all taskMap's logs after TimeStamp TS. - rollBackTaskMap(TS time.Time) + rollBackTaskMap(TS uint64) } // PhysicalPlan is a tree of the physical operators. @@ -253,7 +252,7 @@ type baseLogicalPlan struct { // taskMapBak forms a backlog stack of taskMap, used to roll back the taskMap. taskMapBak []string // taskMapBakTS stores the timestamps of logs. - taskMapBakTS []time.Time + taskMapBakTS []uint64 self LogicalPlan maxOneRow bool children []LogicalPlan @@ -295,11 +294,12 @@ func (p *basePhysicalPlan) ExtractCorrelatedCols() []*expression.CorrelatedColum return nil } -func (p *baseLogicalPlan) GetBakTimeStamp() time.Time { - return time.Now() +func (p *baseLogicalPlan) GetBakTimeStamp() uint64 { + p.ctx.GetSessionVars().StmtCtx.TaskMapBakTS += 1 + return p.ctx.GetSessionVars().StmtCtx.TaskMapBakTS } -func (p *baseLogicalPlan) rollBackTaskMap(TS time.Time) { +func (p *baseLogicalPlan) rollBackTaskMap(TS uint64) { if !p.ctx.GetSessionVars().StmtCtx.StmtHints.TaskMapNeedBackUp() { return } @@ -308,7 +308,7 @@ func (p *baseLogicalPlan) rollBackTaskMap(TS time.Time) { N := len(p.taskMapBak) for i := 0; i < N; i++ { cur := p.taskMapBak[i] - if p.taskMapBakTS[i].String() < TS.String() { + if p.taskMapBakTS[i] < TS { continue } @@ -405,7 +405,7 @@ func newBaseLogicalPlan(ctx sessionctx.Context, tp string, self LogicalPlan, off return baseLogicalPlan{ taskMap: make(map[string]task), taskMapBak: make([]string, 0, 10), - taskMapBakTS: make([]time.Time, 0, 10), + taskMapBakTS: make([]uint64, 0, 10), basePlan: newBasePlan(ctx, tp, offset), self: self, } diff --git a/sessionctx/stmtctx/stmtctx.go b/sessionctx/stmtctx/stmtctx.go index 1612dc5ac45d7..15895b6706510 100644 --- a/sessionctx/stmtctx/stmtctx.go +++ b/sessionctx/stmtctx/stmtctx.go @@ -153,6 +153,7 @@ type StatementContext struct { LockKeysCount int32 TblInfo2UnionScan map[*model.TableInfo]bool TaskID uint64 // unique ID for an execution of a statement + TaskMapBakTS uint64 // counter for } // StmtHints are SessionVars related sql hints. From 955e52088fd33491c52e88dd8ac7f5b733787183 Mon Sep 17 00:00:00 2001 From: lawyerphx Date: Sun, 28 Jun 2020 21:32:31 -0700 Subject: [PATCH 34/37] fix empty table error. --- planner/core/find_best_task.go | 1 + 1 file changed, 1 insertion(+) diff --git a/planner/core/find_best_task.go b/planner/core/find_best_task.go index 909ef18fd92e1..16148869ed7db 100644 --- a/planner/core/find_best_task.go +++ b/planner/core/find_best_task.go @@ -551,6 +551,7 @@ func (ds *DataSource) findBestTask(prop *property.PhysicalProperty, planCounter // If ds is an inner plan in an IndexJoin, the IndexJoin will generate an inner plan by itself, // and set inner child prop nil, so here we do nothing. if prop == nil { + planCounter.Dec(1) return nil, 1, nil } From 4362680e3cc37c76df3926c781bbb8c7982a2afc Mon Sep 17 00:00:00 2001 From: lawyerphx Date: Tue, 30 Jun 2020 11:49:59 +0800 Subject: [PATCH 35/37] Update planner/core/find_best_task.go Co-authored-by: Kenan Yao --- planner/core/find_best_task.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/planner/core/find_best_task.go b/planner/core/find_best_task.go index 16148869ed7db..1d4dc0e7dc2e5 100644 --- a/planner/core/find_best_task.go +++ b/planner/core/find_best_task.go @@ -167,7 +167,7 @@ func (p *LogicalShowDDLJobs) findBestTask(prop *property.PhysicalProperty, planC return &rootTask{p: pShow}, 1, nil } -// The function rebuildChildTasks rebuilds the childTasks to make the clock_th combination. +// rebuildChildTasks rebuilds the childTasks to make the clock_th combination. func (p *baseLogicalPlan) rebuildChildTasks(childTasks *[]task, pp PhysicalPlan, childCnts []int64, planCounter int64, TS uint64) error { // The taskMap of children nodes should be rolled back first. for _, child := range p.children { From c99e8559caf93a63a9bae511b7ec98853f9d0bcd Mon Sep 17 00:00:00 2001 From: lawyerphx Date: Mon, 29 Jun 2020 21:16:00 -0700 Subject: [PATCH 36/37] address comments. --- planner/core/find_best_task.go | 2 +- planner/core/plan.go | 5 +++-- 2 files changed, 4 insertions(+), 3 deletions(-) diff --git a/planner/core/find_best_task.go b/planner/core/find_best_task.go index 1d4dc0e7dc2e5..38c4caf09777a 100644 --- a/planner/core/find_best_task.go +++ b/planner/core/find_best_task.go @@ -210,7 +210,7 @@ func (p *baseLogicalPlan) enumeratePhysicalPlans4Task(physicalPlans []PhysicalPl childTasks = childTasks[:0] // The curCntPlan records the number of possible plans for pp curCntPlan = 1 - TimeStampNow := p.GetBakTimeStamp() + TimeStampNow := p.GetlogicalTS4TaskMap() for j, child := range p.children { childTask, cnt, err := child.findBestTask(pp.GetChildReqProps(j), &PlanCounterDisabled) childCnts[j] = cnt diff --git a/planner/core/plan.go b/planner/core/plan.go index 822dce3bd8424..55c6c5e689f11 100644 --- a/planner/core/plan.go +++ b/planner/core/plan.go @@ -321,7 +321,8 @@ func (p *basePhysicalPlan) ExtractCorrelatedCols() []*expression.CorrelatedColum return nil } -func (p *baseLogicalPlan) GetBakTimeStamp() uint64 { +// GetlogicalTS4TaskMap get the logical TimeStamp now to help rollback the TaskMap changes after that. +func (p *baseLogicalPlan) GetlogicalTS4TaskMap() uint64 { p.ctx.GetSessionVars().StmtCtx.TaskMapBakTS += 1 return p.ctx.GetSessionVars().StmtCtx.TaskMapBakTS } @@ -365,7 +366,7 @@ func (p *baseLogicalPlan) storeTask(prop *property.PhysicalProperty, task task) key := prop.HashCode() if p.ctx.GetSessionVars().StmtCtx.StmtHints.TaskMapNeedBackUp() { // Empty string for useless change. - TS := p.GetBakTimeStamp() + TS := p.GetlogicalTS4TaskMap() if p.taskMap[string(key)] != nil { p.taskMapBakTS = append(p.taskMapBakTS, TS) p.taskMapBak = append(p.taskMapBak, "") From bbf59484457f9980d25a7fe6b00e0574aa55d3f2 Mon Sep 17 00:00:00 2001 From: lawyerphx Date: Mon, 29 Jun 2020 23:26:36 -0700 Subject: [PATCH 37/37] address comments. --- planner/core/find_best_task.go | 4 ++-- planner/core/plan.go | 15 ++++----------- 2 files changed, 6 insertions(+), 13 deletions(-) diff --git a/planner/core/find_best_task.go b/planner/core/find_best_task.go index 38c4caf09777a..2620d387ccb91 100644 --- a/planner/core/find_best_task.go +++ b/planner/core/find_best_task.go @@ -248,11 +248,10 @@ func (p *baseLogicalPlan) enumeratePhysicalPlans4Task(physicalPlans []PhysicalPl // Optimize by shuffle executor to running in parallel manner. if prop.IsEmpty() { - // Currently, we don not regard shuffled plan as a new plan. + // Currently, we do not regard shuffled plan as a new plan. curTask = optimizeByShuffle(pp, curTask, p.basePlan.ctx) } - // Get the most efficient one. cntPlan += curCntPlan planCounter.Dec(int8(curCntPlan)) @@ -261,6 +260,7 @@ func (p *baseLogicalPlan) enumeratePhysicalPlans4Task(physicalPlans []PhysicalPl break } + // Get the most efficient one. if curTask.cost() < bestTask.cost() || (bestTask.invalid() && !curTask.invalid()) { bestTask = curTask } diff --git a/planner/core/plan.go b/planner/core/plan.go index 55c6c5e689f11..04b93c60b18f3 100644 --- a/planner/core/plan.go +++ b/planner/core/plan.go @@ -346,10 +346,8 @@ func (p *baseLogicalPlan) rollBackTaskMap(TS uint64) { i-- N-- - // If cur is a valid log, then roll back. - if cur != "" { - p.taskMap[cur] = nil - } + // Roll back taskMap. + p.taskMap[cur] = nil } } for _, child := range p.children { @@ -367,13 +365,8 @@ func (p *baseLogicalPlan) storeTask(prop *property.PhysicalProperty, task task) if p.ctx.GetSessionVars().StmtCtx.StmtHints.TaskMapNeedBackUp() { // Empty string for useless change. TS := p.GetlogicalTS4TaskMap() - if p.taskMap[string(key)] != nil { - p.taskMapBakTS = append(p.taskMapBakTS, TS) - p.taskMapBak = append(p.taskMapBak, "") - } else { - p.taskMapBakTS = append(p.taskMapBakTS, TS) - p.taskMapBak = append(p.taskMapBak, string(key)) - } + p.taskMapBakTS = append(p.taskMapBakTS, TS) + p.taskMapBak = append(p.taskMapBak, string(key)) } p.taskMap[string(key)] = task }