You can not select more than 25 topics
Topics must start with a letter or number, can include dashes ('-') and can be up to 35 characters long.
274 lines
9.0 KiB
274 lines
9.0 KiB
// Copyright 2020 PingCAP, Inc.
|
|
//
|
|
// Licensed under the Apache License, Version 2.0 (the "License");
|
|
// you may not use this file except in compliance with the License.
|
|
// You may obtain a copy of the License at
|
|
//
|
|
// http://www.apache.org/licenses/LICENSE-2.0
|
|
//
|
|
// Unless required by applicable law or agreed to in writing, software
|
|
// distributed under the License is distributed on an "AS IS" BASIS,
|
|
// See the License for the specific language governing permissions and
|
|
// limitations under the License.
|
|
|
|
package core
|
|
|
|
import (
|
|
"fmt"
|
|
"math"
|
|
|
|
. "github.com/pingcap/check"
|
|
"github.com/pingcap/tidb/expression"
|
|
"github.com/pingcap/tidb/planner/property"
|
|
"github.com/pingcap/tidb/sessionctx"
|
|
)
|
|
|
|
var _ = Suite(&testFindBestTaskSuite{})
|
|
|
|
type testFindBestTaskSuite struct {
|
|
ctx sessionctx.Context
|
|
}
|
|
|
|
func (s *testFindBestTaskSuite) SetUpSuite(c *C) {
|
|
s.ctx = MockContext()
|
|
}
|
|
|
|
type mockDataSource struct {
|
|
baseLogicalPlan
|
|
}
|
|
|
|
func (ds mockDataSource) Init(ctx sessionctx.Context) *mockDataSource {
|
|
ds.baseLogicalPlan = newBaseLogicalPlan(ctx, "mockDS", &ds, 0)
|
|
return &ds
|
|
}
|
|
|
|
func (ds *mockDataSource) findBestTask(prop *property.PhysicalProperty) (task, error) {
|
|
// It can satisfy any of the property!
|
|
// Just use a TableDual for convenience.
|
|
p := PhysicalTableDual{}.Init(ds.ctx, &property.StatsInfo{RowCount: 1}, 0)
|
|
task := &rootTask{
|
|
p: p,
|
|
cst: 10000,
|
|
}
|
|
return task, nil
|
|
}
|
|
|
|
// mockLogicalPlan4Test is a LogicalPlan which is used for unit test.
|
|
// The basic assumption:
|
|
// 1. mockLogicalPlan4Test can generate tow kinds of physical plan: physicalPlan1 and
|
|
// physicalPlan2. physicalPlan1 can pass the property only when they are the same
|
|
// order; while physicalPlan2 cannot match any of the property(in other words, we can
|
|
// generate it only when then property is empty).
|
|
// 2. We have a hint for physicalPlan2.
|
|
// 3. If the property is empty, we still need to check `canGeneratePlan2` to decide
|
|
// whether it can generate physicalPlan2.
|
|
type mockLogicalPlan4Test struct {
|
|
baseLogicalPlan
|
|
// hasHintForPlan2 indicates whether this mockPlan contains hint.
|
|
// This hint is used to generate physicalPlan2. See the implementation
|
|
// of exhaustPhysicalPlans().
|
|
hasHintForPlan2 bool
|
|
// canGeneratePlan2 indicates whether this plan can generate physicalPlan2.
|
|
canGeneratePlan2 bool
|
|
// costOverflow indicates whether this plan will generate physical plan whose cost is overflowed.
|
|
costOverflow bool
|
|
}
|
|
|
|
func (p mockLogicalPlan4Test) Init(ctx sessionctx.Context) *mockLogicalPlan4Test {
|
|
p.baseLogicalPlan = newBaseLogicalPlan(ctx, "mockPlan", &p, 0)
|
|
return &p
|
|
}
|
|
|
|
func (p *mockLogicalPlan4Test) getPhysicalPlan1(prop *property.PhysicalProperty) PhysicalPlan {
|
|
physicalPlan1 := mockPhysicalPlan4Test{planType: 1, costOverflow: p.costOverflow}.Init(p.ctx)
|
|
physicalPlan1.stats = &property.StatsInfo{RowCount: 1}
|
|
physicalPlan1.childrenReqProps = make([]*property.PhysicalProperty, 1)
|
|
physicalPlan1.childrenReqProps[0] = prop.Clone()
|
|
return physicalPlan1
|
|
}
|
|
|
|
func (p *mockLogicalPlan4Test) getPhysicalPlan2(prop *property.PhysicalProperty) PhysicalPlan {
|
|
physicalPlan2 := mockPhysicalPlan4Test{planType: 2, costOverflow: p.costOverflow}.Init(p.ctx)
|
|
physicalPlan2.stats = &property.StatsInfo{RowCount: 1}
|
|
physicalPlan2.childrenReqProps = make([]*property.PhysicalProperty, 1)
|
|
physicalPlan2.childrenReqProps[0] = property.NewPhysicalProperty(prop.TaskTp, nil, false, prop.ExpectedCnt, false)
|
|
return physicalPlan2
|
|
}
|
|
|
|
func (p *mockLogicalPlan4Test) exhaustPhysicalPlans(prop *property.PhysicalProperty) ([]PhysicalPlan, bool) {
|
|
plan1 := make([]PhysicalPlan, 0, 1)
|
|
plan2 := make([]PhysicalPlan, 0, 1)
|
|
if prop.IsEmpty() && p.canGeneratePlan2 {
|
|
// Generate PhysicalPlan2 when the property is empty.
|
|
plan2 = append(plan2, p.getPhysicalPlan2(prop))
|
|
if p.hasHintForPlan2 {
|
|
return plan2, true
|
|
}
|
|
}
|
|
if all, _ := prop.AllSameOrder(); all {
|
|
// Generate PhysicalPlan1 when properties are the same order.
|
|
plan1 = append(plan1, p.getPhysicalPlan1(prop))
|
|
}
|
|
if p.hasHintForPlan2 {
|
|
// The hint cannot work.
|
|
if prop.IsEmpty() {
|
|
p.ctx.GetSessionVars().StmtCtx.AppendWarning(fmt.Errorf("the hint is inapplicable for plan2"))
|
|
}
|
|
return plan1, false
|
|
}
|
|
return append(plan1, plan2...), true
|
|
}
|
|
|
|
type mockPhysicalPlan4Test struct {
|
|
basePhysicalPlan
|
|
// 1 or 2 for physicalPlan1 or physicalPlan2.
|
|
// See the comment of mockLogicalPlan4Test.
|
|
planType int
|
|
costOverflow bool
|
|
}
|
|
|
|
func (p mockPhysicalPlan4Test) Init(ctx sessionctx.Context) *mockPhysicalPlan4Test {
|
|
p.basePhysicalPlan = newBasePhysicalPlan(ctx, "mockPlan", &p, 0)
|
|
return &p
|
|
}
|
|
|
|
func (p *mockPhysicalPlan4Test) attach2Task(tasks ...task) task {
|
|
t := tasks[0].copy()
|
|
attachPlan2Task(p, t)
|
|
if p.costOverflow {
|
|
t.addCost(math.MaxFloat64)
|
|
} else {
|
|
t.addCost(1)
|
|
}
|
|
return t
|
|
}
|
|
|
|
func (s *testFindBestTaskSuite) TestCostOverflow(c *C) {
|
|
ctx := MockContext()
|
|
// Plan Tree: mockPlan -> mockDataSource
|
|
mockPlan := mockLogicalPlan4Test{costOverflow: true}.Init(ctx)
|
|
mockDS := mockDataSource{}.Init(ctx)
|
|
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)
|
|
c.Assert(err, IsNil)
|
|
// The cost should be overflowed, but the task shouldn't be invalid.
|
|
c.Assert(t.invalid(), IsFalse)
|
|
c.Assert(t.cost(), Equals, math.MaxFloat64)
|
|
}
|
|
|
|
func (s *testFindBestTaskSuite) TestEnforcedProperty(c *C) {
|
|
ctx := MockContext()
|
|
// PlanTree : mockLogicalPlan -> mockDataSource
|
|
mockPlan := mockLogicalPlan4Test{}.Init(ctx)
|
|
mockDS := mockDataSource{}.Init(ctx)
|
|
mockPlan.SetChildren(mockDS)
|
|
|
|
col0 := &expression.Column{UniqueID: 1}
|
|
col1 := &expression.Column{UniqueID: 2}
|
|
// Use different order, so that mockLogicalPlan cannot generate any of the
|
|
// physical plans.
|
|
item0 := property.Item{Col: col0, Desc: false}
|
|
item1 := property.Item{Col: col1, Desc: true}
|
|
items := []property.Item{item0, item1}
|
|
|
|
prop0 := &property.PhysicalProperty{
|
|
Items: items,
|
|
Enforced: false,
|
|
}
|
|
// should return invalid task because no physical plan can match this property.
|
|
task, err := mockPlan.findBestTask(prop0)
|
|
c.Assert(err, IsNil)
|
|
c.Assert(task.invalid(), IsTrue)
|
|
|
|
prop1 := &property.PhysicalProperty{
|
|
Items: items,
|
|
Enforced: true,
|
|
}
|
|
// should return the valid task when the property is enforced.
|
|
task, err = mockPlan.findBestTask(prop1)
|
|
c.Assert(err, IsNil)
|
|
c.Assert(task.invalid(), IsFalse)
|
|
}
|
|
|
|
func (s *testFindBestTaskSuite) TestHintCannotFitProperty(c *C) {
|
|
ctx := MockContext()
|
|
// PlanTree : mockLogicalPlan -> mockDataSource
|
|
mockPlan0 := mockLogicalPlan4Test{
|
|
hasHintForPlan2: true,
|
|
canGeneratePlan2: true,
|
|
}.Init(ctx)
|
|
mockDS := mockDataSource{}.Init(ctx)
|
|
mockPlan0.SetChildren(mockDS)
|
|
|
|
col0 := &expression.Column{UniqueID: 1}
|
|
item0 := property.Item{Col: col0}
|
|
items := []property.Item{item0}
|
|
// case 1, The property is not empty and enforced, should enforce a sort.
|
|
prop0 := &property.PhysicalProperty{
|
|
Items: items,
|
|
Enforced: true,
|
|
}
|
|
task, err := mockPlan0.findBestTask(prop0)
|
|
c.Assert(err, IsNil)
|
|
c.Assert(task.invalid(), IsFalse)
|
|
_, enforcedSort := task.plan().(*PhysicalSort)
|
|
c.Assert(enforcedSort, IsTrue)
|
|
plan2 := task.plan().Children()[0]
|
|
mockPhysicalPlan, ok := plan2.(*mockPhysicalPlan4Test)
|
|
c.Assert(ok, IsTrue)
|
|
c.Assert(mockPhysicalPlan.planType, Equals, 2)
|
|
|
|
// case 2, The property is not empty but not enforced, still need to enforce a sort
|
|
// to ensure the hint can work
|
|
prop1 := &property.PhysicalProperty{
|
|
Items: items,
|
|
Enforced: false,
|
|
}
|
|
task, err = mockPlan0.findBestTask(prop1)
|
|
c.Assert(err, IsNil)
|
|
c.Assert(task.invalid(), IsFalse)
|
|
_, enforcedSort = task.plan().(*PhysicalSort)
|
|
c.Assert(enforcedSort, IsTrue)
|
|
plan2 = task.plan().Children()[0]
|
|
mockPhysicalPlan, ok = plan2.(*mockPhysicalPlan4Test)
|
|
c.Assert(ok, IsTrue)
|
|
c.Assert(mockPhysicalPlan.planType, Equals, 2)
|
|
|
|
// case 3, The hint cannot work even if the property is empty, should return a warning
|
|
// and generate physicalPlan1.
|
|
prop2 := &property.PhysicalProperty{
|
|
Items: items,
|
|
Enforced: false,
|
|
}
|
|
mockPlan1 := mockLogicalPlan4Test{
|
|
hasHintForPlan2: true,
|
|
canGeneratePlan2: false,
|
|
}.Init(ctx)
|
|
mockPlan1.SetChildren(mockDS)
|
|
task, err = mockPlan1.findBestTask(prop2)
|
|
c.Assert(err, IsNil)
|
|
c.Assert(task.invalid(), IsFalse)
|
|
c.Assert(ctx.GetSessionVars().StmtCtx.WarningCount(), Equals, uint16(1))
|
|
// Because physicalPlan1 can match the property, so we should get it.
|
|
mockPhysicalPlan, ok = task.plan().(*mockPhysicalPlan4Test)
|
|
c.Assert(ok, IsTrue)
|
|
c.Assert(mockPhysicalPlan.planType, Equals, 1)
|
|
|
|
// case 4, Similar to case 3, but the property is enforced now. Ths result should be
|
|
// the same with case 3.
|
|
ctx.GetSessionVars().StmtCtx.SetWarnings(nil)
|
|
prop3 := &property.PhysicalProperty{
|
|
Items: items,
|
|
Enforced: true,
|
|
}
|
|
task, err = mockPlan1.findBestTask(prop3)
|
|
c.Assert(err, IsNil)
|
|
c.Assert(task.invalid(), IsFalse)
|
|
c.Assert(ctx.GetSessionVars().StmtCtx.WarningCount(), Equals, uint16(1))
|
|
// Because physicalPlan1 can match the property, so we don't need to enforce a sort.
|
|
mockPhysicalPlan, ok = task.plan().(*mockPhysicalPlan4Test)
|
|
c.Assert(ok, IsTrue)
|
|
c.Assert(mockPhysicalPlan.planType, Equals, 1)
|
|
}
|
|
|