From ab82ca57c52ce637b24308efa8740f2de81e6875 Mon Sep 17 00:00:00 2001 From: arenatlx <314806019@qq.com> Date: Wed, 4 Dec 2024 16:09:47 +0800 Subject: [PATCH 01/25] . Signed-off-by: arenatlx <314806019@qq.com> --- pkg/planner/cascades/memo/BUILD.bazel | 2 +- pkg/planner/cascades/memo/group.go | 30 ++++- pkg/planner/cascades/memo/group_expr.go | 38 ++++++ pkg/planner/cascades/memo/memo.go | 120 ++++++++++++------ pkg/planner/cascades/memo/memo_test.go | 41 +++--- pkg/planner/cascades/rule/binder_test.go | 2 +- .../core/casetest/cascades/BUILD.bazel | 27 ++++ .../core/casetest/cascades/main_test.go | 53 ++++++++ .../core/casetest/cascades/memo_test.go | 116 +++++++++++++++++ .../cascades/testdata/cascades_suite_in.json | 57 +++++++++ .../cascades/testdata/cascades_suite_out.json | 107 ++++++++++++++++ pkg/planner/core/casetest/stats_test.go | 1 + pkg/planner/property/logical_property.go | 5 + pkg/testkit/testdata/testdata.go | 2 +- 14 files changed, 540 insertions(+), 61 deletions(-) create mode 100644 pkg/planner/core/casetest/cascades/BUILD.bazel create mode 100644 pkg/planner/core/casetest/cascades/main_test.go create mode 100644 pkg/planner/core/casetest/cascades/memo_test.go create mode 100644 pkg/planner/core/casetest/cascades/testdata/cascades_suite_in.json create mode 100644 pkg/planner/core/casetest/cascades/testdata/cascades_suite_out.json diff --git a/pkg/planner/cascades/memo/BUILD.bazel b/pkg/planner/cascades/memo/BUILD.bazel index 64cf751ca9ec2..c2722b9d235d0 100644 --- a/pkg/planner/cascades/memo/BUILD.bazel +++ b/pkg/planner/cascades/memo/BUILD.bazel @@ -11,12 +11,12 @@ go_library( importpath = "github.com/pingcap/tidb/pkg/planner/cascades/memo", visibility = ["//visibility:public"], deps = [ + "//pkg/expression", "//pkg/planner/cascades/base", "//pkg/planner/cascades/pattern", "//pkg/planner/cascades/util", "//pkg/planner/core/base", "//pkg/planner/property", - "//pkg/sessionctx", "//pkg/util/intest", ], ) diff --git a/pkg/planner/cascades/memo/group.go b/pkg/planner/cascades/memo/group.go index 1fc1887f26e72..f19c1f9a37179 100644 --- a/pkg/planner/cascades/memo/group.go +++ b/pkg/planner/cascades/memo/group.go @@ -127,9 +127,37 @@ func (g *Group) GetFirstElem(operand pattern.Operand) *list.Element { return g.Operand2FirstExpr[operand] } +// GetLogicalProperty return this group's logical property. +func (g *Group) GetLogicalProperty() *property.LogicalProperty { + return g.logicalProp +} + +// SetLogicalProperty set this group's logical property. +func (g *Group) SetLogicalProperty(prop *property.LogicalProperty) { + g.logicalProp = prop +} + +// IsExplored returns whether this group is explored. +func (g *Group) IsExplored() bool { + return g.explored +} + +// SetExplored set the group as tagged as explored. +func (g *Group) SetExplored() { + g.explored = true +} + // String implements fmt.Stringer interface. func (g *Group) String(w util.StrBufferWriter) { - w.WriteString(fmt.Sprintf("inputs:%s", strconv.Itoa(int(g.groupID)))) + w.WriteString(fmt.Sprintf("GID:%s", strconv.Itoa(int(g.groupID)))) +} + +// ForEachGE traverse the inside group expression with f call on them each. +func (g *Group) ForEachGE(f func(ge *GroupExpression)) { + for elem := g.logicalExpressions.Front(); elem != nil; elem = elem.Next() { + expr := elem.Value.(*GroupExpression) + f(expr) + } } // NewGroup creates a new Group with given logical prop. diff --git a/pkg/planner/cascades/memo/group_expr.go b/pkg/planner/cascades/memo/group_expr.go index 9c8d1c83a6115..bcb457da8bde2 100644 --- a/pkg/planner/cascades/memo/group_expr.go +++ b/pkg/planner/cascades/memo/group_expr.go @@ -15,10 +15,12 @@ package memo import ( + "github.com/pingcap/tidb/pkg/expression" base2 "github.com/pingcap/tidb/pkg/planner/cascades/base" "github.com/pingcap/tidb/pkg/planner/cascades/pattern" "github.com/pingcap/tidb/pkg/planner/cascades/util" "github.com/pingcap/tidb/pkg/planner/core/base" + "github.com/pingcap/tidb/pkg/planner/property" "github.com/pingcap/tidb/pkg/util/intest" ) @@ -122,3 +124,39 @@ func (e *GroupExpression) Init(h base2.Hasher) { e.Hash64(h) e.hash64 = h.Sum64() } + +// DeriveLogicalProp derive the new group's logical property from a specific GE. +// DeriveLogicalProp is not called with recursive, because we only examine and +// init new group from bottom-up, so we can sure that this new group's children +// has already gotten its logical prop. +func (e *GroupExpression) DeriveLogicalProp() (err error) { + if e.GetGroup().GetLogicalProperty() != nil { + return nil + } + childStats := make([]*property.StatsInfo, 0, len(e.Inputs)) + childSchema := make([]*expression.Schema, 0, len(e.Inputs)) + for _, childG := range e.Inputs { + childGProp := childG.GetLogicalProperty() + intest.Assert(childGProp != nil) + childStats = append(childStats, childGProp.Stats) + childSchema = append(childSchema, childGProp.Schema) + } + e.GetGroup().SetLogicalProperty(property.NewLogicalProp()) + // currently the schemaProducer side logical op is still useful for group schema. + // just add this mock for a mocked logical-plan which is with the id less than 0. + // todo: functional dependency + tmpSchema := e.LogicalPlan.Schema() + tmpStats := e.LogicalPlan.StatsInfo() + // only for those new created logical op from XForm, we should rebuild their stats; + // in memo init phase, all logical ops has maintained their stats already, just use them. + if tmpStats == nil && e.LogicalPlan.ID() > 0 { + // here can only derive the basic stats from bottom up, we can't pass any colGroups required by parents. + tmpStats, err = e.LogicalPlan.DeriveStats(childStats, tmpSchema, childSchema, nil) + if err != nil { + return err + } + } + e.GetGroup().GetLogicalProperty().Schema = tmpSchema + e.GetGroup().GetLogicalProperty().Stats = tmpStats + return nil +} diff --git a/pkg/planner/cascades/memo/memo.go b/pkg/planner/cascades/memo/memo.go index f6b0438f02e41..9edc4e94fc0e6 100644 --- a/pkg/planner/cascades/memo/memo.go +++ b/pkg/planner/cascades/memo/memo.go @@ -19,17 +19,13 @@ import ( base2 "github.com/pingcap/tidb/pkg/planner/cascades/base" "github.com/pingcap/tidb/pkg/planner/core/base" - "github.com/pingcap/tidb/pkg/sessionctx" "github.com/pingcap/tidb/pkg/util/intest" ) // Memo is the main structure of the memo package. type Memo struct { - // ctx is the context of the memo. - sCtx sessionctx.Context - // groupIDGen is the incremental group id for internal usage. - groupIDGen GroupIDGenerator + groupIDGen *GroupIDGenerator // rootGroup is the root group of the memo. rootGroup *Group @@ -48,10 +44,9 @@ type Memo struct { } // NewMemo creates a new memo. -func NewMemo(ctx sessionctx.Context) *Memo { +func NewMemo() *Memo { return &Memo{ - sCtx: ctx, - groupIDGen: GroupIDGenerator{id: 0}, + groupIDGen: &GroupIDGenerator{id: 0}, groups: list.New(), groupID2Group: make(map[GroupID]*list.Element), hasher: base2.NewHashEqualer(), @@ -59,73 +54,118 @@ func NewMemo(ctx sessionctx.Context) *Memo { } // GetHasher gets a hasher from the memo that ready to use. -func (m *Memo) GetHasher() base2.Hasher { - m.hasher.Reset() - return m.hasher +func (mm *Memo) GetHasher() base2.Hasher { + mm.hasher.Reset() + return mm.hasher } -// CopyIn copies a logical plan into the memo with format as GroupExpression. -func (m *Memo) CopyIn(target *Group, lp base.LogicalPlan) *GroupExpression { +// CopyIn copies a MemoExpression representation into the memo with format as GroupExpression inside. +// The generic logical forest inside memo is represented as memo group expression tree, while for entering +// and re-feeding the memo, we use the memoExpression as the currency: +// +// entering(init memo) +// +// lp ┌──────────┐ +// / \ │ memo: │ +// lp lp --copyIN-> │ G(ge) │ +// / \ │ / \ │ +// ... ... │ G G │ +// └──────────┘ +// +// re-feeding (intake XForm output) +// +// lp ┌──────────┐ +// / \ │ memo: │ +// GE lp --copyIN-> │ G(ge) │ +// | │ / \ │ +// GE │ G G │ +// └──────────┘ +// +// the bare lp means the new created logical op or that whose child has changed which invalidate it's original +// old belonged group, make it back to bare-lp for re-inserting again in copyIn. +func (mm *Memo) CopyIn(target *Group, lp base.LogicalPlan) (*GroupExpression, error) { // Group the children first. childGroups := make([]*Group, 0, len(lp.Children())) for _, child := range lp.Children() { - // todo: child.getGroupExpression.GetGroup directly - groupExpr := m.CopyIn(nil, child) - group := groupExpr.group - intest.Assert(group != nil) - intest.Assert(group != target) - childGroups = append(childGroups, group) + var currentChildG *Group + if ge, ok := child.(*GroupExpression); ok { + // which means it's the earliest unchanged GroupExpression from rule XForm. + currentChildG = ge.GetGroup() + } else { + // which means it's a new/changed logical op, downward to get its input group ids to complete it. + ge, err := mm.CopyIn(nil, child) + if err != nil { + return nil, err + } + currentChildG = ge.GetGroup() + } + intest.Assert(currentChildG != nil) + intest.Assert(currentChildG != target) + childGroups = append(childGroups, currentChildG) } - - hasher := m.GetHasher() + hasher := mm.GetHasher() groupExpr := NewGroupExpression(lp, childGroups) groupExpr.Init(hasher) - m.insertGroupExpression(groupExpr, target) - // todo: new group need to derive the logical property. - return groupExpr + if mm.InsertGroupExpression(groupExpr, target) && target == nil { + // derive logical property for new group. + err := groupExpr.DeriveLogicalProp() + if err != nil { + return nil, err + } + } + return groupExpr, nil } // GetGroups gets all groups in the memo. -func (m *Memo) GetGroups() *list.List { - return m.groups +func (mm *Memo) GetGroups() *list.List { + return mm.groups } // GetGroupID2Group gets the map from group id to group. -func (m *Memo) GetGroupID2Group() map[GroupID]*list.Element { - return m.groupID2Group +func (mm *Memo) GetGroupID2Group() map[GroupID]*list.Element { + return mm.groupID2Group } // GetRootGroup gets the root group of the memo. -func (m *Memo) GetRootGroup() *Group { - return m.rootGroup +func (mm *Memo) GetRootGroup() *Group { + return mm.rootGroup } +// InsertGroupExpression insert ge into a target group. // @bool indicates whether the groupExpr is inserted to a new group. -func (m *Memo) insertGroupExpression(groupExpr *GroupExpression, target *Group) bool { +func (mm *Memo) InsertGroupExpression(groupExpr *GroupExpression, target *Group) bool { // for group merge, here groupExpr is the new groupExpr with undetermined belonged group. // we need to use groupExpr hash to find whether there is same groupExpr existed before. // if existed and the existed groupExpr.Group is not same with target, we should merge them up. // todo: merge group if target == nil { - target = m.NewGroup() - m.groups.PushBack(target) - m.groupID2Group[target.groupID] = m.groups.Back() + target = mm.NewGroup() + mm.groups.PushBack(target) + mm.groupID2Group[target.groupID] = mm.groups.Back() } target.Insert(groupExpr) return true } // NewGroup creates a new group. -func (m *Memo) NewGroup() *Group { +func (mm *Memo) NewGroup() *Group { group := NewGroup(nil) - group.groupID = m.groupIDGen.NextGroupID() + group.groupID = mm.groupIDGen.NextGroupID() return group } // Init initializes the memo with a logical plan, converting logical plan tree format into group tree. -func (m *Memo) Init(plan base.LogicalPlan) *GroupExpression { - intest.Assert(m.groups.Len() == 0) - gE := m.CopyIn(nil, plan) - m.rootGroup = gE.GetGroup() +func (mm *Memo) Init(plan base.LogicalPlan) *GroupExpression { + intest.Assert(mm.groups.Len() == 0) + gE, _ := mm.CopyIn(nil, plan) + mm.rootGroup = gE.GetGroup() return gE } + +// ForEachGroup traverse the inside group expression with f call on them each. +func (mm *Memo) ForEachGroup(f func(g *Group)) { + for elem := mm.GetGroups().Front(); elem != nil; elem = elem.Next() { + expr := elem.Value.(*Group) + f(expr) + } +} diff --git a/pkg/planner/cascades/memo/memo_test.go b/pkg/planner/cascades/memo/memo_test.go index 38018780197e3..513a3868ad14e 100644 --- a/pkg/planner/cascades/memo/memo_test.go +++ b/pkg/planner/cascades/memo/memo_test.go @@ -25,18 +25,21 @@ import ( func TestMemo(t *testing.T) { ctx := plannercore.MockContext() t1 := logicalop.DataSource{}.Init(ctx, 0) + t1.SetID(-1) t2 := logicalop.DataSource{}.Init(ctx, 0) + t2.SetID(-2) join := logicalop.LogicalJoin{}.Init(ctx, 0) + join.SetID(-3) join.SetChildren(t1, t2) - memo := NewMemo(ctx) - memo.Init(join) - require.Equal(t, 3, memo.groups.Len()) - require.Equal(t, 3, len(memo.groupID2Group)) + mm := NewMemo() + mm.Init(join) + require.Equal(t, 3, mm.GetGroups().Len()) + require.Equal(t, 3, len(mm.GetGroupID2Group())) // iter memo.groups to assert group ids. cnt := 1 - for e := memo.groups.Front(); e != nil; e = e.Next() { + for e := mm.GetGroups().Front(); e != nil; e = e.Next() { group := e.Value.(*Group) require.NotNil(t, group) require.Equal(t, GroupID(cnt), group.groupID) @@ -47,33 +50,37 @@ func TestMemo(t *testing.T) { func TestInsertGE(t *testing.T) { ctx := plannercore.MockContext() t1 := logicalop.DataSource{}.Init(ctx, 0) + t1.SetID(-1) t2 := logicalop.DataSource{}.Init(ctx, 0) + t2.SetID(-2) join := logicalop.LogicalJoin{}.Init(ctx, 0) + join.SetID(-3) join.SetChildren(t1, t2) - memo := NewMemo(ctx) - memo.Init(join) - require.Equal(t, 3, memo.groups.Len()) - require.Equal(t, 3, len(memo.groupID2Group)) + mm := NewMemo() + mm.Init(join) + require.Equal(t, 3, mm.GetGroups().Len()) + require.Equal(t, 3, len(mm.GetGroupID2Group())) // prepare a new group expression with join's group as its children. limit := logicalop.LogicalLimit{}.Init(ctx, 0) - hasher := memo.GetHasher() - groupExpr := NewGroupExpression(limit, []*Group{memo.rootGroup}) + limit.SetID(-4) + hasher := mm.GetHasher() + groupExpr := NewGroupExpression(limit, []*Group{mm.GetRootGroup()}) groupExpr.Init(hasher) // Insert a new group with a new expression. - memo.insertGroupExpression(groupExpr, nil) - require.Equal(t, 4, memo.groups.Len()) - require.Equal(t, 4, len(memo.groupID2Group)) + mm.InsertGroupExpression(groupExpr, nil) + require.Equal(t, 4, mm.GetGroups().Len()) + require.Equal(t, 4, len(mm.GetGroupID2Group())) // iter memo.groups to assert group ids. cnt := 1 - for e := memo.groups.Front(); e != nil; e = e.Next() { + for e := mm.GetGroups().Front(); e != nil; e = e.Next() { group := e.Value.(*Group) require.NotNil(t, group) - require.Equal(t, GroupID(cnt), group.groupID) + require.Equal(t, GroupID(cnt), group.GetGroupID()) cnt++ } - require.Equal(t, memo.groups.Back().Value.(*Group).groupID, GroupID(cnt-1)) + require.Equal(t, mm.GetGroups().Back().Value.(*Group).GetGroupID(), GroupID(cnt-1)) } diff --git a/pkg/planner/cascades/rule/binder_test.go b/pkg/planner/cascades/rule/binder_test.go index 44bea77656d16..d0a858d7cf331 100644 --- a/pkg/planner/cascades/rule/binder_test.go +++ b/pkg/planner/cascades/rule/binder_test.go @@ -107,7 +107,7 @@ func TestBinderFail(t *testing.T) { require.Equal(t, b.String(), "") // renew memo - mm = memo.NewMemo(ctx) + mm = memo.NewMemo() mm.Init(p1) rootGE = mm.GetRootGroup().GetLogicalExpressions().Back().Value.(*memo.GroupExpression) binder = NewBinder(pa, rootGE) diff --git a/pkg/planner/core/casetest/cascades/BUILD.bazel b/pkg/planner/core/casetest/cascades/BUILD.bazel new file mode 100644 index 0000000000000..03afd3005333c --- /dev/null +++ b/pkg/planner/core/casetest/cascades/BUILD.bazel @@ -0,0 +1,27 @@ +load("@io_bazel_rules_go//go:def.bzl", "go_test") + +go_test( + name = "cascades_test", + timeout = "short", + srcs = [ + "main_test.go", + "memo_test.go", + ], + data = glob(["testdata/**"]), + flaky = True, + deps = [ + "//pkg/parser", + "//pkg/planner/cascades/memo", + "//pkg/planner/cascades/util", + "//pkg/planner/core", + "//pkg/planner/core/base", + "//pkg/planner/core/resolve", + "//pkg/testkit", + "//pkg/testkit/testdata", + "//pkg/testkit/testmain", + "//pkg/testkit/testsetup", + "//pkg/util/hint", + "@com_github_stretchr_testify//require", + "@org_uber_go_goleak//:goleak", + ], +) diff --git a/pkg/planner/core/casetest/cascades/main_test.go b/pkg/planner/core/casetest/cascades/main_test.go new file mode 100644 index 0000000000000..0f14a5abcfe3d --- /dev/null +++ b/pkg/planner/core/casetest/cascades/main_test.go @@ -0,0 +1,53 @@ +// 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, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package cascades + +import ( + "flag" + "testing" + + "github.com/pingcap/tidb/pkg/testkit/testdata" + "github.com/pingcap/tidb/pkg/testkit/testmain" + "github.com/pingcap/tidb/pkg/testkit/testsetup" + "go.uber.org/goleak" +) + +var testDataMap = make(testdata.BookKeeper) + +func TestMain(m *testing.M) { + testsetup.SetupForCommonTest() + flag.Parse() + testDataMap.LoadTestSuiteData("testdata", "cascades_suite") + opts := []goleak.Option{ + goleak.IgnoreTopFunction("github.com/golang/glog.(*fileSink).flushDaemon"), + goleak.IgnoreTopFunction("github.com/bazelbuild/rules_go/go/tools/bzltestutil.RegisterTimeoutHandler.func1"), + goleak.IgnoreTopFunction("github.com/lestrrat-go/httprc.runFetchWorker"), + goleak.IgnoreTopFunction("go.etcd.io/etcd/client/pkg/v3/logutil.(*MergeLogger).outputLoop"), + goleak.IgnoreTopFunction("gopkg.in/natefinch/lumberjack%2ev2.(*Logger).millRun"), + goleak.IgnoreTopFunction("github.com/tikv/client-go/v2/txnkv/transaction.keepAlive"), + goleak.IgnoreTopFunction("go.opencensus.io/stats/view.(*worker).start"), + } + + callback := func(i int) int { + testDataMap.GenerateOutputIfNeeded() + return i + } + + goleak.VerifyTestMain(testmain.WrapTestingM(m, callback), opts...) +} + +func GetCascadesSuiteData() testdata.TestData { + return testDataMap["cascades_suite"] +} diff --git a/pkg/planner/core/casetest/cascades/memo_test.go b/pkg/planner/core/casetest/cascades/memo_test.go new file mode 100644 index 0000000000000..6f926d4841d0c --- /dev/null +++ b/pkg/planner/core/casetest/cascades/memo_test.go @@ -0,0 +1,116 @@ +// 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, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package cascades + +import ( + "bytes" + "context" + "strings" + "testing" + + "github.com/pingcap/tidb/pkg/parser" + "github.com/pingcap/tidb/pkg/planner/cascades/memo" + "github.com/pingcap/tidb/pkg/planner/cascades/util" + plannercore "github.com/pingcap/tidb/pkg/planner/core" + "github.com/pingcap/tidb/pkg/planner/core/base" + "github.com/pingcap/tidb/pkg/planner/core/resolve" + "github.com/pingcap/tidb/pkg/testkit" + "github.com/pingcap/tidb/pkg/testkit/testdata" + "github.com/pingcap/tidb/pkg/util/hint" + "github.com/stretchr/testify/require" +) + +func TestDeriveStats(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t1, t2") + tk.MustExec("create table t1(a int not null, b int not null, key(a,b))") + tk.MustExec("insert into t1 values(1,1),(1,2),(2,1),(2,2),(1,1)") + tk.MustExec("create table t2(a int not null, b int not null, key(a,b))") + tk.MustExec("insert into t2 values(1,1),(1,2),(1,3),(2,1),(2,2),(2,3),(3,1),(3,2),(3,3),(1,1)") + tk.MustExec("analyze table t1") + tk.MustExec("analyze table t2") + + ctx := context.Background() + p := parser.New() + var input []string + var output []struct { + SQL string + Str string + } + statsSuiteData := GetCascadesSuiteData() + statsSuiteData.LoadTestCases(t, &input, &output) + for i, tt := range input { + stmt, err := p.ParseOneStmt(tt, "", "") + require.NoError(t, err, tt) + ret := &plannercore.PreprocessorReturn{} + nodeW := resolve.NewNodeW(stmt) + err = plannercore.Preprocess(context.Background(), tk.Session(), nodeW, plannercore.WithPreprocessorReturn(ret)) + require.NoError(t, err) + tk.Session().GetSessionVars().PlanColumnID.Store(0) + builder, _ := plannercore.NewPlanBuilder().Init(tk.Session().GetPlanCtx(), ret.InfoSchema, hint.NewQBHintHandler(nil)) + p, err := builder.Build(ctx, nodeW) + require.NoError(t, err, tt) + p, err = plannercore.LogicalOptimizeTest(ctx, builder.GetOptFlag(), p.(base.LogicalPlan)) + require.NoError(t, err, tt) + lp := p.(base.LogicalPlan) + _, err = plannercore.RecursiveDeriveStats4Test(lp) + require.NoError(t, err, tt) + // after stats derive is done, which means the up-down propagation of group ndv is done, in bottom-up building phase + // of memo, we don't have to expect the upper operator's group cols passing down anymore. + mm := memo.NewMemo() + mm.Init(lp) + // check the stats state in memo group. + str := strings.Builder{} + b := &bytes.Buffer{} + mm.ForEachGroup(func(g *memo.Group) { + if str.Len() != 0 { + str.WriteString("\n") + } + b.Reset() + sb := util.NewStrBuffer(b) + // record group + g.String(sb) + sb.WriteString(", ") + // record group stats + logicProp := g.GetLogicalProperty() + if logicProp == nil { + sb.WriteString("logic prop:nil") + } else { + sb.WriteString("logic prop:{") + if logicProp.Stats == nil { + sb.WriteString("stats:nil,") + } else { + sb.WriteString("stats:{" + logicProp.Stats.String() + "}") + } + sb.WriteString(", ") + if logicProp.Schema == nil { + sb.WriteString("schema:nil") + } else { + sb.WriteString("schema:{" + logicProp.Schema.String() + "}") + } + sb.WriteString("}") + } + sb.Flush() + str.WriteString(b.String()) + }) + testdata.OnRecord(func() { + output[i].SQL = tt + output[i].Str = str.String() + }) + require.Equal(t, output[i].Str, str.String(), "case i "+tt) + } +} diff --git a/pkg/planner/core/casetest/cascades/testdata/cascades_suite_in.json b/pkg/planner/core/casetest/cascades/testdata/cascades_suite_in.json new file mode 100644 index 0000000000000..b87d5e655a16e --- /dev/null +++ b/pkg/planner/core/casetest/cascades/testdata/cascades_suite_in.json @@ -0,0 +1,57 @@ +[ + { + "name": "TestDeriveStats", + "cases": [ + // DataSource -> Aggregation. + "select count(1) from t1 group by a, b", + // DataSource -> Join. + "select * from t1, t2 where t1.a = t2.a and t1.b = t2.b", + // DataSource(Range) -> Aggregation. + "select count(1) from t1 where a > 0 group by a, b", + // DataSource(Selection) -> Aggregation. + "select count(1) from t1 where b > 0 group by a, b", + // DataSource -> Selection -> Aggregation. Change `cos` to another function if it can be pushed down to copr later. + "select count(1) from t1 where cos(a) > 0 group by a, b", + // DataSource -> Projection -> Aggregation. + "select count(c3) from (select a as c1, b as c2, a+1 as c3 from t1) as tmp group by c2, c1", + // DataSource -> Projection -> Aggregation. + "select count(c3) from (select a+b as c1, b as c2, a+1 as c3 from t1) as tmp group by c2, c1", + // DataSource -> Apply(LeftOuterJoin) -> Aggregation. + "select count(tmp.cmp) from (select t1.a as a, t1.b as b, (t1.b > (select t2.b from t2 where t2.a = t1.a)) as cmp from t1) tmp group by tmp.a, tmp.b", + // DataSource -> Apply(InnerJoin) -> Aggregation. + "select count(1) from (select t1.a as a, t1.b as b from t1 where t1.b > (select t2.b from t2 where t2.a = t1.a)) tmp group by tmp.a, tmp.b", + // DataSource -> Apply(LeftOuterSemiJoin) -> Aggregation. + "select count(tmp.cmp) from (select t1.a as a, t1.b as b, (t1.b in (select t2.b from t2 where t2.a = t1.a limit 3)) as cmp from t1) tmp group by tmp.a, tmp.b", + // DataSource -> Apply(AntiLeftOuterSemiJoin) -> Aggregation. + "select count(tmp.cmp) from (select t1.a as a, t1.b as b, (t1.b not in (select t2.b from t2 where t2.a = t1.a limit 3)) as cmp from t1) tmp group by tmp.a, tmp.b", + // DataSource -> Apply(SemiJoin) -> Aggregation. + "select count(1) from (select t1.a as a, t1.b as b from t1 where t1.b in (select t2.b from t2 where t2.a = t1.a limit 3)) tmp group by tmp.a, tmp.b", + // DataSource -> Apply(AntiSemiJoin) -> Aggregation. + "select count(1) from (select t1.a as a, t1.b as b from t1 where t1.b not in (select t2.b from t2 where t2.a = t1.a limit 3)) tmp group by tmp.a, tmp.b", + // DataSource -> InnerJoin -> Aggregation. + "select count(1) from t1, t2 where t1.a = t2.a group by t1.a, t1.b", + // DataSource -> LeftOuterJoin -> Aggregation. + "select count(1) from t1 left join t2 on t1.a = t2.a group by t1.a, t1.b", + // DataSource -> LeftOuterJoin -> Aggregation. + "select count(1) from t1 left join t2 on t1.a = t2.a group by t2.a, t2.b", + // DataSource -> RightOuterJoin -> Aggregation. + "select count(1) from t1 right join t2 on t1.a = t2.a group by t1.a, t1.b", + // DataSource -> RightOuterJoin -> Aggregation. + "select count(1) from t1 right join t2 on t1.a = t2.a group by t2.a, t2.b", + // DataSource -> LeftOuterSemiJoin -> Aggregation. + "select count(tmp.cmp) from (select t1.a as a, t1.b as b, (t1.b in (select t2.b from t2 where t2.a > t1.a)) as cmp from t1) tmp group by tmp.a, tmp.b", + // DataSource -> AntiLeftOuterSemiJoin -> Aggregation. + "select count(tmp.cmp) from (select t1.a as a, t1.b as b, (t1.b not in (select t2.b from t2 where t2.a > t1.a)) as cmp from t1) tmp group by tmp.a, tmp.b", + // DataSource -> SemiJoin -> Aggregation. + "select count(1) from (select t1.a as a, t1.b as b from t1 where t1.b in (select t2.b from t2 where t2.a > t1.a)) tmp group by tmp.a, tmp.b", + // DataSource -> AntiSemiJoin -> Aggregation. + "select count(1) from (select t1.a as a, t1.b as b from t1 where t1.b not in (select t2.b from t2 where t2.a > t1.a)) tmp group by tmp.a, tmp.b", + // DataSource -> Aggregation -> Join. + "select * from t1 left join (select t2.a as a, t2.b as b, count(1) as cnt from t2 group by t2.a, t2.b) as tmp on t1.a = tmp.a and t1.b = tmp.b", + // DataSource -> Limit -> Aggregation. + "select count(1) from (select t1.a as a, t1.b as b from t1 limit 3) tmp group by tmp.a, tmp.b", + // DataSource -> Window -> Aggregation. + "select count(tmp.a_sum) from (select t1.a as a, t1.b as b, sum(a) over() as a_sum from t1) tmp group by tmp.a, tmp.b" + ] + } +] diff --git a/pkg/planner/core/casetest/cascades/testdata/cascades_suite_out.json b/pkg/planner/core/casetest/cascades/testdata/cascades_suite_out.json new file mode 100644 index 0000000000000..e93eca7ac022d --- /dev/null +++ b/pkg/planner/core/casetest/cascades/testdata/cascades_suite_out.json @@ -0,0 +1,107 @@ +[ + { + "Name": "TestDeriveStats", + "Cases": [ + { + "SQL": "select count(1) from t1 group by a, b", + "Str": "GID:1, logic prop:{stats:{count 5, ColNDVs map[1:2 2:2]}, schema:{Column: [test.t1.a,test.t1.b] Key: [] Unique key: []}}\nGID:2, logic prop:{stats:{count 4, ColNDVs map[4:4]}, schema:{Column: [Column#4] Key: [] Unique key: []}}\nGID:3, logic prop:{stats:{count 4, ColNDVs map[4:4]}, schema:{Column: [Column#4] Key: [] Unique key: []}}" + }, + { + "SQL": "select * from t1, t2 where t1.a = t2.a and t1.b = t2.b", + "Str": "GID:1, logic prop:{stats:{count 5, ColNDVs map[1:2 2:2]}, schema:{Column: [test.t1.a,test.t1.b] Key: [] Unique key: []}}\nGID:2, logic prop:{stats:{count 10, ColNDVs map[4:3 5:3]}, schema:{Column: [test.t2.a,test.t2.b] Key: [] Unique key: []}}\nGID:3, logic prop:{stats:{count 5.555555555555555, ColNDVs map[1:2 2:2 4:3 5:3]}, schema:{Column: [test.t1.a,test.t1.b,test.t2.a,test.t2.b] Key: [] Unique key: []}}\nGID:4, logic prop:{stats:{count 5.555555555555555, ColNDVs map[1:2 2:2 4:3 5:3]}, schema:{Column: [test.t1.a,test.t1.b,test.t2.a,test.t2.b] Key: [] Unique key: []}}" + }, + { + "SQL": "select count(1) from t1 where a > 0 group by a, b", + "Str": "GID:1, logic prop:{stats:{count 5, ColNDVs map[1:2 2:2]}, schema:{Column: [test.t1.a,test.t1.b] Key: [] Unique key: []}}\nGID:2, logic prop:{stats:{count 4, ColNDVs map[4:4]}, schema:{Column: [Column#4] Key: [] Unique key: []}}\nGID:3, logic prop:{stats:{count 4, ColNDVs map[4:4]}, schema:{Column: [Column#4] Key: [] Unique key: []}}" + }, + { + "SQL": "select count(1) from t1 where b > 0 group by a, b", + "Str": "GID:1, logic prop:{stats:{count 5, ColNDVs map[1:2 2:2]}, schema:{Column: [test.t1.a,test.t1.b] Key: [] Unique key: []}}\nGID:2, logic prop:{stats:{count 4, ColNDVs map[4:4]}, schema:{Column: [Column#4] Key: [] Unique key: []}}\nGID:3, logic prop:{stats:{count 4, ColNDVs map[4:4]}, schema:{Column: [Column#4] Key: [] Unique key: []}}" + }, + { + "SQL": "select count(1) from t1 where cos(a) > 0 group by a, b", + "Str": "GID:1, logic prop:{stats:{count 4, ColNDVs map[1:1.6 2:1.6]}, schema:{Column: [test.t1.a,test.t1.b] Key: [] Unique key: []}}\nGID:2, logic prop:{stats:{count 3.2, ColNDVs map[4:3.2]}, schema:{Column: [Column#4] Key: [] Unique key: []}}\nGID:3, logic prop:{stats:{count 3.2, ColNDVs map[4:3.2]}, schema:{Column: [Column#4] Key: [] Unique key: []}}" + }, + { + "SQL": "select count(c3) from (select a as c1, b as c2, a+1 as c3 from t1) as tmp group by c2, c1", + "Str": "GID:1, logic prop:{stats:{count 5, ColNDVs map[1:2 2:2]}, schema:{Column: [test.t1.a,test.t1.b] Key: [] Unique key: []}}\nGID:2, logic prop:{stats:{count 4, ColNDVs map[5:4]}, schema:{Column: [Column#5] Key: [] Unique key: []}}\nGID:3, logic prop:{stats:{count 4, ColNDVs map[5:4]}, schema:{Column: [Column#5] Key: [] Unique key: []}}" + }, + { + "SQL": "select count(c3) from (select a+b as c1, b as c2, a+1 as c3 from t1) as tmp group by c2, c1", + "Str": "GID:1, logic prop:{stats:{count 5, ColNDVs map[1:2 2:2]}, schema:{Column: [test.t1.a,test.t1.b] Key: [] Unique key: []}}\nGID:2, logic prop:{stats:{count 2, ColNDVs map[6:2]}, schema:{Column: [Column#6] Key: [] Unique key: []}}\nGID:3, logic prop:{stats:{count 2, ColNDVs map[6:2]}, schema:{Column: [Column#6] Key: [] Unique key: []}}" + }, + { + "SQL": "select count(tmp.cmp) from (select t1.a as a, t1.b as b, (t1.b > (select t2.b from t2 where t2.a = t1.a)) as cmp from t1) tmp group by tmp.a, tmp.b", + "Str": "GID:1, logic prop:{stats:{count 5, ColNDVs map[1:2 2:2]}, schema:{Column: [test.t1.a,test.t1.b] Key: [] Unique key: []}}\nGID:2, logic prop:{stats:{count 3.333333333333333, ColNDVs map[7:1 8:1]}, schema:{Column: [test.t2.a,test.t2.b] Key: [] Unique key: []}}\nGID:3, logic prop:{stats:{count 1, ColNDVs map[7:1 8:1]}, schema:{Column: [test.t2.a,test.t2.b] Key: [] Unique key: []}}\nGID:4, logic prop:{stats:{count 5, ColNDVs map[1:2 2:2 7:5 8:5]}, schema:{Column: [test.t1.a,test.t1.b,test.t2.a,test.t2.b] Key: [] Unique key: []}}\nGID:5, logic prop:{stats:{count 4, ColNDVs map[11:4]}, schema:{Column: [Column#11] Key: [] Unique key: []}}\nGID:6, logic prop:{stats:{count 4, ColNDVs map[11:4]}, schema:{Column: [Column#11] Key: [] Unique key: []}}" + }, + { + "SQL": "select count(1) from (select t1.a as a, t1.b as b from t1 where t1.b > (select t2.b from t2 where t2.a = t1.a)) tmp group by tmp.a, tmp.b", + "Str": "GID:1, logic prop:{stats:{count 5, ColNDVs map[1:2 2:2]}, schema:{Column: [test.t1.a,test.t1.b] Key: [] Unique key: []}}\nGID:2, logic prop:{stats:{count 3.333333333333333, ColNDVs map[4:1 5:1]}, schema:{Column: [test.t2.a,test.t2.b] Key: [] Unique key: []}}\nGID:3, logic prop:{stats:{count 1, ColNDVs map[4:1 5:1]}, schema:{Column: [test.t2.a,test.t2.b] Key: [] Unique key: []}}\nGID:4, logic prop:{stats:{count 0.8, ColNDVs map[4:0.8 5:0.8]}, schema:{Column: [test.t2.a,test.t2.b] Key: [] Unique key: []}}\nGID:5, logic prop:{stats:{count 5, ColNDVs map[1:2 2:2 4:5 5:5]}, schema:{Column: [test.t1.a,test.t1.b,test.t2.a,test.t2.b] Key: [] Unique key: []}}\nGID:6, logic prop:{stats:{count 2, ColNDVs map[7:2]}, schema:{Column: [Column#7] Key: [] Unique key: []}}\nGID:7, logic prop:{stats:{count 2, ColNDVs map[7:2]}, schema:{Column: [Column#7] Key: [] Unique key: []}}" + }, + { + "SQL": "select count(tmp.cmp) from (select t1.a as a, t1.b as b, (t1.b in (select t2.b from t2 where t2.a = t1.a limit 3)) as cmp from t1) tmp group by tmp.a, tmp.b", + "Str": "GID:1, logic prop:{stats:{count 5, ColNDVs map[1:2 2:2]}, schema:{Column: [test.t1.a,test.t1.b] Key: [] Unique key: []}}\nGID:2, logic prop:{stats:{count 3.333333333333333, ColNDVs map[7:1 8:1]}, schema:{Column: [test.t2.a,test.t2.b] Key: [] Unique key: []}}\nGID:3, logic prop:{stats:{count 3, ColNDVs map[7:1 8:1]}, schema:{Column: [test.t2.a,test.t2.b] Key: [] Unique key: []}}\nGID:4, logic prop:{stats:{count 5, ColNDVs map[1:2 2:2 10:2]}, schema:{Column: [test.t1.a,test.t1.b,Column#10] Key: [] Unique key: []}}\nGID:5, logic prop:{stats:{count 4, ColNDVs map[11:4]}, schema:{Column: [Column#11] Key: [] Unique key: []}}\nGID:6, logic prop:{stats:{count 4, ColNDVs map[11:4]}, schema:{Column: [Column#11] Key: [] Unique key: []}}" + }, + { + "SQL": "select count(tmp.cmp) from (select t1.a as a, t1.b as b, (t1.b not in (select t2.b from t2 where t2.a = t1.a limit 3)) as cmp from t1) tmp group by tmp.a, tmp.b", + "Str": "GID:1, logic prop:{stats:{count 5, ColNDVs map[1:2 2:2]}, schema:{Column: [test.t1.a,test.t1.b] Key: [] Unique key: []}}\nGID:2, logic prop:{stats:{count 3.333333333333333, ColNDVs map[7:1 8:1]}, schema:{Column: [test.t2.a,test.t2.b] Key: [] Unique key: []}}\nGID:3, logic prop:{stats:{count 3, ColNDVs map[7:1 8:1]}, schema:{Column: [test.t2.a,test.t2.b] Key: [] Unique key: []}}\nGID:4, logic prop:{stats:{count 5, ColNDVs map[1:2 2:2 10:2]}, schema:{Column: [test.t1.a,test.t1.b,Column#10] Key: [] Unique key: []}}\nGID:5, logic prop:{stats:{count 4, ColNDVs map[11:4]}, schema:{Column: [Column#11] Key: [] Unique key: []}}\nGID:6, logic prop:{stats:{count 4, ColNDVs map[11:4]}, schema:{Column: [Column#11] Key: [] Unique key: []}}" + }, + { + "SQL": "select count(1) from (select t1.a as a, t1.b as b from t1 where t1.b in (select t2.b from t2 where t2.a = t1.a limit 3)) tmp group by tmp.a, tmp.b", + "Str": "GID:1, logic prop:{stats:{count 5, ColNDVs map[1:2 2:2]}, schema:{Column: [test.t1.a,test.t1.b] Key: [] Unique key: []}}\nGID:2, logic prop:{stats:{count 3.333333333333333, ColNDVs map[4:1 5:1]}, schema:{Column: [test.t2.a,test.t2.b] Key: [] Unique key: []}}\nGID:3, logic prop:{stats:{count 3, ColNDVs map[4:1 5:1]}, schema:{Column: [test.t2.a,test.t2.b] Key: [] Unique key: []}}\nGID:4, logic prop:{stats:{count 5, ColNDVs map[1:2 2:2]}, schema:{Column: [test.t1.a,test.t1.b] Key: [] Unique key: []}}\nGID:5, logic prop:{stats:{count 2, ColNDVs map[7:2]}, schema:{Column: [Column#7] Key: [] Unique key: []}}\nGID:6, logic prop:{stats:{count 2, ColNDVs map[7:2]}, schema:{Column: [Column#7] Key: [] Unique key: []}}" + }, + { + "SQL": "select count(1) from (select t1.a as a, t1.b as b from t1 where t1.b not in (select t2.b from t2 where t2.a = t1.a limit 3)) tmp group by tmp.a, tmp.b", + "Str": "GID:1, logic prop:{stats:{count 5, ColNDVs map[1:2 2:2]}, schema:{Column: [test.t1.a,test.t1.b] Key: [] Unique key: []}}\nGID:2, logic prop:{stats:{count 3.333333333333333, ColNDVs map[4:1 5:1]}, schema:{Column: [test.t2.a,test.t2.b] Key: [] Unique key: []}}\nGID:3, logic prop:{stats:{count 3, ColNDVs map[4:1 5:1]}, schema:{Column: [test.t2.a,test.t2.b] Key: [] Unique key: []}}\nGID:4, logic prop:{stats:{count 5, ColNDVs map[1:2 2:2]}, schema:{Column: [test.t1.a,test.t1.b] Key: [] Unique key: []}}\nGID:5, logic prop:{stats:{count 2, ColNDVs map[7:2]}, schema:{Column: [Column#7] Key: [] Unique key: []}}\nGID:6, logic prop:{stats:{count 2, ColNDVs map[7:2]}, schema:{Column: [Column#7] Key: [] Unique key: []}}" + }, + { + "SQL": "select count(1) from t1, t2 where t1.a = t2.a group by t1.a, t1.b", + "Str": "GID:1, logic prop:{stats:{count 5, ColNDVs map[1:2 2:2]}, schema:{Column: [test.t1.a,test.t1.b] Key: [] Unique key: []}}\nGID:2, logic prop:{stats:{count 10, ColNDVs map[4:3]}, schema:{Column: [test.t2.a] Key: [] Unique key: []}}\nGID:3, logic prop:{stats:{count 16.666666666666668, ColNDVs map[1:2 2:2 4:3]}, schema:{Column: [test.t1.a,test.t1.b,test.t2.a] Key: [] Unique key: []}}\nGID:4, logic prop:{stats:{count 2, ColNDVs map[7:2]}, schema:{Column: [Column#7] Key: [] Unique key: []}}\nGID:5, logic prop:{stats:{count 2, ColNDVs map[7:2]}, schema:{Column: [Column#7] Key: [] Unique key: []}}" + }, + { + "SQL": "select count(1) from t1 left join t2 on t1.a = t2.a group by t1.a, t1.b", + "Str": "GID:1, logic prop:{stats:{count 5, ColNDVs map[1:2 2:2]}, schema:{Column: [test.t1.a,test.t1.b] Key: [] Unique key: []}}\nGID:2, logic prop:{stats:{count 10, ColNDVs map[4:3]}, schema:{Column: [test.t2.a] Key: [] Unique key: []}}\nGID:3, logic prop:{stats:{count 16.666666666666668, ColNDVs map[1:2 2:2 4:3]}, schema:{Column: [test.t1.a,test.t1.b,test.t2.a] Key: [] Unique key: []}}\nGID:4, logic prop:{stats:{count 4, ColNDVs map[7:4]}, schema:{Column: [Column#7] Key: [] Unique key: []}}\nGID:5, logic prop:{stats:{count 4, ColNDVs map[7:4]}, schema:{Column: [Column#7] Key: [] Unique key: []}}" + }, + { + "SQL": "select count(1) from t1 left join t2 on t1.a = t2.a group by t2.a, t2.b", + "Str": "GID:1, logic prop:{stats:{count 5, ColNDVs map[1:2]}, schema:{Column: [test.t1.a] Key: [] Unique key: []}}\nGID:2, logic prop:{stats:{count 10, ColNDVs map[4:3 5:3]}, schema:{Column: [test.t2.a,test.t2.b] Key: [] Unique key: []}}\nGID:3, logic prop:{stats:{count 16.666666666666668, ColNDVs map[1:2 4:3 5:3]}, schema:{Column: [test.t1.a,test.t2.a,test.t2.b] Key: [] Unique key: []}}\nGID:4, logic prop:{stats:{count 3, ColNDVs map[7:3]}, schema:{Column: [Column#7] Key: [] Unique key: []}}\nGID:5, logic prop:{stats:{count 3, ColNDVs map[7:3]}, schema:{Column: [Column#7] Key: [] Unique key: []}}" + }, + { + "SQL": "select count(1) from t1 right join t2 on t1.a = t2.a group by t1.a, t1.b", + "Str": "GID:1, logic prop:{stats:{count 5, ColNDVs map[1:2 2:2]}, schema:{Column: [test.t1.a,test.t1.b] Key: [] Unique key: []}}\nGID:2, logic prop:{stats:{count 10, ColNDVs map[4:3]}, schema:{Column: [test.t2.a] Key: [] Unique key: []}}\nGID:3, logic prop:{stats:{count 16.666666666666668, ColNDVs map[1:2 2:2 4:3]}, schema:{Column: [test.t1.a,test.t1.b,test.t2.a] Key: [] Unique key: []}}\nGID:4, logic prop:{stats:{count 2, ColNDVs map[7:2]}, schema:{Column: [Column#7] Key: [] Unique key: []}}\nGID:5, logic prop:{stats:{count 2, ColNDVs map[7:2]}, schema:{Column: [Column#7] Key: [] Unique key: []}}" + }, + { + "SQL": "select count(1) from t1 right join t2 on t1.a = t2.a group by t2.a, t2.b", + "Str": "GID:1, logic prop:{stats:{count 5, ColNDVs map[1:2]}, schema:{Column: [test.t1.a] Key: [] Unique key: []}}\nGID:2, logic prop:{stats:{count 10, ColNDVs map[4:3 5:3]}, schema:{Column: [test.t2.a,test.t2.b] Key: [] Unique key: []}}\nGID:3, logic prop:{stats:{count 16.666666666666668, ColNDVs map[1:2 4:3 5:3]}, schema:{Column: [test.t1.a,test.t2.a,test.t2.b] Key: [] Unique key: []}}\nGID:4, logic prop:{stats:{count 9, ColNDVs map[7:9]}, schema:{Column: [Column#7] Key: [] Unique key: []}}\nGID:5, logic prop:{stats:{count 9, ColNDVs map[7:9]}, schema:{Column: [Column#7] Key: [] Unique key: []}}" + }, + { + "SQL": "select count(tmp.cmp) from (select t1.a as a, t1.b as b, (t1.b in (select t2.b from t2 where t2.a > t1.a)) as cmp from t1) tmp group by tmp.a, tmp.b", + "Str": "GID:1, logic prop:{stats:{count 5, ColNDVs map[1:2 2:2]}, schema:{Column: [test.t1.a,test.t1.b] Key: [] Unique key: []}}\nGID:2, logic prop:{stats:{count 10, ColNDVs map[7:3 8:3]}, schema:{Column: [test.t2.a,test.t2.b] Key: [] Unique key: []}}\nGID:3, logic prop:{stats:{count 5, ColNDVs map[1:2 2:2 10:2]}, schema:{Column: [test.t1.a,test.t1.b,Column#10] Key: [] Unique key: []}}\nGID:4, logic prop:{stats:{count 4, ColNDVs map[11:4]}, schema:{Column: [Column#11] Key: [] Unique key: []}}\nGID:5, logic prop:{stats:{count 4, ColNDVs map[11:4]}, schema:{Column: [Column#11] Key: [] Unique key: []}}" + }, + { + "SQL": "select count(tmp.cmp) from (select t1.a as a, t1.b as b, (t1.b not in (select t2.b from t2 where t2.a > t1.a)) as cmp from t1) tmp group by tmp.a, tmp.b", + "Str": "GID:1, logic prop:{stats:{count 5, ColNDVs map[1:2 2:2]}, schema:{Column: [test.t1.a,test.t1.b] Key: [] Unique key: []}}\nGID:2, logic prop:{stats:{count 10, ColNDVs map[7:3 8:3]}, schema:{Column: [test.t2.a,test.t2.b] Key: [] Unique key: []}}\nGID:3, logic prop:{stats:{count 5, ColNDVs map[1:2 2:2 10:2]}, schema:{Column: [test.t1.a,test.t1.b,Column#10] Key: [] Unique key: []}}\nGID:4, logic prop:{stats:{count 4, ColNDVs map[11:4]}, schema:{Column: [Column#11] Key: [] Unique key: []}}\nGID:5, logic prop:{stats:{count 4, ColNDVs map[11:4]}, schema:{Column: [Column#11] Key: [] Unique key: []}}" + }, + { + "SQL": "select count(1) from (select t1.a as a, t1.b as b from t1 where t1.b in (select t2.b from t2 where t2.a > t1.a)) tmp group by tmp.a, tmp.b", + "Str": "GID:1, logic prop:{stats:{count 5, ColNDVs map[1:2 2:2]}, schema:{Column: [test.t1.a,test.t1.b] Key: [] Unique key: []}}\nGID:2, logic prop:{stats:{count 10, ColNDVs map[4:3 5:3]}, schema:{Column: [test.t2.a,test.t2.b] Key: [] Unique key: []}}\nGID:3, logic prop:{stats:{count 4, ColNDVs map[1:1.6 2:1.6]}, schema:{Column: [test.t1.a,test.t1.b] Key: [] Unique key: []}}\nGID:4, logic prop:{stats:{count 1.6, ColNDVs map[7:1.6]}, schema:{Column: [Column#7] Key: [] Unique key: []}}\nGID:5, logic prop:{stats:{count 1.6, ColNDVs map[7:1.6]}, schema:{Column: [Column#7] Key: [] Unique key: []}}" + }, + { + "SQL": "select count(1) from (select t1.a as a, t1.b as b from t1 where t1.b not in (select t2.b from t2 where t2.a > t1.a)) tmp group by tmp.a, tmp.b", + "Str": "GID:1, logic prop:{stats:{count 5, ColNDVs map[1:2 2:2]}, schema:{Column: [test.t1.a,test.t1.b] Key: [] Unique key: []}}\nGID:2, logic prop:{stats:{count 10, ColNDVs map[4:3 5:3]}, schema:{Column: [test.t2.a,test.t2.b] Key: [] Unique key: []}}\nGID:3, logic prop:{stats:{count 4, ColNDVs map[1:1.6 2:1.6]}, schema:{Column: [test.t1.a,test.t1.b] Key: [] Unique key: []}}\nGID:4, logic prop:{stats:{count 1.6, ColNDVs map[7:1.6]}, schema:{Column: [Column#7] Key: [] Unique key: []}}\nGID:5, logic prop:{stats:{count 1.6, ColNDVs map[7:1.6]}, schema:{Column: [Column#7] Key: [] Unique key: []}}" + }, + { + "SQL": "select * from t1 left join (select t2.a as a, t2.b as b, count(1) as cnt from t2 group by t2.a, t2.b) as tmp on t1.a = tmp.a and t1.b = tmp.b", + "Str": "GID:1, logic prop:{stats:{count 5, ColNDVs map[1:2 2:2]}, schema:{Column: [test.t1.a,test.t1.b] Key: [] Unique key: []}}\nGID:2, logic prop:{stats:{count 10, ColNDVs map[4:3 5:3]}, schema:{Column: [test.t2.a,test.t2.b] Key: [] Unique key: []}}\nGID:3, logic prop:{stats:{count 9, ColNDVs map[4:9 5:9 7:9]}, schema:{Column: [Column#7,test.t2.a,test.t2.b] Key: [[test.t2.a,test.t2.b]] Unique key: []}}\nGID:4, logic prop:{stats:{count 5, ColNDVs map[1:2 2:2 4:5 5:5 7:5]}, schema:{Column: [test.t1.a,test.t1.b,Column#7,test.t2.a,test.t2.b] Key: [] Unique key: []}}\nGID:5, logic prop:{stats:{count 5, ColNDVs map[1:2 2:2 4:5 5:5 7:5]}, schema:{Column: [test.t1.a,test.t1.b,test.t2.a,test.t2.b,Column#7] Key: [] Unique key: []}}" + }, + { + "SQL": "select count(1) from (select t1.a as a, t1.b as b from t1 limit 3) tmp group by tmp.a, tmp.b", + "Str": "GID:1, logic prop:{stats:{count 5, ColNDVs map[1:2 2:2]}, schema:{Column: [test.t1.a,test.t1.b] Key: [] Unique key: []}}\nGID:2, logic prop:{stats:{count 3, ColNDVs map[1:2 2:2]}, schema:{Column: [test.t1.a,test.t1.b] Key: [] Unique key: []}}\nGID:3, logic prop:{stats:{count 2, ColNDVs map[4:2]}, schema:{Column: [Column#4] Key: [] Unique key: []}}\nGID:4, logic prop:{stats:{count 2, ColNDVs map[4:2]}, schema:{Column: [Column#4] Key: [] Unique key: []}}" + }, + { + "SQL": "select count(tmp.a_sum) from (select t1.a as a, t1.b as b, sum(a) over() as a_sum from t1) tmp group by tmp.a, tmp.b", + "Str": "GID:1, logic prop:{stats:{count 5, ColNDVs map[1:2 2:2]}, schema:{Column: [test.t1.a,test.t1.b] Key: [] Unique key: []}}\nGID:2, logic prop:{stats:{count 5, ColNDVs map[1:2 2:2 5:5]}, schema:{Column: [test.t1.a,test.t1.b,Column#5] Key: [] Unique key: []}}\nGID:3, logic prop:{stats:{count 4, ColNDVs map[6:4]}, schema:{Column: [Column#6] Key: [] Unique key: []}}\nGID:4, logic prop:{stats:{count 4, ColNDVs map[6:4]}, schema:{Column: [Column#6] Key: [] Unique key: []}}" + } + ] + } +] diff --git a/pkg/planner/core/casetest/stats_test.go b/pkg/planner/core/casetest/stats_test.go index 93c4df86a3b80..f76b29464f303 100644 --- a/pkg/planner/core/casetest/stats_test.go +++ b/pkg/planner/core/casetest/stats_test.go @@ -16,6 +16,7 @@ package casetest import ( "context" + "fmt" "testing" diff --git a/pkg/planner/property/logical_property.go b/pkg/planner/property/logical_property.go index e1e6156e156e0..64cc48485be0d 100644 --- a/pkg/planner/property/logical_property.go +++ b/pkg/planner/property/logical_property.go @@ -25,4 +25,9 @@ type LogicalProperty struct { MaxOneRow bool } +// NewLogicalProp returns a new empty LogicalProperty. +func NewLogicalProp() *LogicalProperty { + return &LogicalProperty{} +} + // todo: ScalarProperty: usedColumns in current scalar expr, null reject, cor-related, subq contained and so on diff --git a/pkg/testkit/testdata/testdata.go b/pkg/testkit/testdata/testdata.go index 4453e257b930b..7b45fb1d9d470 100644 --- a/pkg/testkit/testdata/testdata.go +++ b/pkg/testkit/testdata/testdata.go @@ -39,7 +39,7 @@ import ( var record bool func init() { - flag.BoolVar(&record, "record", false, "to generate test result") + flag.BoolVar(&record, "record", true, "to generate test result") } type testCases struct { From 5e1234e7483cbc6b3799bdff3e6ce26d70993d4a Mon Sep 17 00:00:00 2001 From: arenatlx <314806019@qq.com> Date: Wed, 4 Dec 2024 16:19:53 +0800 Subject: [PATCH 02/25] . Signed-off-by: arenatlx <314806019@qq.com> --- pkg/planner/cascades/rule/binder_test.go | 16 ++++++++-------- pkg/planner/core/casetest/stats_test.go | 1 - pkg/testkit/testdata/testdata.go | 2 +- 3 files changed, 9 insertions(+), 10 deletions(-) diff --git a/pkg/planner/cascades/rule/binder_test.go b/pkg/planner/cascades/rule/binder_test.go index d0a858d7cf331..50f5686fde5c1 100644 --- a/pkg/planner/cascades/rule/binder_test.go +++ b/pkg/planner/cascades/rule/binder_test.go @@ -34,7 +34,7 @@ func TestBinderSuccess(t *testing.T) { join := logicalop.LogicalJoin{}.Init(ctx, 0) join.SetChildren(t1, t2) - mm := memo.NewMemo(ctx) + mm := memo.NewMemo() mm.Init(join) require.Equal(t, 3, mm.GetGroups().Len()) require.Equal(t, 3, len(mm.GetGroupID2Group())) @@ -68,7 +68,7 @@ func TestBinderFail(t *testing.T) { join := logicalop.LogicalJoin{}.Init(ctx, 0) join.SetChildren(t1, t2) - mm := memo.NewMemo(ctx) + mm := memo.NewMemo() mm.Init(join) require.Equal(t, 3, mm.GetGroups().Len()) require.Equal(t, 3, len(mm.GetGroupID2Group())) @@ -127,7 +127,7 @@ func TestBinderTopNode(t *testing.T) { join := logicalop.LogicalJoin{}.Init(ctx, 0) join.SetChildren(t1, t2) - mm := memo.NewMemo(ctx) + mm := memo.NewMemo() mm.Init(join) require.Equal(t, 3, mm.GetGroups().Len()) require.Equal(t, 3, len(mm.GetGroupID2Group())) @@ -144,7 +144,7 @@ func TestBinderOneNode(t *testing.T) { ctx := mock.NewContext() join := logicalop.LogicalJoin{}.Init(ctx, 0) - mm := memo.NewMemo(ctx) + mm := memo.NewMemo() mm.Init(join) require.Equal(t, 1, mm.GetGroups().Len()) require.Equal(t, 1, len(mm.GetGroupID2Group())) @@ -171,7 +171,7 @@ func TestBinderSubTreeMatch(t *testing.T) { join3 := logicalop.LogicalJoin{}.Init(ctx, 0) join3.SetChildren(join1, join2) - mm := memo.NewMemo(ctx) + mm := memo.NewMemo() mm.Init(join3) require.Equal(t, 7, mm.GetGroups().Len()) require.Equal(t, 7, len(mm.GetGroupID2Group())) @@ -213,7 +213,7 @@ func TestBinderMultiNext(t *testing.T) { t3 := logicalop.DataSource{TableAsName: &asT3}.Init(ctx, 0) t4 := logicalop.DataSource{TableAsName: &asT4}.Init(ctx, 0) - mm := memo.NewMemo(ctx) + mm := memo.NewMemo() gE := mm.Init(join1) // which means t1 and t3 are equivalent class. @@ -310,7 +310,7 @@ func TestBinderAny(t *testing.T) { t3 := logicalop.DataSource{TableAsName: &asT3}.Init(ctx, 0) t4 := logicalop.DataSource{TableAsName: &asT4}.Init(ctx, 0) - mm := memo.NewMemo(ctx) + mm := memo.NewMemo() gE := mm.Init(join1) // which means t1 and t3 are equivalent class. @@ -393,7 +393,7 @@ func TestBinderMultiAny(t *testing.T) { t3 := logicalop.DataSource{TableAsName: &asT3}.Init(ctx, 0) t4 := logicalop.DataSource{TableAsName: &asT4}.Init(ctx, 0) - mm := memo.NewMemo(ctx) + mm := memo.NewMemo() gE := mm.Init(join1) // which means t1 and t3 are equivalent class. diff --git a/pkg/planner/core/casetest/stats_test.go b/pkg/planner/core/casetest/stats_test.go index f76b29464f303..93c4df86a3b80 100644 --- a/pkg/planner/core/casetest/stats_test.go +++ b/pkg/planner/core/casetest/stats_test.go @@ -16,7 +16,6 @@ package casetest import ( "context" - "fmt" "testing" diff --git a/pkg/testkit/testdata/testdata.go b/pkg/testkit/testdata/testdata.go index 7b45fb1d9d470..4453e257b930b 100644 --- a/pkg/testkit/testdata/testdata.go +++ b/pkg/testkit/testdata/testdata.go @@ -39,7 +39,7 @@ import ( var record bool func init() { - flag.BoolVar(&record, "record", true, "to generate test result") + flag.BoolVar(&record, "record", false, "to generate test result") } type testCases struct { From a64accfc62c1391bf3461ec010b0b4107b6ff297 Mon Sep 17 00:00:00 2001 From: arenatlx <314806019@qq.com> Date: Fri, 6 Dec 2024 12:20:08 +0800 Subject: [PATCH 03/25] . Signed-off-by: arenatlx <314806019@qq.com> --- .../core/casetest/cascades/memo_test.go | 16 +- .../cascades/testdata/cascades_suite_out.json | 192 +++++++++++++++--- 2 files changed, 173 insertions(+), 35 deletions(-) diff --git a/pkg/planner/core/casetest/cascades/memo_test.go b/pkg/planner/core/casetest/cascades/memo_test.go index 6f926d4841d0c..1d56007502fb2 100644 --- a/pkg/planner/core/casetest/cascades/memo_test.go +++ b/pkg/planner/core/casetest/cascades/memo_test.go @@ -17,7 +17,6 @@ package cascades import ( "bytes" "context" - "strings" "testing" "github.com/pingcap/tidb/pkg/parser" @@ -49,7 +48,7 @@ func TestDeriveStats(t *testing.T) { var input []string var output []struct { SQL string - Str string + Str []string } statsSuiteData := GetCascadesSuiteData() statsSuiteData.LoadTestCases(t, &input, &output) @@ -74,14 +73,11 @@ func TestDeriveStats(t *testing.T) { mm := memo.NewMemo() mm.Init(lp) // check the stats state in memo group. - str := strings.Builder{} b := &bytes.Buffer{} + sb := util.NewStrBuffer(b) + var strs []string mm.ForEachGroup(func(g *memo.Group) { - if str.Len() != 0 { - str.WriteString("\n") - } b.Reset() - sb := util.NewStrBuffer(b) // record group g.String(sb) sb.WriteString(", ") @@ -105,12 +101,12 @@ func TestDeriveStats(t *testing.T) { sb.WriteString("}") } sb.Flush() - str.WriteString(b.String()) + strs = append(strs, b.String()) }) testdata.OnRecord(func() { output[i].SQL = tt - output[i].Str = str.String() + output[i].Str = strs }) - require.Equal(t, output[i].Str, str.String(), "case i "+tt) + require.Equal(t, output[i].Str, strs, "case i "+tt) } } diff --git a/pkg/planner/core/casetest/cascades/testdata/cascades_suite_out.json b/pkg/planner/core/casetest/cascades/testdata/cascades_suite_out.json index e93eca7ac022d..d8315b7f5f081 100644 --- a/pkg/planner/core/casetest/cascades/testdata/cascades_suite_out.json +++ b/pkg/planner/core/casetest/cascades/testdata/cascades_suite_out.json @@ -4,103 +4,245 @@ "Cases": [ { "SQL": "select count(1) from t1 group by a, b", - "Str": "GID:1, logic prop:{stats:{count 5, ColNDVs map[1:2 2:2]}, schema:{Column: [test.t1.a,test.t1.b] Key: [] Unique key: []}}\nGID:2, logic prop:{stats:{count 4, ColNDVs map[4:4]}, schema:{Column: [Column#4] Key: [] Unique key: []}}\nGID:3, logic prop:{stats:{count 4, ColNDVs map[4:4]}, schema:{Column: [Column#4] Key: [] Unique key: []}}" + "Str": [ + "GID:1, logic prop:{stats:{count 5, ColNDVs map[1:2 2:2]}, schema:{Column: [test.t1.a,test.t1.b] Key: [] Unique key: []}}", + "GID:2, logic prop:{stats:{count 4, ColNDVs map[4:4]}, schema:{Column: [Column#4] Key: [] Unique key: []}}", + "GID:3, logic prop:{stats:{count 4, ColNDVs map[4:4]}, schema:{Column: [Column#4] Key: [] Unique key: []}}" + ] }, { "SQL": "select * from t1, t2 where t1.a = t2.a and t1.b = t2.b", - "Str": "GID:1, logic prop:{stats:{count 5, ColNDVs map[1:2 2:2]}, schema:{Column: [test.t1.a,test.t1.b] Key: [] Unique key: []}}\nGID:2, logic prop:{stats:{count 10, ColNDVs map[4:3 5:3]}, schema:{Column: [test.t2.a,test.t2.b] Key: [] Unique key: []}}\nGID:3, logic prop:{stats:{count 5.555555555555555, ColNDVs map[1:2 2:2 4:3 5:3]}, schema:{Column: [test.t1.a,test.t1.b,test.t2.a,test.t2.b] Key: [] Unique key: []}}\nGID:4, logic prop:{stats:{count 5.555555555555555, ColNDVs map[1:2 2:2 4:3 5:3]}, schema:{Column: [test.t1.a,test.t1.b,test.t2.a,test.t2.b] Key: [] Unique key: []}}" + "Str": [ + "GID:1, logic prop:{stats:{count 5, ColNDVs map[1:2 2:2]}, schema:{Column: [test.t1.a,test.t1.b] Key: [] Unique key: []}}", + "GID:2, logic prop:{stats:{count 10, ColNDVs map[4:3 5:3]}, schema:{Column: [test.t2.a,test.t2.b] Key: [] Unique key: []}}", + "GID:3, logic prop:{stats:{count 5.555555555555555, ColNDVs map[1:2 2:2 4:3 5:3]}, schema:{Column: [test.t1.a,test.t1.b,test.t2.a,test.t2.b] Key: [] Unique key: []}}", + "GID:4, logic prop:{stats:{count 5.555555555555555, ColNDVs map[1:2 2:2 4:3 5:3]}, schema:{Column: [test.t1.a,test.t1.b,test.t2.a,test.t2.b] Key: [] Unique key: []}}" + ] }, { "SQL": "select count(1) from t1 where a > 0 group by a, b", - "Str": "GID:1, logic prop:{stats:{count 5, ColNDVs map[1:2 2:2]}, schema:{Column: [test.t1.a,test.t1.b] Key: [] Unique key: []}}\nGID:2, logic prop:{stats:{count 4, ColNDVs map[4:4]}, schema:{Column: [Column#4] Key: [] Unique key: []}}\nGID:3, logic prop:{stats:{count 4, ColNDVs map[4:4]}, schema:{Column: [Column#4] Key: [] Unique key: []}}" + "Str": [ + "GID:1, logic prop:{stats:{count 5, ColNDVs map[1:2 2:2]}, schema:{Column: [test.t1.a,test.t1.b] Key: [] Unique key: []}}", + "GID:2, logic prop:{stats:{count 4, ColNDVs map[4:4]}, schema:{Column: [Column#4] Key: [] Unique key: []}}", + "GID:3, logic prop:{stats:{count 4, ColNDVs map[4:4]}, schema:{Column: [Column#4] Key: [] Unique key: []}}" + ] }, { "SQL": "select count(1) from t1 where b > 0 group by a, b", - "Str": "GID:1, logic prop:{stats:{count 5, ColNDVs map[1:2 2:2]}, schema:{Column: [test.t1.a,test.t1.b] Key: [] Unique key: []}}\nGID:2, logic prop:{stats:{count 4, ColNDVs map[4:4]}, schema:{Column: [Column#4] Key: [] Unique key: []}}\nGID:3, logic prop:{stats:{count 4, ColNDVs map[4:4]}, schema:{Column: [Column#4] Key: [] Unique key: []}}" + "Str": [ + "GID:1, logic prop:{stats:{count 5, ColNDVs map[1:2 2:2]}, schema:{Column: [test.t1.a,test.t1.b] Key: [] Unique key: []}}", + "GID:2, logic prop:{stats:{count 4, ColNDVs map[4:4]}, schema:{Column: [Column#4] Key: [] Unique key: []}}", + "GID:3, logic prop:{stats:{count 4, ColNDVs map[4:4]}, schema:{Column: [Column#4] Key: [] Unique key: []}}" + ] }, { "SQL": "select count(1) from t1 where cos(a) > 0 group by a, b", - "Str": "GID:1, logic prop:{stats:{count 4, ColNDVs map[1:1.6 2:1.6]}, schema:{Column: [test.t1.a,test.t1.b] Key: [] Unique key: []}}\nGID:2, logic prop:{stats:{count 3.2, ColNDVs map[4:3.2]}, schema:{Column: [Column#4] Key: [] Unique key: []}}\nGID:3, logic prop:{stats:{count 3.2, ColNDVs map[4:3.2]}, schema:{Column: [Column#4] Key: [] Unique key: []}}" + "Str": [ + "GID:1, logic prop:{stats:{count 4, ColNDVs map[1:1.6 2:1.6]}, schema:{Column: [test.t1.a,test.t1.b] Key: [] Unique key: []}}", + "GID:2, logic prop:{stats:{count 3.2, ColNDVs map[4:3.2]}, schema:{Column: [Column#4] Key: [] Unique key: []}}", + "GID:3, logic prop:{stats:{count 3.2, ColNDVs map[4:3.2]}, schema:{Column: [Column#4] Key: [] Unique key: []}}" + ] }, { "SQL": "select count(c3) from (select a as c1, b as c2, a+1 as c3 from t1) as tmp group by c2, c1", - "Str": "GID:1, logic prop:{stats:{count 5, ColNDVs map[1:2 2:2]}, schema:{Column: [test.t1.a,test.t1.b] Key: [] Unique key: []}}\nGID:2, logic prop:{stats:{count 4, ColNDVs map[5:4]}, schema:{Column: [Column#5] Key: [] Unique key: []}}\nGID:3, logic prop:{stats:{count 4, ColNDVs map[5:4]}, schema:{Column: [Column#5] Key: [] Unique key: []}}" + "Str": [ + "GID:1, logic prop:{stats:{count 5, ColNDVs map[1:2 2:2]}, schema:{Column: [test.t1.a,test.t1.b] Key: [] Unique key: []}}", + "GID:2, logic prop:{stats:{count 4, ColNDVs map[5:4]}, schema:{Column: [Column#5] Key: [] Unique key: []}}", + "GID:3, logic prop:{stats:{count 4, ColNDVs map[5:4]}, schema:{Column: [Column#5] Key: [] Unique key: []}}" + ] }, { "SQL": "select count(c3) from (select a+b as c1, b as c2, a+1 as c3 from t1) as tmp group by c2, c1", - "Str": "GID:1, logic prop:{stats:{count 5, ColNDVs map[1:2 2:2]}, schema:{Column: [test.t1.a,test.t1.b] Key: [] Unique key: []}}\nGID:2, logic prop:{stats:{count 2, ColNDVs map[6:2]}, schema:{Column: [Column#6] Key: [] Unique key: []}}\nGID:3, logic prop:{stats:{count 2, ColNDVs map[6:2]}, schema:{Column: [Column#6] Key: [] Unique key: []}}" + "Str": [ + "GID:1, logic prop:{stats:{count 5, ColNDVs map[1:2 2:2]}, schema:{Column: [test.t1.a,test.t1.b] Key: [] Unique key: []}}", + "GID:2, logic prop:{stats:{count 2, ColNDVs map[6:2]}, schema:{Column: [Column#6] Key: [] Unique key: []}}", + "GID:3, logic prop:{stats:{count 2, ColNDVs map[6:2]}, schema:{Column: [Column#6] Key: [] Unique key: []}}" + ] }, { "SQL": "select count(tmp.cmp) from (select t1.a as a, t1.b as b, (t1.b > (select t2.b from t2 where t2.a = t1.a)) as cmp from t1) tmp group by tmp.a, tmp.b", - "Str": "GID:1, logic prop:{stats:{count 5, ColNDVs map[1:2 2:2]}, schema:{Column: [test.t1.a,test.t1.b] Key: [] Unique key: []}}\nGID:2, logic prop:{stats:{count 3.333333333333333, ColNDVs map[7:1 8:1]}, schema:{Column: [test.t2.a,test.t2.b] Key: [] Unique key: []}}\nGID:3, logic prop:{stats:{count 1, ColNDVs map[7:1 8:1]}, schema:{Column: [test.t2.a,test.t2.b] Key: [] Unique key: []}}\nGID:4, logic prop:{stats:{count 5, ColNDVs map[1:2 2:2 7:5 8:5]}, schema:{Column: [test.t1.a,test.t1.b,test.t2.a,test.t2.b] Key: [] Unique key: []}}\nGID:5, logic prop:{stats:{count 4, ColNDVs map[11:4]}, schema:{Column: [Column#11] Key: [] Unique key: []}}\nGID:6, logic prop:{stats:{count 4, ColNDVs map[11:4]}, schema:{Column: [Column#11] Key: [] Unique key: []}}" + "Str": [ + "GID:1, logic prop:{stats:{count 5, ColNDVs map[1:2 2:2]}, schema:{Column: [test.t1.a,test.t1.b] Key: [] Unique key: []}}", + "GID:2, logic prop:{stats:{count 3.333333333333333, ColNDVs map[7:1 8:1]}, schema:{Column: [test.t2.a,test.t2.b] Key: [] Unique key: []}}", + "GID:3, logic prop:{stats:{count 1, ColNDVs map[7:1 8:1]}, schema:{Column: [test.t2.a,test.t2.b] Key: [] Unique key: []}}", + "GID:4, logic prop:{stats:{count 5, ColNDVs map[1:2 2:2 7:5 8:5]}, schema:{Column: [test.t1.a,test.t1.b,test.t2.a,test.t2.b] Key: [] Unique key: []}}", + "GID:5, logic prop:{stats:{count 4, ColNDVs map[11:4]}, schema:{Column: [Column#11] Key: [] Unique key: []}}", + "GID:6, logic prop:{stats:{count 4, ColNDVs map[11:4]}, schema:{Column: [Column#11] Key: [] Unique key: []}}" + ] }, { "SQL": "select count(1) from (select t1.a as a, t1.b as b from t1 where t1.b > (select t2.b from t2 where t2.a = t1.a)) tmp group by tmp.a, tmp.b", - "Str": "GID:1, logic prop:{stats:{count 5, ColNDVs map[1:2 2:2]}, schema:{Column: [test.t1.a,test.t1.b] Key: [] Unique key: []}}\nGID:2, logic prop:{stats:{count 3.333333333333333, ColNDVs map[4:1 5:1]}, schema:{Column: [test.t2.a,test.t2.b] Key: [] Unique key: []}}\nGID:3, logic prop:{stats:{count 1, ColNDVs map[4:1 5:1]}, schema:{Column: [test.t2.a,test.t2.b] Key: [] Unique key: []}}\nGID:4, logic prop:{stats:{count 0.8, ColNDVs map[4:0.8 5:0.8]}, schema:{Column: [test.t2.a,test.t2.b] Key: [] Unique key: []}}\nGID:5, logic prop:{stats:{count 5, ColNDVs map[1:2 2:2 4:5 5:5]}, schema:{Column: [test.t1.a,test.t1.b,test.t2.a,test.t2.b] Key: [] Unique key: []}}\nGID:6, logic prop:{stats:{count 2, ColNDVs map[7:2]}, schema:{Column: [Column#7] Key: [] Unique key: []}}\nGID:7, logic prop:{stats:{count 2, ColNDVs map[7:2]}, schema:{Column: [Column#7] Key: [] Unique key: []}}" + "Str": [ + "GID:1, logic prop:{stats:{count 5, ColNDVs map[1:2 2:2]}, schema:{Column: [test.t1.a,test.t1.b] Key: [] Unique key: []}}", + "GID:2, logic prop:{stats:{count 3.333333333333333, ColNDVs map[4:1 5:1]}, schema:{Column: [test.t2.a,test.t2.b] Key: [] Unique key: []}}", + "GID:3, logic prop:{stats:{count 1, ColNDVs map[4:1 5:1]}, schema:{Column: [test.t2.a,test.t2.b] Key: [] Unique key: []}}", + "GID:4, logic prop:{stats:{count 0.8, ColNDVs map[4:0.8 5:0.8]}, schema:{Column: [test.t2.a,test.t2.b] Key: [] Unique key: []}}", + "GID:5, logic prop:{stats:{count 5, ColNDVs map[1:2 2:2 4:5 5:5]}, schema:{Column: [test.t1.a,test.t1.b,test.t2.a,test.t2.b] Key: [] Unique key: []}}", + "GID:6, logic prop:{stats:{count 2, ColNDVs map[7:2]}, schema:{Column: [Column#7] Key: [] Unique key: []}}", + "GID:7, logic prop:{stats:{count 2, ColNDVs map[7:2]}, schema:{Column: [Column#7] Key: [] Unique key: []}}" + ] }, { "SQL": "select count(tmp.cmp) from (select t1.a as a, t1.b as b, (t1.b in (select t2.b from t2 where t2.a = t1.a limit 3)) as cmp from t1) tmp group by tmp.a, tmp.b", - "Str": "GID:1, logic prop:{stats:{count 5, ColNDVs map[1:2 2:2]}, schema:{Column: [test.t1.a,test.t1.b] Key: [] Unique key: []}}\nGID:2, logic prop:{stats:{count 3.333333333333333, ColNDVs map[7:1 8:1]}, schema:{Column: [test.t2.a,test.t2.b] Key: [] Unique key: []}}\nGID:3, logic prop:{stats:{count 3, ColNDVs map[7:1 8:1]}, schema:{Column: [test.t2.a,test.t2.b] Key: [] Unique key: []}}\nGID:4, logic prop:{stats:{count 5, ColNDVs map[1:2 2:2 10:2]}, schema:{Column: [test.t1.a,test.t1.b,Column#10] Key: [] Unique key: []}}\nGID:5, logic prop:{stats:{count 4, ColNDVs map[11:4]}, schema:{Column: [Column#11] Key: [] Unique key: []}}\nGID:6, logic prop:{stats:{count 4, ColNDVs map[11:4]}, schema:{Column: [Column#11] Key: [] Unique key: []}}" + "Str": [ + "GID:1, logic prop:{stats:{count 5, ColNDVs map[1:2 2:2]}, schema:{Column: [test.t1.a,test.t1.b] Key: [] Unique key: []}}", + "GID:2, logic prop:{stats:{count 3.333333333333333, ColNDVs map[7:1 8:1]}, schema:{Column: [test.t2.a,test.t2.b] Key: [] Unique key: []}}", + "GID:3, logic prop:{stats:{count 3, ColNDVs map[7:1 8:1]}, schema:{Column: [test.t2.a,test.t2.b] Key: [] Unique key: []}}", + "GID:4, logic prop:{stats:{count 5, ColNDVs map[1:2 2:2 10:2]}, schema:{Column: [test.t1.a,test.t1.b,Column#10] Key: [] Unique key: []}}", + "GID:5, logic prop:{stats:{count 4, ColNDVs map[11:4]}, schema:{Column: [Column#11] Key: [] Unique key: []}}", + "GID:6, logic prop:{stats:{count 4, ColNDVs map[11:4]}, schema:{Column: [Column#11] Key: [] Unique key: []}}" + ] }, { "SQL": "select count(tmp.cmp) from (select t1.a as a, t1.b as b, (t1.b not in (select t2.b from t2 where t2.a = t1.a limit 3)) as cmp from t1) tmp group by tmp.a, tmp.b", - "Str": "GID:1, logic prop:{stats:{count 5, ColNDVs map[1:2 2:2]}, schema:{Column: [test.t1.a,test.t1.b] Key: [] Unique key: []}}\nGID:2, logic prop:{stats:{count 3.333333333333333, ColNDVs map[7:1 8:1]}, schema:{Column: [test.t2.a,test.t2.b] Key: [] Unique key: []}}\nGID:3, logic prop:{stats:{count 3, ColNDVs map[7:1 8:1]}, schema:{Column: [test.t2.a,test.t2.b] Key: [] Unique key: []}}\nGID:4, logic prop:{stats:{count 5, ColNDVs map[1:2 2:2 10:2]}, schema:{Column: [test.t1.a,test.t1.b,Column#10] Key: [] Unique key: []}}\nGID:5, logic prop:{stats:{count 4, ColNDVs map[11:4]}, schema:{Column: [Column#11] Key: [] Unique key: []}}\nGID:6, logic prop:{stats:{count 4, ColNDVs map[11:4]}, schema:{Column: [Column#11] Key: [] Unique key: []}}" + "Str": [ + "GID:1, logic prop:{stats:{count 5, ColNDVs map[1:2 2:2]}, schema:{Column: [test.t1.a,test.t1.b] Key: [] Unique key: []}}", + "GID:2, logic prop:{stats:{count 3.333333333333333, ColNDVs map[7:1 8:1]}, schema:{Column: [test.t2.a,test.t2.b] Key: [] Unique key: []}}", + "GID:3, logic prop:{stats:{count 3, ColNDVs map[7:1 8:1]}, schema:{Column: [test.t2.a,test.t2.b] Key: [] Unique key: []}}", + "GID:4, logic prop:{stats:{count 5, ColNDVs map[1:2 2:2 10:2]}, schema:{Column: [test.t1.a,test.t1.b,Column#10] Key: [] Unique key: []}}", + "GID:5, logic prop:{stats:{count 4, ColNDVs map[11:4]}, schema:{Column: [Column#11] Key: [] Unique key: []}}", + "GID:6, logic prop:{stats:{count 4, ColNDVs map[11:4]}, schema:{Column: [Column#11] Key: [] Unique key: []}}" + ] }, { "SQL": "select count(1) from (select t1.a as a, t1.b as b from t1 where t1.b in (select t2.b from t2 where t2.a = t1.a limit 3)) tmp group by tmp.a, tmp.b", - "Str": "GID:1, logic prop:{stats:{count 5, ColNDVs map[1:2 2:2]}, schema:{Column: [test.t1.a,test.t1.b] Key: [] Unique key: []}}\nGID:2, logic prop:{stats:{count 3.333333333333333, ColNDVs map[4:1 5:1]}, schema:{Column: [test.t2.a,test.t2.b] Key: [] Unique key: []}}\nGID:3, logic prop:{stats:{count 3, ColNDVs map[4:1 5:1]}, schema:{Column: [test.t2.a,test.t2.b] Key: [] Unique key: []}}\nGID:4, logic prop:{stats:{count 5, ColNDVs map[1:2 2:2]}, schema:{Column: [test.t1.a,test.t1.b] Key: [] Unique key: []}}\nGID:5, logic prop:{stats:{count 2, ColNDVs map[7:2]}, schema:{Column: [Column#7] Key: [] Unique key: []}}\nGID:6, logic prop:{stats:{count 2, ColNDVs map[7:2]}, schema:{Column: [Column#7] Key: [] Unique key: []}}" + "Str": [ + "GID:1, logic prop:{stats:{count 5, ColNDVs map[1:2 2:2]}, schema:{Column: [test.t1.a,test.t1.b] Key: [] Unique key: []}}", + "GID:2, logic prop:{stats:{count 3.333333333333333, ColNDVs map[4:1 5:1]}, schema:{Column: [test.t2.a,test.t2.b] Key: [] Unique key: []}}", + "GID:3, logic prop:{stats:{count 3, ColNDVs map[4:1 5:1]}, schema:{Column: [test.t2.a,test.t2.b] Key: [] Unique key: []}}", + "GID:4, logic prop:{stats:{count 5, ColNDVs map[1:2 2:2]}, schema:{Column: [test.t1.a,test.t1.b] Key: [] Unique key: []}}", + "GID:5, logic prop:{stats:{count 2, ColNDVs map[7:2]}, schema:{Column: [Column#7] Key: [] Unique key: []}}", + "GID:6, logic prop:{stats:{count 2, ColNDVs map[7:2]}, schema:{Column: [Column#7] Key: [] Unique key: []}}" + ] }, { "SQL": "select count(1) from (select t1.a as a, t1.b as b from t1 where t1.b not in (select t2.b from t2 where t2.a = t1.a limit 3)) tmp group by tmp.a, tmp.b", - "Str": "GID:1, logic prop:{stats:{count 5, ColNDVs map[1:2 2:2]}, schema:{Column: [test.t1.a,test.t1.b] Key: [] Unique key: []}}\nGID:2, logic prop:{stats:{count 3.333333333333333, ColNDVs map[4:1 5:1]}, schema:{Column: [test.t2.a,test.t2.b] Key: [] Unique key: []}}\nGID:3, logic prop:{stats:{count 3, ColNDVs map[4:1 5:1]}, schema:{Column: [test.t2.a,test.t2.b] Key: [] Unique key: []}}\nGID:4, logic prop:{stats:{count 5, ColNDVs map[1:2 2:2]}, schema:{Column: [test.t1.a,test.t1.b] Key: [] Unique key: []}}\nGID:5, logic prop:{stats:{count 2, ColNDVs map[7:2]}, schema:{Column: [Column#7] Key: [] Unique key: []}}\nGID:6, logic prop:{stats:{count 2, ColNDVs map[7:2]}, schema:{Column: [Column#7] Key: [] Unique key: []}}" + "Str": [ + "GID:1, logic prop:{stats:{count 5, ColNDVs map[1:2 2:2]}, schema:{Column: [test.t1.a,test.t1.b] Key: [] Unique key: []}}", + "GID:2, logic prop:{stats:{count 3.333333333333333, ColNDVs map[4:1 5:1]}, schema:{Column: [test.t2.a,test.t2.b] Key: [] Unique key: []}}", + "GID:3, logic prop:{stats:{count 3, ColNDVs map[4:1 5:1]}, schema:{Column: [test.t2.a,test.t2.b] Key: [] Unique key: []}}", + "GID:4, logic prop:{stats:{count 5, ColNDVs map[1:2 2:2]}, schema:{Column: [test.t1.a,test.t1.b] Key: [] Unique key: []}}", + "GID:5, logic prop:{stats:{count 2, ColNDVs map[7:2]}, schema:{Column: [Column#7] Key: [] Unique key: []}}", + "GID:6, logic prop:{stats:{count 2, ColNDVs map[7:2]}, schema:{Column: [Column#7] Key: [] Unique key: []}}" + ] }, { "SQL": "select count(1) from t1, t2 where t1.a = t2.a group by t1.a, t1.b", - "Str": "GID:1, logic prop:{stats:{count 5, ColNDVs map[1:2 2:2]}, schema:{Column: [test.t1.a,test.t1.b] Key: [] Unique key: []}}\nGID:2, logic prop:{stats:{count 10, ColNDVs map[4:3]}, schema:{Column: [test.t2.a] Key: [] Unique key: []}}\nGID:3, logic prop:{stats:{count 16.666666666666668, ColNDVs map[1:2 2:2 4:3]}, schema:{Column: [test.t1.a,test.t1.b,test.t2.a] Key: [] Unique key: []}}\nGID:4, logic prop:{stats:{count 2, ColNDVs map[7:2]}, schema:{Column: [Column#7] Key: [] Unique key: []}}\nGID:5, logic prop:{stats:{count 2, ColNDVs map[7:2]}, schema:{Column: [Column#7] Key: [] Unique key: []}}" + "Str": [ + "GID:1, logic prop:{stats:{count 5, ColNDVs map[1:2 2:2]}, schema:{Column: [test.t1.a,test.t1.b] Key: [] Unique key: []}}", + "GID:2, logic prop:{stats:{count 10, ColNDVs map[4:3]}, schema:{Column: [test.t2.a] Key: [] Unique key: []}}", + "GID:3, logic prop:{stats:{count 16.666666666666668, ColNDVs map[1:2 2:2 4:3]}, schema:{Column: [test.t1.a,test.t1.b,test.t2.a] Key: [] Unique key: []}}", + "GID:4, logic prop:{stats:{count 2, ColNDVs map[7:2]}, schema:{Column: [Column#7] Key: [] Unique key: []}}", + "GID:5, logic prop:{stats:{count 2, ColNDVs map[7:2]}, schema:{Column: [Column#7] Key: [] Unique key: []}}" + ] }, { "SQL": "select count(1) from t1 left join t2 on t1.a = t2.a group by t1.a, t1.b", - "Str": "GID:1, logic prop:{stats:{count 5, ColNDVs map[1:2 2:2]}, schema:{Column: [test.t1.a,test.t1.b] Key: [] Unique key: []}}\nGID:2, logic prop:{stats:{count 10, ColNDVs map[4:3]}, schema:{Column: [test.t2.a] Key: [] Unique key: []}}\nGID:3, logic prop:{stats:{count 16.666666666666668, ColNDVs map[1:2 2:2 4:3]}, schema:{Column: [test.t1.a,test.t1.b,test.t2.a] Key: [] Unique key: []}}\nGID:4, logic prop:{stats:{count 4, ColNDVs map[7:4]}, schema:{Column: [Column#7] Key: [] Unique key: []}}\nGID:5, logic prop:{stats:{count 4, ColNDVs map[7:4]}, schema:{Column: [Column#7] Key: [] Unique key: []}}" + "Str": [ + "GID:1, logic prop:{stats:{count 5, ColNDVs map[1:2 2:2]}, schema:{Column: [test.t1.a,test.t1.b] Key: [] Unique key: []}}", + "GID:2, logic prop:{stats:{count 10, ColNDVs map[4:3]}, schema:{Column: [test.t2.a] Key: [] Unique key: []}}", + "GID:3, logic prop:{stats:{count 16.666666666666668, ColNDVs map[1:2 2:2 4:3]}, schema:{Column: [test.t1.a,test.t1.b,test.t2.a] Key: [] Unique key: []}}", + "GID:4, logic prop:{stats:{count 4, ColNDVs map[7:4]}, schema:{Column: [Column#7] Key: [] Unique key: []}}", + "GID:5, logic prop:{stats:{count 4, ColNDVs map[7:4]}, schema:{Column: [Column#7] Key: [] Unique key: []}}" + ] }, { "SQL": "select count(1) from t1 left join t2 on t1.a = t2.a group by t2.a, t2.b", - "Str": "GID:1, logic prop:{stats:{count 5, ColNDVs map[1:2]}, schema:{Column: [test.t1.a] Key: [] Unique key: []}}\nGID:2, logic prop:{stats:{count 10, ColNDVs map[4:3 5:3]}, schema:{Column: [test.t2.a,test.t2.b] Key: [] Unique key: []}}\nGID:3, logic prop:{stats:{count 16.666666666666668, ColNDVs map[1:2 4:3 5:3]}, schema:{Column: [test.t1.a,test.t2.a,test.t2.b] Key: [] Unique key: []}}\nGID:4, logic prop:{stats:{count 3, ColNDVs map[7:3]}, schema:{Column: [Column#7] Key: [] Unique key: []}}\nGID:5, logic prop:{stats:{count 3, ColNDVs map[7:3]}, schema:{Column: [Column#7] Key: [] Unique key: []}}" + "Str": [ + "GID:1, logic prop:{stats:{count 5, ColNDVs map[1:2]}, schema:{Column: [test.t1.a] Key: [] Unique key: []}}", + "GID:2, logic prop:{stats:{count 10, ColNDVs map[4:3 5:3]}, schema:{Column: [test.t2.a,test.t2.b] Key: [] Unique key: []}}", + "GID:3, logic prop:{stats:{count 16.666666666666668, ColNDVs map[1:2 4:3 5:3]}, schema:{Column: [test.t1.a,test.t2.a,test.t2.b] Key: [] Unique key: []}}", + "GID:4, logic prop:{stats:{count 3, ColNDVs map[7:3]}, schema:{Column: [Column#7] Key: [] Unique key: []}}", + "GID:5, logic prop:{stats:{count 3, ColNDVs map[7:3]}, schema:{Column: [Column#7] Key: [] Unique key: []}}" + ] }, { "SQL": "select count(1) from t1 right join t2 on t1.a = t2.a group by t1.a, t1.b", - "Str": "GID:1, logic prop:{stats:{count 5, ColNDVs map[1:2 2:2]}, schema:{Column: [test.t1.a,test.t1.b] Key: [] Unique key: []}}\nGID:2, logic prop:{stats:{count 10, ColNDVs map[4:3]}, schema:{Column: [test.t2.a] Key: [] Unique key: []}}\nGID:3, logic prop:{stats:{count 16.666666666666668, ColNDVs map[1:2 2:2 4:3]}, schema:{Column: [test.t1.a,test.t1.b,test.t2.a] Key: [] Unique key: []}}\nGID:4, logic prop:{stats:{count 2, ColNDVs map[7:2]}, schema:{Column: [Column#7] Key: [] Unique key: []}}\nGID:5, logic prop:{stats:{count 2, ColNDVs map[7:2]}, schema:{Column: [Column#7] Key: [] Unique key: []}}" + "Str": [ + "GID:1, logic prop:{stats:{count 5, ColNDVs map[1:2 2:2]}, schema:{Column: [test.t1.a,test.t1.b] Key: [] Unique key: []}}", + "GID:2, logic prop:{stats:{count 10, ColNDVs map[4:3]}, schema:{Column: [test.t2.a] Key: [] Unique key: []}}", + "GID:3, logic prop:{stats:{count 16.666666666666668, ColNDVs map[1:2 2:2 4:3]}, schema:{Column: [test.t1.a,test.t1.b,test.t2.a] Key: [] Unique key: []}}", + "GID:4, logic prop:{stats:{count 2, ColNDVs map[7:2]}, schema:{Column: [Column#7] Key: [] Unique key: []}}", + "GID:5, logic prop:{stats:{count 2, ColNDVs map[7:2]}, schema:{Column: [Column#7] Key: [] Unique key: []}}" + ] }, { "SQL": "select count(1) from t1 right join t2 on t1.a = t2.a group by t2.a, t2.b", - "Str": "GID:1, logic prop:{stats:{count 5, ColNDVs map[1:2]}, schema:{Column: [test.t1.a] Key: [] Unique key: []}}\nGID:2, logic prop:{stats:{count 10, ColNDVs map[4:3 5:3]}, schema:{Column: [test.t2.a,test.t2.b] Key: [] Unique key: []}}\nGID:3, logic prop:{stats:{count 16.666666666666668, ColNDVs map[1:2 4:3 5:3]}, schema:{Column: [test.t1.a,test.t2.a,test.t2.b] Key: [] Unique key: []}}\nGID:4, logic prop:{stats:{count 9, ColNDVs map[7:9]}, schema:{Column: [Column#7] Key: [] Unique key: []}}\nGID:5, logic prop:{stats:{count 9, ColNDVs map[7:9]}, schema:{Column: [Column#7] Key: [] Unique key: []}}" + "Str": [ + "GID:1, logic prop:{stats:{count 5, ColNDVs map[1:2]}, schema:{Column: [test.t1.a] Key: [] Unique key: []}}", + "GID:2, logic prop:{stats:{count 10, ColNDVs map[4:3 5:3]}, schema:{Column: [test.t2.a,test.t2.b] Key: [] Unique key: []}}", + "GID:3, logic prop:{stats:{count 16.666666666666668, ColNDVs map[1:2 4:3 5:3]}, schema:{Column: [test.t1.a,test.t2.a,test.t2.b] Key: [] Unique key: []}}", + "GID:4, logic prop:{stats:{count 9, ColNDVs map[7:9]}, schema:{Column: [Column#7] Key: [] Unique key: []}}", + "GID:5, logic prop:{stats:{count 9, ColNDVs map[7:9]}, schema:{Column: [Column#7] Key: [] Unique key: []}}" + ] }, { "SQL": "select count(tmp.cmp) from (select t1.a as a, t1.b as b, (t1.b in (select t2.b from t2 where t2.a > t1.a)) as cmp from t1) tmp group by tmp.a, tmp.b", - "Str": "GID:1, logic prop:{stats:{count 5, ColNDVs map[1:2 2:2]}, schema:{Column: [test.t1.a,test.t1.b] Key: [] Unique key: []}}\nGID:2, logic prop:{stats:{count 10, ColNDVs map[7:3 8:3]}, schema:{Column: [test.t2.a,test.t2.b] Key: [] Unique key: []}}\nGID:3, logic prop:{stats:{count 5, ColNDVs map[1:2 2:2 10:2]}, schema:{Column: [test.t1.a,test.t1.b,Column#10] Key: [] Unique key: []}}\nGID:4, logic prop:{stats:{count 4, ColNDVs map[11:4]}, schema:{Column: [Column#11] Key: [] Unique key: []}}\nGID:5, logic prop:{stats:{count 4, ColNDVs map[11:4]}, schema:{Column: [Column#11] Key: [] Unique key: []}}" + "Str": [ + "GID:1, logic prop:{stats:{count 5, ColNDVs map[1:2 2:2]}, schema:{Column: [test.t1.a,test.t1.b] Key: [] Unique key: []}}", + "GID:2, logic prop:{stats:{count 10, ColNDVs map[7:3 8:3]}, schema:{Column: [test.t2.a,test.t2.b] Key: [] Unique key: []}}", + "GID:3, logic prop:{stats:{count 5, ColNDVs map[1:2 2:2 10:2]}, schema:{Column: [test.t1.a,test.t1.b,Column#10] Key: [] Unique key: []}}", + "GID:4, logic prop:{stats:{count 4, ColNDVs map[11:4]}, schema:{Column: [Column#11] Key: [] Unique key: []}}", + "GID:5, logic prop:{stats:{count 4, ColNDVs map[11:4]}, schema:{Column: [Column#11] Key: [] Unique key: []}}" + ] }, { "SQL": "select count(tmp.cmp) from (select t1.a as a, t1.b as b, (t1.b not in (select t2.b from t2 where t2.a > t1.a)) as cmp from t1) tmp group by tmp.a, tmp.b", - "Str": "GID:1, logic prop:{stats:{count 5, ColNDVs map[1:2 2:2]}, schema:{Column: [test.t1.a,test.t1.b] Key: [] Unique key: []}}\nGID:2, logic prop:{stats:{count 10, ColNDVs map[7:3 8:3]}, schema:{Column: [test.t2.a,test.t2.b] Key: [] Unique key: []}}\nGID:3, logic prop:{stats:{count 5, ColNDVs map[1:2 2:2 10:2]}, schema:{Column: [test.t1.a,test.t1.b,Column#10] Key: [] Unique key: []}}\nGID:4, logic prop:{stats:{count 4, ColNDVs map[11:4]}, schema:{Column: [Column#11] Key: [] Unique key: []}}\nGID:5, logic prop:{stats:{count 4, ColNDVs map[11:4]}, schema:{Column: [Column#11] Key: [] Unique key: []}}" + "Str": [ + "GID:1, logic prop:{stats:{count 5, ColNDVs map[1:2 2:2]}, schema:{Column: [test.t1.a,test.t1.b] Key: [] Unique key: []}}", + "GID:2, logic prop:{stats:{count 10, ColNDVs map[7:3 8:3]}, schema:{Column: [test.t2.a,test.t2.b] Key: [] Unique key: []}}", + "GID:3, logic prop:{stats:{count 5, ColNDVs map[1:2 2:2 10:2]}, schema:{Column: [test.t1.a,test.t1.b,Column#10] Key: [] Unique key: []}}", + "GID:4, logic prop:{stats:{count 4, ColNDVs map[11:4]}, schema:{Column: [Column#11] Key: [] Unique key: []}}", + "GID:5, logic prop:{stats:{count 4, ColNDVs map[11:4]}, schema:{Column: [Column#11] Key: [] Unique key: []}}" + ] }, { "SQL": "select count(1) from (select t1.a as a, t1.b as b from t1 where t1.b in (select t2.b from t2 where t2.a > t1.a)) tmp group by tmp.a, tmp.b", - "Str": "GID:1, logic prop:{stats:{count 5, ColNDVs map[1:2 2:2]}, schema:{Column: [test.t1.a,test.t1.b] Key: [] Unique key: []}}\nGID:2, logic prop:{stats:{count 10, ColNDVs map[4:3 5:3]}, schema:{Column: [test.t2.a,test.t2.b] Key: [] Unique key: []}}\nGID:3, logic prop:{stats:{count 4, ColNDVs map[1:1.6 2:1.6]}, schema:{Column: [test.t1.a,test.t1.b] Key: [] Unique key: []}}\nGID:4, logic prop:{stats:{count 1.6, ColNDVs map[7:1.6]}, schema:{Column: [Column#7] Key: [] Unique key: []}}\nGID:5, logic prop:{stats:{count 1.6, ColNDVs map[7:1.6]}, schema:{Column: [Column#7] Key: [] Unique key: []}}" + "Str": [ + "GID:1, logic prop:{stats:{count 5, ColNDVs map[1:2 2:2]}, schema:{Column: [test.t1.a,test.t1.b] Key: [] Unique key: []}}", + "GID:2, logic prop:{stats:{count 10, ColNDVs map[4:3 5:3]}, schema:{Column: [test.t2.a,test.t2.b] Key: [] Unique key: []}}", + "GID:3, logic prop:{stats:{count 4, ColNDVs map[1:1.6 2:1.6]}, schema:{Column: [test.t1.a,test.t1.b] Key: [] Unique key: []}}", + "GID:4, logic prop:{stats:{count 1.6, ColNDVs map[7:1.6]}, schema:{Column: [Column#7] Key: [] Unique key: []}}", + "GID:5, logic prop:{stats:{count 1.6, ColNDVs map[7:1.6]}, schema:{Column: [Column#7] Key: [] Unique key: []}}" + ] }, { "SQL": "select count(1) from (select t1.a as a, t1.b as b from t1 where t1.b not in (select t2.b from t2 where t2.a > t1.a)) tmp group by tmp.a, tmp.b", - "Str": "GID:1, logic prop:{stats:{count 5, ColNDVs map[1:2 2:2]}, schema:{Column: [test.t1.a,test.t1.b] Key: [] Unique key: []}}\nGID:2, logic prop:{stats:{count 10, ColNDVs map[4:3 5:3]}, schema:{Column: [test.t2.a,test.t2.b] Key: [] Unique key: []}}\nGID:3, logic prop:{stats:{count 4, ColNDVs map[1:1.6 2:1.6]}, schema:{Column: [test.t1.a,test.t1.b] Key: [] Unique key: []}}\nGID:4, logic prop:{stats:{count 1.6, ColNDVs map[7:1.6]}, schema:{Column: [Column#7] Key: [] Unique key: []}}\nGID:5, logic prop:{stats:{count 1.6, ColNDVs map[7:1.6]}, schema:{Column: [Column#7] Key: [] Unique key: []}}" + "Str": [ + "GID:1, logic prop:{stats:{count 5, ColNDVs map[1:2 2:2]}, schema:{Column: [test.t1.a,test.t1.b] Key: [] Unique key: []}}", + "GID:2, logic prop:{stats:{count 10, ColNDVs map[4:3 5:3]}, schema:{Column: [test.t2.a,test.t2.b] Key: [] Unique key: []}}", + "GID:3, logic prop:{stats:{count 4, ColNDVs map[1:1.6 2:1.6]}, schema:{Column: [test.t1.a,test.t1.b] Key: [] Unique key: []}}", + "GID:4, logic prop:{stats:{count 1.6, ColNDVs map[7:1.6]}, schema:{Column: [Column#7] Key: [] Unique key: []}}", + "GID:5, logic prop:{stats:{count 1.6, ColNDVs map[7:1.6]}, schema:{Column: [Column#7] Key: [] Unique key: []}}" + ] }, { "SQL": "select * from t1 left join (select t2.a as a, t2.b as b, count(1) as cnt from t2 group by t2.a, t2.b) as tmp on t1.a = tmp.a and t1.b = tmp.b", - "Str": "GID:1, logic prop:{stats:{count 5, ColNDVs map[1:2 2:2]}, schema:{Column: [test.t1.a,test.t1.b] Key: [] Unique key: []}}\nGID:2, logic prop:{stats:{count 10, ColNDVs map[4:3 5:3]}, schema:{Column: [test.t2.a,test.t2.b] Key: [] Unique key: []}}\nGID:3, logic prop:{stats:{count 9, ColNDVs map[4:9 5:9 7:9]}, schema:{Column: [Column#7,test.t2.a,test.t2.b] Key: [[test.t2.a,test.t2.b]] Unique key: []}}\nGID:4, logic prop:{stats:{count 5, ColNDVs map[1:2 2:2 4:5 5:5 7:5]}, schema:{Column: [test.t1.a,test.t1.b,Column#7,test.t2.a,test.t2.b] Key: [] Unique key: []}}\nGID:5, logic prop:{stats:{count 5, ColNDVs map[1:2 2:2 4:5 5:5 7:5]}, schema:{Column: [test.t1.a,test.t1.b,test.t2.a,test.t2.b,Column#7] Key: [] Unique key: []}}" + "Str": [ + "GID:1, logic prop:{stats:{count 5, ColNDVs map[1:2 2:2]}, schema:{Column: [test.t1.a,test.t1.b] Key: [] Unique key: []}}", + "GID:2, logic prop:{stats:{count 10, ColNDVs map[4:3 5:3]}, schema:{Column: [test.t2.a,test.t2.b] Key: [] Unique key: []}}", + "GID:3, logic prop:{stats:{count 9, ColNDVs map[4:9 5:9 7:9]}, schema:{Column: [Column#7,test.t2.a,test.t2.b] Key: [[test.t2.a,test.t2.b]] Unique key: []}}", + "GID:4, logic prop:{stats:{count 5, ColNDVs map[1:2 2:2 4:5 5:5 7:5]}, schema:{Column: [test.t1.a,test.t1.b,Column#7,test.t2.a,test.t2.b] Key: [] Unique key: []}}", + "GID:5, logic prop:{stats:{count 5, ColNDVs map[1:2 2:2 4:5 5:5 7:5]}, schema:{Column: [test.t1.a,test.t1.b,test.t2.a,test.t2.b,Column#7] Key: [] Unique key: []}}" + ] }, { "SQL": "select count(1) from (select t1.a as a, t1.b as b from t1 limit 3) tmp group by tmp.a, tmp.b", - "Str": "GID:1, logic prop:{stats:{count 5, ColNDVs map[1:2 2:2]}, schema:{Column: [test.t1.a,test.t1.b] Key: [] Unique key: []}}\nGID:2, logic prop:{stats:{count 3, ColNDVs map[1:2 2:2]}, schema:{Column: [test.t1.a,test.t1.b] Key: [] Unique key: []}}\nGID:3, logic prop:{stats:{count 2, ColNDVs map[4:2]}, schema:{Column: [Column#4] Key: [] Unique key: []}}\nGID:4, logic prop:{stats:{count 2, ColNDVs map[4:2]}, schema:{Column: [Column#4] Key: [] Unique key: []}}" + "Str": [ + "GID:1, logic prop:{stats:{count 5, ColNDVs map[1:2 2:2]}, schema:{Column: [test.t1.a,test.t1.b] Key: [] Unique key: []}}", + "GID:2, logic prop:{stats:{count 3, ColNDVs map[1:2 2:2]}, schema:{Column: [test.t1.a,test.t1.b] Key: [] Unique key: []}}", + "GID:3, logic prop:{stats:{count 2, ColNDVs map[4:2]}, schema:{Column: [Column#4] Key: [] Unique key: []}}", + "GID:4, logic prop:{stats:{count 2, ColNDVs map[4:2]}, schema:{Column: [Column#4] Key: [] Unique key: []}}" + ] }, { "SQL": "select count(tmp.a_sum) from (select t1.a as a, t1.b as b, sum(a) over() as a_sum from t1) tmp group by tmp.a, tmp.b", - "Str": "GID:1, logic prop:{stats:{count 5, ColNDVs map[1:2 2:2]}, schema:{Column: [test.t1.a,test.t1.b] Key: [] Unique key: []}}\nGID:2, logic prop:{stats:{count 5, ColNDVs map[1:2 2:2 5:5]}, schema:{Column: [test.t1.a,test.t1.b,Column#5] Key: [] Unique key: []}}\nGID:3, logic prop:{stats:{count 4, ColNDVs map[6:4]}, schema:{Column: [Column#6] Key: [] Unique key: []}}\nGID:4, logic prop:{stats:{count 4, ColNDVs map[6:4]}, schema:{Column: [Column#6] Key: [] Unique key: []}}" + "Str": [ + "GID:1, logic prop:{stats:{count 5, ColNDVs map[1:2 2:2]}, schema:{Column: [test.t1.a,test.t1.b] Key: [] Unique key: []}}", + "GID:2, logic prop:{stats:{count 5, ColNDVs map[1:2 2:2 5:5]}, schema:{Column: [test.t1.a,test.t1.b,Column#5] Key: [] Unique key: []}}", + "GID:3, logic prop:{stats:{count 4, ColNDVs map[6:4]}, schema:{Column: [Column#6] Key: [] Unique key: []}}", + "GID:4, logic prop:{stats:{count 4, ColNDVs map[6:4]}, schema:{Column: [Column#6] Key: [] Unique key: []}}" + ] } ] } From 07bdc92f9d137fbcd32bb0289802f2307fc6c6fc Mon Sep 17 00:00:00 2001 From: arenatlx <314806019@qq.com> Date: Fri, 6 Dec 2024 12:21:26 +0800 Subject: [PATCH 04/25] . Signed-off-by: arenatlx <314806019@qq.com> --- pkg/planner/core/casetest/cascades/memo_test.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pkg/planner/core/casetest/cascades/memo_test.go b/pkg/planner/core/casetest/cascades/memo_test.go index 1d56007502fb2..909268eab4aeb 100644 --- a/pkg/planner/core/casetest/cascades/memo_test.go +++ b/pkg/planner/core/casetest/cascades/memo_test.go @@ -1,4 +1,4 @@ -// Copyright 2020 PingCAP, Inc. +// Copyright 2024 PingCAP, Inc. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. From 7d3e58852cf3e4e72476b64be2ec060a1f677850 Mon Sep 17 00:00:00 2001 From: arenatlx <314806019@qq.com> Date: Fri, 6 Dec 2024 14:28:00 +0800 Subject: [PATCH 05/25] . Signed-off-by: arenatlx <314806019@qq.com> --- pkg/planner/cascades/rule/binder_test.go | 33 +++++++++++++--------- pkg/planner/core/operator/baseimpl/plan.go | 9 +++++- pkg/sessionctx/variable/session.go | 3 ++ 3 files changed, 31 insertions(+), 14 deletions(-) diff --git a/pkg/planner/cascades/rule/binder_test.go b/pkg/planner/cascades/rule/binder_test.go index 50f5686fde5c1..6d3349787aaba 100644 --- a/pkg/planner/cascades/rule/binder_test.go +++ b/pkg/planner/cascades/rule/binder_test.go @@ -29,6 +29,7 @@ import ( func TestBinderSuccess(t *testing.T) { ctx := mock.NewContext() + ctx.GetSessionVars().MockPlan = true t1 := logicalop.DataSource{}.Init(ctx, 0) t2 := logicalop.DataSource{}.Init(ctx, 0) join := logicalop.LogicalJoin{}.Init(ctx, 0) @@ -63,6 +64,7 @@ func TestBinderSuccess(t *testing.T) { func TestBinderFail(t *testing.T) { ctx := mock.NewContext() + ctx.GetSessionVars().MockPlan = true t1 := logicalop.DataSource{}.Init(ctx, 0) t2 := logicalop.DataSource{}.Init(ctx, 0) join := logicalop.LogicalJoin{}.Init(ctx, 0) @@ -86,7 +88,7 @@ func TestBinderFail(t *testing.T) { holder := binder.Next() require.Nil(t, holder) buf.Flush() - require.Equal(t, b.String(), "GE:DataSource_1{}\n") + require.Equal(t, b.String(), "GE:DataSource_-1{}\n") s1 := logicalop.LogicalLimit{}.Init(ctx, 0) s1.SetChildren(t1) @@ -117,11 +119,12 @@ func TestBinderFail(t *testing.T) { holder = binder.Next() require.Nil(t, holder) buf.Flush() - require.Equal(t, b.String(), "GE:Limit_4{inputs:1}\n") + require.Equal(t, b.String(), "GE:Limit_-4{GID:1}\n") } func TestBinderTopNode(t *testing.T) { ctx := mock.NewContext() + ctx.GetSessionVars().MockPlan = true t1 := logicalop.DataSource{}.Init(ctx, 0) t2 := logicalop.DataSource{}.Init(ctx, 0) join := logicalop.LogicalJoin{}.Init(ctx, 0) @@ -142,6 +145,7 @@ func TestBinderTopNode(t *testing.T) { func TestBinderOneNode(t *testing.T) { ctx := mock.NewContext() + ctx.GetSessionVars().MockPlan = true join := logicalop.LogicalJoin{}.Init(ctx, 0) mm := memo.NewMemo() @@ -158,6 +162,7 @@ func TestBinderOneNode(t *testing.T) { func TestBinderSubTreeMatch(t *testing.T) { ctx := mock.NewContext() + ctx.GetSessionVars().MockPlan = true t1 := logicalop.DataSource{}.Init(ctx, 0) t2 := logicalop.DataSource{}.Init(ctx, 0) join1 := logicalop.LogicalJoin{}.Init(ctx, 0) @@ -201,6 +206,7 @@ func TestBinderSubTreeMatch(t *testing.T) { func TestBinderMultiNext(t *testing.T) { ctx := mock.NewContext() + ctx.GetSessionVars().MockPlan = true asT1 := pmodel.NewCIStr("t1") asT2 := pmodel.NewCIStr("t2") t1 := logicalop.DataSource{TableAsName: &asT1}.Init(ctx, 0) @@ -289,15 +295,16 @@ func TestBinderMultiNext(t *testing.T) { // when G3 is exhausted, and next gE will be nil, and next() loop will enter next round with stack info popped as // G2(id(1)) which is what the third line comes from, and the next round will start from G2.next element starting // as G2(id(4)) which is the prefix of the fourth and fifth stack info. - require.Equal(t, b.String(), "GE:DataSource_1{} -> GE:DataSource_2{}\n"+ - "GE:DataSource_1{} -> GE:DataSource_5{}\n"+ - "GE:DataSource_1{}\n"+ - "GE:DataSource_4{} -> GE:DataSource_2{}\n"+ - "GE:DataSource_4{} -> GE:DataSource_5{}\n") + require.Equal(t, b.String(), "GE:DataSource_-1{} -> GE:DataSource_-2{}\n"+ + "GE:DataSource_-1{} -> GE:DataSource_-5{}\n"+ + "GE:DataSource_-1{}\n"+ + "GE:DataSource_-4{} -> GE:DataSource_-2{}\n"+ + "GE:DataSource_-4{} -> GE:DataSource_-5{}\n") } func TestBinderAny(t *testing.T) { ctx := mock.NewContext() + ctx.GetSessionVars().MockPlan = true asT1 := pmodel.NewCIStr("t1") asT2 := pmodel.NewCIStr("t2") t1 := logicalop.DataSource{TableAsName: &asT1}.Init(ctx, 0) @@ -373,10 +380,10 @@ func TestBinderAny(t *testing.T) { // In a conclusion: the Group matched with Any pattern only generate the first group expression since we don't // care what the concrete group expression it is. Because the final generated group expression if any, will be // substituted ANY pattern with the referred group at last not a concrete one group expression inside. - require.Equal(t, b.String(), "GE:DataSource_1{} -> GE:DataSource_2{}\n"+ - "GE:DataSource_1{}\n"+ - "GE:DataSource_4{} -> GE:DataSource_2{}\n"+ - "GE:DataSource_4{}\n") + require.Equal(t, b.String(), "GE:DataSource_-1{} -> GE:DataSource_-2{}\n"+ + "GE:DataSource_-1{}\n"+ + "GE:DataSource_-4{} -> GE:DataSource_-2{}\n"+ + "GE:DataSource_-4{}\n") } func TestBinderMultiAny(t *testing.T) { @@ -444,6 +451,6 @@ func TestBinderMultiAny(t *testing.T) { // G2{t1,t3} G3{t2,t4} // ▴ (already matched, pop stack) // final state: empty stack - require.Equal(t, b.String(), "GE:DataSource_1{} -> GE:DataSource_2{}\n"+ - "GE:DataSource_1{}\n") + require.Equal(t, b.String(), "GE:DataSource_-1{} -> GE:DataSource_-2{}\n"+ + "GE:DataSource_-1{}\n") } diff --git a/pkg/planner/core/operator/baseimpl/plan.go b/pkg/planner/core/operator/baseimpl/plan.go index 60459a651d1fa..45b4434768f96 100644 --- a/pkg/planner/core/operator/baseimpl/plan.go +++ b/pkg/planner/core/operator/baseimpl/plan.go @@ -16,6 +16,7 @@ package baseimpl import ( "fmt" + "github.com/pingcap/tidb/pkg/util/intest" "strconv" "unsafe" @@ -39,7 +40,13 @@ type Plan struct { // NewBasePlan creates a new base plan. func NewBasePlan(ctx planctx.PlanContext, tp string, qbBlock int) Plan { - id := ctx.GetSessionVars().PlanID.Add(1) + delta := int32(1) + if ctx.GetSessionVars().MockPlan { + delta = int32(-1) + intest.Assert(ctx.GetSessionVars().PlanID.Load() <= 0) + } + intest.Assert(ctx.GetSessionVars().PlanID.Load() >= 0) + id := ctx.GetSessionVars().PlanID.Add(delta) return Plan{ tp: tp, id: int(id), diff --git a/pkg/sessionctx/variable/session.go b/pkg/sessionctx/variable/session.go index 0551e777188c8..696c6af2ba8df 100644 --- a/pkg/sessionctx/variable/session.go +++ b/pkg/sessionctx/variable/session.go @@ -857,6 +857,9 @@ type SessionVars struct { // SQLCPUUsages records tidb/tikv cpu usages for current sql SQLCPUUsages ppcpuusage.SQLCPUUsages + // MockPlan is tagged in test to generate logical plan with negative plan ids. + MockPlan bool + // PlanID is the unique id of logical and physical plan. PlanID atomic.Int32 From 45b263094a61c6ca718786da0ebec1941ed1b9bf Mon Sep 17 00:00:00 2001 From: arenatlx <314806019@qq.com> Date: Fri, 6 Dec 2024 14:29:09 +0800 Subject: [PATCH 06/25] . Signed-off-by: arenatlx <314806019@qq.com> --- pkg/planner/core/operator/baseimpl/plan.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pkg/planner/core/operator/baseimpl/plan.go b/pkg/planner/core/operator/baseimpl/plan.go index 45b4434768f96..49dd28fb811dc 100644 --- a/pkg/planner/core/operator/baseimpl/plan.go +++ b/pkg/planner/core/operator/baseimpl/plan.go @@ -16,7 +16,6 @@ package baseimpl import ( "fmt" - "github.com/pingcap/tidb/pkg/util/intest" "strconv" "unsafe" @@ -25,6 +24,7 @@ import ( "github.com/pingcap/tidb/pkg/planner/planctx" "github.com/pingcap/tidb/pkg/planner/property" "github.com/pingcap/tidb/pkg/types" + "github.com/pingcap/tidb/pkg/util/intest" "github.com/pingcap/tidb/pkg/util/stringutil" "github.com/pingcap/tidb/pkg/util/tracing" ) From a6ff6ff017fff5df21e6ec4662cb8691d80222c5 Mon Sep 17 00:00:00 2001 From: arenatlx <314806019@qq.com> Date: Fri, 6 Dec 2024 15:10:53 +0800 Subject: [PATCH 07/25] . Signed-off-by: arenatlx <314806019@qq.com> --- pkg/planner/core/operator/baseimpl/BUILD.bazel | 1 + 1 file changed, 1 insertion(+) diff --git a/pkg/planner/core/operator/baseimpl/BUILD.bazel b/pkg/planner/core/operator/baseimpl/BUILD.bazel index 01c216d35fda2..ff054bc3209e5 100644 --- a/pkg/planner/core/operator/baseimpl/BUILD.bazel +++ b/pkg/planner/core/operator/baseimpl/BUILD.bazel @@ -11,6 +11,7 @@ go_library( "//pkg/planner/planctx", "//pkg/planner/property", "//pkg/types", + "//pkg/util/intest", "//pkg/util/stringutil", "//pkg/util/tracing", ], From f9279e5a5cafcf359b560c732af55033a2d7e83b Mon Sep 17 00:00:00 2001 From: arenatlx <314806019@qq.com> Date: Fri, 6 Dec 2024 15:25:14 +0800 Subject: [PATCH 08/25] . Signed-off-by: arenatlx <314806019@qq.com> --- pkg/planner/cascades/memo/memo_test.go | 8 ++------ 1 file changed, 2 insertions(+), 6 deletions(-) diff --git a/pkg/planner/cascades/memo/memo_test.go b/pkg/planner/cascades/memo/memo_test.go index 513a3868ad14e..e9d2313cf070e 100644 --- a/pkg/planner/cascades/memo/memo_test.go +++ b/pkg/planner/cascades/memo/memo_test.go @@ -24,12 +24,10 @@ import ( func TestMemo(t *testing.T) { ctx := plannercore.MockContext() + ctx.GetSessionVars().MockPlan = true t1 := logicalop.DataSource{}.Init(ctx, 0) - t1.SetID(-1) t2 := logicalop.DataSource{}.Init(ctx, 0) - t2.SetID(-2) join := logicalop.LogicalJoin{}.Init(ctx, 0) - join.SetID(-3) join.SetChildren(t1, t2) mm := NewMemo() @@ -49,12 +47,10 @@ func TestMemo(t *testing.T) { func TestInsertGE(t *testing.T) { ctx := plannercore.MockContext() + ctx.GetSessionVars().MockPlan = true t1 := logicalop.DataSource{}.Init(ctx, 0) - t1.SetID(-1) t2 := logicalop.DataSource{}.Init(ctx, 0) - t2.SetID(-2) join := logicalop.LogicalJoin{}.Init(ctx, 0) - join.SetID(-3) join.SetChildren(t1, t2) mm := NewMemo() From 23eae91e59ab4b056c44215feceaa3fbc87938d2 Mon Sep 17 00:00:00 2001 From: arenatlx <314806019@qq.com> Date: Fri, 6 Dec 2024 16:17:21 +0800 Subject: [PATCH 09/25] . Signed-off-by: arenatlx <314806019@qq.com> --- pkg/planner/cascades/memo/group.go | 8 +- pkg/planner/cascades/memo/memo.go | 8 +- .../core/casetest/cascades/memo_test.go | 13 +- .../cascades/testdata/cascades_suite_out.json | 234 +++++++++--------- 4 files changed, 140 insertions(+), 123 deletions(-) diff --git a/pkg/planner/cascades/memo/group.go b/pkg/planner/cascades/memo/group.go index f19c1f9a37179..37f0f6301a2a5 100644 --- a/pkg/planner/cascades/memo/group.go +++ b/pkg/planner/cascades/memo/group.go @@ -153,10 +153,14 @@ func (g *Group) String(w util.StrBufferWriter) { } // ForEachGE traverse the inside group expression with f call on them each. -func (g *Group) ForEachGE(f func(ge *GroupExpression)) { +func (g *Group) ForEachGE(f func(ge *GroupExpression) bool) { + next := true for elem := g.logicalExpressions.Front(); elem != nil; elem = elem.Next() { expr := elem.Value.(*GroupExpression) - f(expr) + next = f(expr) + if !next { + break + } } } diff --git a/pkg/planner/cascades/memo/memo.go b/pkg/planner/cascades/memo/memo.go index 9edc4e94fc0e6..a09822a447039 100644 --- a/pkg/planner/cascades/memo/memo.go +++ b/pkg/planner/cascades/memo/memo.go @@ -163,9 +163,13 @@ func (mm *Memo) Init(plan base.LogicalPlan) *GroupExpression { } // ForEachGroup traverse the inside group expression with f call on them each. -func (mm *Memo) ForEachGroup(f func(g *Group)) { +func (mm *Memo) ForEachGroup(f func(g *Group) bool) { + next := true for elem := mm.GetGroups().Front(); elem != nil; elem = elem.Next() { expr := elem.Value.(*Group) - f(expr) + next = f(expr) + if !next { + break + } } } diff --git a/pkg/planner/core/casetest/cascades/memo_test.go b/pkg/planner/core/casetest/cascades/memo_test.go index 909268eab4aeb..dae0149f20db5 100644 --- a/pkg/planner/core/casetest/cascades/memo_test.go +++ b/pkg/planner/core/casetest/cascades/memo_test.go @@ -17,6 +17,7 @@ package cascades import ( "bytes" "context" + "fmt" "testing" "github.com/pingcap/tidb/pkg/parser" @@ -76,11 +77,17 @@ func TestDeriveStats(t *testing.T) { b := &bytes.Buffer{} sb := util.NewStrBuffer(b) var strs []string - mm.ForEachGroup(func(g *memo.Group) { + mm.ForEachGroup(func(g *memo.Group) bool { b.Reset() // record group g.String(sb) sb.WriteString(", ") + // record first ge + g.ForEachGE(func(ge *memo.GroupExpression) bool { + ge.String(sb) + return false + }) + sb.WriteString(", ") // record group stats logicProp := g.GetLogicalProperty() if logicProp == nil { @@ -90,7 +97,8 @@ func TestDeriveStats(t *testing.T) { if logicProp.Stats == nil { sb.WriteString("stats:nil,") } else { - sb.WriteString("stats:{" + logicProp.Stats.String() + "}") + statsStr := fmt.Sprintf("count %v, ColNDVs %v, GroupNDVs %v", logicProp.Stats.RowCount, logicProp.Stats.ColNDVs, logicProp.Stats.GroupNDVs) + sb.WriteString("stats:{" + statsStr + "}") } sb.WriteString(", ") if logicProp.Schema == nil { @@ -102,6 +110,7 @@ func TestDeriveStats(t *testing.T) { } sb.Flush() strs = append(strs, b.String()) + return true }) testdata.OnRecord(func() { output[i].SQL = tt diff --git a/pkg/planner/core/casetest/cascades/testdata/cascades_suite_out.json b/pkg/planner/core/casetest/cascades/testdata/cascades_suite_out.json index d8315b7f5f081..af8b391df6569 100644 --- a/pkg/planner/core/casetest/cascades/testdata/cascades_suite_out.json +++ b/pkg/planner/core/casetest/cascades/testdata/cascades_suite_out.json @@ -5,243 +5,243 @@ { "SQL": "select count(1) from t1 group by a, b", "Str": [ - "GID:1, logic prop:{stats:{count 5, ColNDVs map[1:2 2:2]}, schema:{Column: [test.t1.a,test.t1.b] Key: [] Unique key: []}}", - "GID:2, logic prop:{stats:{count 4, ColNDVs map[4:4]}, schema:{Column: [Column#4] Key: [] Unique key: []}}", - "GID:3, logic prop:{stats:{count 4, ColNDVs map[4:4]}, schema:{Column: [Column#4] Key: [] Unique key: []}}" + "GID:1, GE:DataSource_1{}, logic prop:{stats:{count 5, ColNDVs map[1:2 2:2], GroupNDVs [{[1 2] 4}]}, schema:{Column: [test.t1.a,test.t1.b] Key: [] Unique key: []}}", + "GID:2, GE:Aggregation_2{GID:1}, logic prop:{stats:{count 4, ColNDVs map[4:4], GroupNDVs []}, schema:{Column: [Column#4] Key: [] Unique key: []}}", + "GID:3, GE:Projection_3{GID:2}, logic prop:{stats:{count 4, ColNDVs map[4:4], GroupNDVs []}, schema:{Column: [Column#4] Key: [] Unique key: []}}" ] }, { "SQL": "select * from t1, t2 where t1.a = t2.a and t1.b = t2.b", "Str": [ - "GID:1, logic prop:{stats:{count 5, ColNDVs map[1:2 2:2]}, schema:{Column: [test.t1.a,test.t1.b] Key: [] Unique key: []}}", - "GID:2, logic prop:{stats:{count 10, ColNDVs map[4:3 5:3]}, schema:{Column: [test.t2.a,test.t2.b] Key: [] Unique key: []}}", - "GID:3, logic prop:{stats:{count 5.555555555555555, ColNDVs map[1:2 2:2 4:3 5:3]}, schema:{Column: [test.t1.a,test.t1.b,test.t2.a,test.t2.b] Key: [] Unique key: []}}", - "GID:4, logic prop:{stats:{count 5.555555555555555, ColNDVs map[1:2 2:2 4:3 5:3]}, schema:{Column: [test.t1.a,test.t1.b,test.t2.a,test.t2.b] Key: [] Unique key: []}}" + "GID:1, GE:DataSource_4{}, logic prop:{stats:{count 5, ColNDVs map[1:2 2:2], GroupNDVs [{[1 2] 4}]}, schema:{Column: [test.t1.a,test.t1.b] Key: [] Unique key: []}}", + "GID:2, GE:DataSource_5{}, logic prop:{stats:{count 10, ColNDVs map[4:3 5:3], GroupNDVs [{[4 5] 9}]}, schema:{Column: [test.t2.a,test.t2.b] Key: [] Unique key: []}}", + "GID:3, GE:Join_9{GID:1, GID:2}, logic prop:{stats:{count 5.555555555555555, ColNDVs map[1:2 2:2 4:3 5:3], GroupNDVs []}, schema:{Column: [test.t1.a,test.t1.b,test.t2.a,test.t2.b] Key: [] Unique key: []}}", + "GID:4, GE:Projection_8{GID:3}, logic prop:{stats:{count 5.555555555555555, ColNDVs map[1:2 2:2 4:3 5:3], GroupNDVs []}, schema:{Column: [test.t1.a,test.t1.b,test.t2.a,test.t2.b] Key: [] Unique key: []}}" ] }, { "SQL": "select count(1) from t1 where a > 0 group by a, b", "Str": [ - "GID:1, logic prop:{stats:{count 5, ColNDVs map[1:2 2:2]}, schema:{Column: [test.t1.a,test.t1.b] Key: [] Unique key: []}}", - "GID:2, logic prop:{stats:{count 4, ColNDVs map[4:4]}, schema:{Column: [Column#4] Key: [] Unique key: []}}", - "GID:3, logic prop:{stats:{count 4, ColNDVs map[4:4]}, schema:{Column: [Column#4] Key: [] Unique key: []}}" + "GID:1, GE:DataSource_10{}, logic prop:{stats:{count 5, ColNDVs map[1:2 2:2], GroupNDVs [{[1 2] 4}]}, schema:{Column: [test.t1.a,test.t1.b] Key: [] Unique key: []}}", + "GID:2, GE:Aggregation_12{GID:1}, logic prop:{stats:{count 4, ColNDVs map[4:4], GroupNDVs []}, schema:{Column: [Column#4] Key: [] Unique key: []}}", + "GID:3, GE:Projection_13{GID:2}, logic prop:{stats:{count 4, ColNDVs map[4:4], GroupNDVs []}, schema:{Column: [Column#4] Key: [] Unique key: []}}" ] }, { "SQL": "select count(1) from t1 where b > 0 group by a, b", "Str": [ - "GID:1, logic prop:{stats:{count 5, ColNDVs map[1:2 2:2]}, schema:{Column: [test.t1.a,test.t1.b] Key: [] Unique key: []}}", - "GID:2, logic prop:{stats:{count 4, ColNDVs map[4:4]}, schema:{Column: [Column#4] Key: [] Unique key: []}}", - "GID:3, logic prop:{stats:{count 4, ColNDVs map[4:4]}, schema:{Column: [Column#4] Key: [] Unique key: []}}" + "GID:1, GE:DataSource_14{}, logic prop:{stats:{count 5, ColNDVs map[1:2 2:2], GroupNDVs [{[1 2] 4}]}, schema:{Column: [test.t1.a,test.t1.b] Key: [] Unique key: []}}", + "GID:2, GE:Aggregation_16{GID:1}, logic prop:{stats:{count 4, ColNDVs map[4:4], GroupNDVs []}, schema:{Column: [Column#4] Key: [] Unique key: []}}", + "GID:3, GE:Projection_17{GID:2}, logic prop:{stats:{count 4, ColNDVs map[4:4], GroupNDVs []}, schema:{Column: [Column#4] Key: [] Unique key: []}}" ] }, { "SQL": "select count(1) from t1 where cos(a) > 0 group by a, b", "Str": [ - "GID:1, logic prop:{stats:{count 4, ColNDVs map[1:1.6 2:1.6]}, schema:{Column: [test.t1.a,test.t1.b] Key: [] Unique key: []}}", - "GID:2, logic prop:{stats:{count 3.2, ColNDVs map[4:3.2]}, schema:{Column: [Column#4] Key: [] Unique key: []}}", - "GID:3, logic prop:{stats:{count 3.2, ColNDVs map[4:3.2]}, schema:{Column: [Column#4] Key: [] Unique key: []}}" + "GID:1, GE:DataSource_18{}, logic prop:{stats:{count 4, ColNDVs map[1:1.6 2:1.6], GroupNDVs [{[1 2] 3.2}]}, schema:{Column: [test.t1.a,test.t1.b] Key: [] Unique key: []}}", + "GID:2, GE:Aggregation_20{GID:1}, logic prop:{stats:{count 3.2, ColNDVs map[4:3.2], GroupNDVs []}, schema:{Column: [Column#4] Key: [] Unique key: []}}", + "GID:3, GE:Projection_21{GID:2}, logic prop:{stats:{count 3.2, ColNDVs map[4:3.2], GroupNDVs []}, schema:{Column: [Column#4] Key: [] Unique key: []}}" ] }, { "SQL": "select count(c3) from (select a as c1, b as c2, a+1 as c3 from t1) as tmp group by c2, c1", "Str": [ - "GID:1, logic prop:{stats:{count 5, ColNDVs map[1:2 2:2]}, schema:{Column: [test.t1.a,test.t1.b] Key: [] Unique key: []}}", - "GID:2, logic prop:{stats:{count 4, ColNDVs map[5:4]}, schema:{Column: [Column#5] Key: [] Unique key: []}}", - "GID:3, logic prop:{stats:{count 4, ColNDVs map[5:4]}, schema:{Column: [Column#5] Key: [] Unique key: []}}" + "GID:1, GE:DataSource_22{}, logic prop:{stats:{count 5, ColNDVs map[1:2 2:2], GroupNDVs [{[1 2] 4}]}, schema:{Column: [test.t1.a,test.t1.b] Key: [] Unique key: []}}", + "GID:2, GE:Aggregation_24{GID:1}, logic prop:{stats:{count 4, ColNDVs map[5:4], GroupNDVs []}, schema:{Column: [Column#5] Key: [] Unique key: []}}", + "GID:3, GE:Projection_25{GID:2}, logic prop:{stats:{count 4, ColNDVs map[5:4], GroupNDVs []}, schema:{Column: [Column#5] Key: [] Unique key: []}}" ] }, { "SQL": "select count(c3) from (select a+b as c1, b as c2, a+1 as c3 from t1) as tmp group by c2, c1", "Str": [ - "GID:1, logic prop:{stats:{count 5, ColNDVs map[1:2 2:2]}, schema:{Column: [test.t1.a,test.t1.b] Key: [] Unique key: []}}", - "GID:2, logic prop:{stats:{count 2, ColNDVs map[6:2]}, schema:{Column: [Column#6] Key: [] Unique key: []}}", - "GID:3, logic prop:{stats:{count 2, ColNDVs map[6:2]}, schema:{Column: [Column#6] Key: [] Unique key: []}}" + "GID:1, GE:DataSource_26{}, logic prop:{stats:{count 5, ColNDVs map[1:2 2:2], GroupNDVs []}, schema:{Column: [test.t1.a,test.t1.b] Key: [] Unique key: []}}", + "GID:2, GE:Aggregation_28{GID:1}, logic prop:{stats:{count 2, ColNDVs map[6:2], GroupNDVs []}, schema:{Column: [Column#6] Key: [] Unique key: []}}", + "GID:3, GE:Projection_29{GID:2}, logic prop:{stats:{count 2, ColNDVs map[6:2], GroupNDVs []}, schema:{Column: [Column#6] Key: [] Unique key: []}}" ] }, { "SQL": "select count(tmp.cmp) from (select t1.a as a, t1.b as b, (t1.b > (select t2.b from t2 where t2.a = t1.a)) as cmp from t1) tmp group by tmp.a, tmp.b", "Str": [ - "GID:1, logic prop:{stats:{count 5, ColNDVs map[1:2 2:2]}, schema:{Column: [test.t1.a,test.t1.b] Key: [] Unique key: []}}", - "GID:2, logic prop:{stats:{count 3.333333333333333, ColNDVs map[7:1 8:1]}, schema:{Column: [test.t2.a,test.t2.b] Key: [] Unique key: []}}", - "GID:3, logic prop:{stats:{count 1, ColNDVs map[7:1 8:1]}, schema:{Column: [test.t2.a,test.t2.b] Key: [] Unique key: []}}", - "GID:4, logic prop:{stats:{count 5, ColNDVs map[1:2 2:2 7:5 8:5]}, schema:{Column: [test.t1.a,test.t1.b,test.t2.a,test.t2.b] Key: [] Unique key: []}}", - "GID:5, logic prop:{stats:{count 4, ColNDVs map[11:4]}, schema:{Column: [Column#11] Key: [] Unique key: []}}", - "GID:6, logic prop:{stats:{count 4, ColNDVs map[11:4]}, schema:{Column: [Column#11] Key: [] Unique key: []}}" + "GID:1, GE:DataSource_30{}, logic prop:{stats:{count 5, ColNDVs map[1:2 2:2], GroupNDVs [{[1 2] 4}]}, schema:{Column: [test.t1.a,test.t1.b] Key: [] Unique key: []}}", + "GID:2, GE:DataSource_33{}, logic prop:{stats:{count 3.333333333333333, ColNDVs map[7:1 8:1], GroupNDVs []}, schema:{Column: [test.t2.a,test.t2.b] Key: [] Unique key: []}}", + "GID:3, GE:MaxOneRow_36{GID:2}, logic prop:{stats:{count 1, ColNDVs map[7:1 8:1], GroupNDVs []}, schema:{Column: [test.t2.a,test.t2.b] Key: [] Unique key: []}}", + "GID:4, GE:Apply_37{GID:1, GID:3}, logic prop:{stats:{count 5, ColNDVs map[1:2 2:2 7:5 8:5], GroupNDVs [{[1 2] 4}]}, schema:{Column: [test.t1.a,test.t1.b,test.t2.a,test.t2.b] Key: [] Unique key: []}}", + "GID:5, GE:Aggregation_38{GID:4}, logic prop:{stats:{count 4, ColNDVs map[11:4], GroupNDVs []}, schema:{Column: [Column#11] Key: [] Unique key: []}}", + "GID:6, GE:Projection_39{GID:5}, logic prop:{stats:{count 4, ColNDVs map[11:4], GroupNDVs []}, schema:{Column: [Column#11] Key: [] Unique key: []}}" ] }, { "SQL": "select count(1) from (select t1.a as a, t1.b as b from t1 where t1.b > (select t2.b from t2 where t2.a = t1.a)) tmp group by tmp.a, tmp.b", "Str": [ - "GID:1, logic prop:{stats:{count 5, ColNDVs map[1:2 2:2]}, schema:{Column: [test.t1.a,test.t1.b] Key: [] Unique key: []}}", - "GID:2, logic prop:{stats:{count 3.333333333333333, ColNDVs map[4:1 5:1]}, schema:{Column: [test.t2.a,test.t2.b] Key: [] Unique key: []}}", - "GID:3, logic prop:{stats:{count 1, ColNDVs map[4:1 5:1]}, schema:{Column: [test.t2.a,test.t2.b] Key: [] Unique key: []}}", - "GID:4, logic prop:{stats:{count 0.8, ColNDVs map[4:0.8 5:0.8]}, schema:{Column: [test.t2.a,test.t2.b] Key: [] Unique key: []}}", - "GID:5, logic prop:{stats:{count 5, ColNDVs map[1:2 2:2 4:5 5:5]}, schema:{Column: [test.t1.a,test.t1.b,test.t2.a,test.t2.b] Key: [] Unique key: []}}", - "GID:6, logic prop:{stats:{count 2, ColNDVs map[7:2]}, schema:{Column: [Column#7] Key: [] Unique key: []}}", - "GID:7, logic prop:{stats:{count 2, ColNDVs map[7:2]}, schema:{Column: [Column#7] Key: [] Unique key: []}}" + "GID:1, GE:DataSource_40{}, logic prop:{stats:{count 5, ColNDVs map[1:2 2:2], GroupNDVs []}, schema:{Column: [test.t1.a,test.t1.b] Key: [] Unique key: []}}", + "GID:2, GE:DataSource_42{}, logic prop:{stats:{count 3.333333333333333, ColNDVs map[4:1 5:1], GroupNDVs []}, schema:{Column: [test.t2.a,test.t2.b] Key: [] Unique key: []}}", + "GID:3, GE:MaxOneRow_45{GID:2}, logic prop:{stats:{count 1, ColNDVs map[4:1 5:1], GroupNDVs []}, schema:{Column: [test.t2.a,test.t2.b] Key: [] Unique key: []}}", + "GID:4, GE:Selection_50{GID:3}, logic prop:{stats:{count 0.8, ColNDVs map[4:0.8 5:0.8], GroupNDVs []}, schema:{Column: [test.t2.a,test.t2.b] Key: [] Unique key: []}}", + "GID:5, GE:Apply_46{GID:1, GID:4}, logic prop:{stats:{count 5, ColNDVs map[1:2 2:2 4:5 5:5], GroupNDVs []}, schema:{Column: [test.t1.a,test.t1.b,test.t2.a,test.t2.b] Key: [] Unique key: []}}", + "GID:6, GE:Aggregation_48{GID:5}, logic prop:{stats:{count 2, ColNDVs map[7:2], GroupNDVs []}, schema:{Column: [Column#7] Key: [] Unique key: []}}", + "GID:7, GE:Projection_49{GID:6}, logic prop:{stats:{count 2, ColNDVs map[7:2], GroupNDVs []}, schema:{Column: [Column#7] Key: [] Unique key: []}}" ] }, { "SQL": "select count(tmp.cmp) from (select t1.a as a, t1.b as b, (t1.b in (select t2.b from t2 where t2.a = t1.a limit 3)) as cmp from t1) tmp group by tmp.a, tmp.b", "Str": [ - "GID:1, logic prop:{stats:{count 5, ColNDVs map[1:2 2:2]}, schema:{Column: [test.t1.a,test.t1.b] Key: [] Unique key: []}}", - "GID:2, logic prop:{stats:{count 3.333333333333333, ColNDVs map[7:1 8:1]}, schema:{Column: [test.t2.a,test.t2.b] Key: [] Unique key: []}}", - "GID:3, logic prop:{stats:{count 3, ColNDVs map[7:1 8:1]}, schema:{Column: [test.t2.a,test.t2.b] Key: [] Unique key: []}}", - "GID:4, logic prop:{stats:{count 5, ColNDVs map[1:2 2:2 10:2]}, schema:{Column: [test.t1.a,test.t1.b,Column#10] Key: [] Unique key: []}}", - "GID:5, logic prop:{stats:{count 4, ColNDVs map[11:4]}, schema:{Column: [Column#11] Key: [] Unique key: []}}", - "GID:6, logic prop:{stats:{count 4, ColNDVs map[11:4]}, schema:{Column: [Column#11] Key: [] Unique key: []}}" + "GID:1, GE:DataSource_51{}, logic prop:{stats:{count 5, ColNDVs map[1:2 2:2], GroupNDVs [{[1 2] 4}]}, schema:{Column: [test.t1.a,test.t1.b] Key: [] Unique key: []}}", + "GID:2, GE:DataSource_54{}, logic prop:{stats:{count 3.333333333333333, ColNDVs map[7:1 8:1], GroupNDVs []}, schema:{Column: [test.t2.a,test.t2.b] Key: [] Unique key: []}}", + "GID:3, GE:Limit_62{GID:2}, logic prop:{stats:{count 3, ColNDVs map[7:1 8:1], GroupNDVs []}, schema:{Column: [test.t2.a,test.t2.b] Key: [] Unique key: []}}", + "GID:4, GE:Apply_58{GID:1, GID:3}, logic prop:{stats:{count 5, ColNDVs map[1:2 2:2 10:2], GroupNDVs [{[1 2] 4}]}, schema:{Column: [test.t1.a,test.t1.b,Column#10] Key: [] Unique key: []}}", + "GID:5, GE:Aggregation_59{GID:4}, logic prop:{stats:{count 4, ColNDVs map[11:4], GroupNDVs []}, schema:{Column: [Column#11] Key: [] Unique key: []}}", + "GID:6, GE:Projection_60{GID:5}, logic prop:{stats:{count 4, ColNDVs map[11:4], GroupNDVs []}, schema:{Column: [Column#11] Key: [] Unique key: []}}" ] }, { "SQL": "select count(tmp.cmp) from (select t1.a as a, t1.b as b, (t1.b not in (select t2.b from t2 where t2.a = t1.a limit 3)) as cmp from t1) tmp group by tmp.a, tmp.b", "Str": [ - "GID:1, logic prop:{stats:{count 5, ColNDVs map[1:2 2:2]}, schema:{Column: [test.t1.a,test.t1.b] Key: [] Unique key: []}}", - "GID:2, logic prop:{stats:{count 3.333333333333333, ColNDVs map[7:1 8:1]}, schema:{Column: [test.t2.a,test.t2.b] Key: [] Unique key: []}}", - "GID:3, logic prop:{stats:{count 3, ColNDVs map[7:1 8:1]}, schema:{Column: [test.t2.a,test.t2.b] Key: [] Unique key: []}}", - "GID:4, logic prop:{stats:{count 5, ColNDVs map[1:2 2:2 10:2]}, schema:{Column: [test.t1.a,test.t1.b,Column#10] Key: [] Unique key: []}}", - "GID:5, logic prop:{stats:{count 4, ColNDVs map[11:4]}, schema:{Column: [Column#11] Key: [] Unique key: []}}", - "GID:6, logic prop:{stats:{count 4, ColNDVs map[11:4]}, schema:{Column: [Column#11] Key: [] Unique key: []}}" + "GID:1, GE:DataSource_63{}, logic prop:{stats:{count 5, ColNDVs map[1:2 2:2], GroupNDVs [{[1 2] 4}]}, schema:{Column: [test.t1.a,test.t1.b] Key: [] Unique key: []}}", + "GID:2, GE:DataSource_66{}, logic prop:{stats:{count 3.333333333333333, ColNDVs map[7:1 8:1], GroupNDVs []}, schema:{Column: [test.t2.a,test.t2.b] Key: [] Unique key: []}}", + "GID:3, GE:Limit_74{GID:2}, logic prop:{stats:{count 3, ColNDVs map[7:1 8:1], GroupNDVs []}, schema:{Column: [test.t2.a,test.t2.b] Key: [] Unique key: []}}", + "GID:4, GE:Apply_70{GID:1, GID:3}, logic prop:{stats:{count 5, ColNDVs map[1:2 2:2 10:2], GroupNDVs [{[1 2] 4}]}, schema:{Column: [test.t1.a,test.t1.b,Column#10] Key: [] Unique key: []}}", + "GID:5, GE:Aggregation_71{GID:4}, logic prop:{stats:{count 4, ColNDVs map[11:4], GroupNDVs []}, schema:{Column: [Column#11] Key: [] Unique key: []}}", + "GID:6, GE:Projection_72{GID:5}, logic prop:{stats:{count 4, ColNDVs map[11:4], GroupNDVs []}, schema:{Column: [Column#11] Key: [] Unique key: []}}" ] }, { "SQL": "select count(1) from (select t1.a as a, t1.b as b from t1 where t1.b in (select t2.b from t2 where t2.a = t1.a limit 3)) tmp group by tmp.a, tmp.b", "Str": [ - "GID:1, logic prop:{stats:{count 5, ColNDVs map[1:2 2:2]}, schema:{Column: [test.t1.a,test.t1.b] Key: [] Unique key: []}}", - "GID:2, logic prop:{stats:{count 3.333333333333333, ColNDVs map[4:1 5:1]}, schema:{Column: [test.t2.a,test.t2.b] Key: [] Unique key: []}}", - "GID:3, logic prop:{stats:{count 3, ColNDVs map[4:1 5:1]}, schema:{Column: [test.t2.a,test.t2.b] Key: [] Unique key: []}}", - "GID:4, logic prop:{stats:{count 5, ColNDVs map[1:2 2:2]}, schema:{Column: [test.t1.a,test.t1.b] Key: [] Unique key: []}}", - "GID:5, logic prop:{stats:{count 2, ColNDVs map[7:2]}, schema:{Column: [Column#7] Key: [] Unique key: []}}", - "GID:6, logic prop:{stats:{count 2, ColNDVs map[7:2]}, schema:{Column: [Column#7] Key: [] Unique key: []}}" + "GID:1, GE:DataSource_75{}, logic prop:{stats:{count 5, ColNDVs map[1:2 2:2], GroupNDVs []}, schema:{Column: [test.t1.a,test.t1.b] Key: [] Unique key: []}}", + "GID:2, GE:DataSource_77{}, logic prop:{stats:{count 3.333333333333333, ColNDVs map[4:1 5:1], GroupNDVs []}, schema:{Column: [test.t2.a,test.t2.b] Key: [] Unique key: []}}", + "GID:3, GE:Limit_86{GID:2}, logic prop:{stats:{count 3, ColNDVs map[4:1 5:1], GroupNDVs []}, schema:{Column: [test.t2.a,test.t2.b] Key: [] Unique key: []}}", + "GID:4, GE:Apply_81{GID:1, GID:3}, logic prop:{stats:{count 5, ColNDVs map[1:2 2:2], GroupNDVs []}, schema:{Column: [test.t1.a,test.t1.b] Key: [] Unique key: []}}", + "GID:5, GE:Aggregation_83{GID:4}, logic prop:{stats:{count 2, ColNDVs map[7:2], GroupNDVs []}, schema:{Column: [Column#7] Key: [] Unique key: []}}", + "GID:6, GE:Projection_84{GID:5}, logic prop:{stats:{count 2, ColNDVs map[7:2], GroupNDVs []}, schema:{Column: [Column#7] Key: [] Unique key: []}}" ] }, { "SQL": "select count(1) from (select t1.a as a, t1.b as b from t1 where t1.b not in (select t2.b from t2 where t2.a = t1.a limit 3)) tmp group by tmp.a, tmp.b", "Str": [ - "GID:1, logic prop:{stats:{count 5, ColNDVs map[1:2 2:2]}, schema:{Column: [test.t1.a,test.t1.b] Key: [] Unique key: []}}", - "GID:2, logic prop:{stats:{count 3.333333333333333, ColNDVs map[4:1 5:1]}, schema:{Column: [test.t2.a,test.t2.b] Key: [] Unique key: []}}", - "GID:3, logic prop:{stats:{count 3, ColNDVs map[4:1 5:1]}, schema:{Column: [test.t2.a,test.t2.b] Key: [] Unique key: []}}", - "GID:4, logic prop:{stats:{count 5, ColNDVs map[1:2 2:2]}, schema:{Column: [test.t1.a,test.t1.b] Key: [] Unique key: []}}", - "GID:5, logic prop:{stats:{count 2, ColNDVs map[7:2]}, schema:{Column: [Column#7] Key: [] Unique key: []}}", - "GID:6, logic prop:{stats:{count 2, ColNDVs map[7:2]}, schema:{Column: [Column#7] Key: [] Unique key: []}}" + "GID:1, GE:DataSource_87{}, logic prop:{stats:{count 5, ColNDVs map[1:2 2:2], GroupNDVs []}, schema:{Column: [test.t1.a,test.t1.b] Key: [] Unique key: []}}", + "GID:2, GE:DataSource_89{}, logic prop:{stats:{count 3.333333333333333, ColNDVs map[4:1 5:1], GroupNDVs []}, schema:{Column: [test.t2.a,test.t2.b] Key: [] Unique key: []}}", + "GID:3, GE:Limit_98{GID:2}, logic prop:{stats:{count 3, ColNDVs map[4:1 5:1], GroupNDVs []}, schema:{Column: [test.t2.a,test.t2.b] Key: [] Unique key: []}}", + "GID:4, GE:Apply_93{GID:1, GID:3}, logic prop:{stats:{count 5, ColNDVs map[1:2 2:2], GroupNDVs []}, schema:{Column: [test.t1.a,test.t1.b] Key: [] Unique key: []}}", + "GID:5, GE:Aggregation_95{GID:4}, logic prop:{stats:{count 2, ColNDVs map[7:2], GroupNDVs []}, schema:{Column: [Column#7] Key: [] Unique key: []}}", + "GID:6, GE:Projection_96{GID:5}, logic prop:{stats:{count 2, ColNDVs map[7:2], GroupNDVs []}, schema:{Column: [Column#7] Key: [] Unique key: []}}" ] }, { "SQL": "select count(1) from t1, t2 where t1.a = t2.a group by t1.a, t1.b", "Str": [ - "GID:1, logic prop:{stats:{count 5, ColNDVs map[1:2 2:2]}, schema:{Column: [test.t1.a,test.t1.b] Key: [] Unique key: []}}", - "GID:2, logic prop:{stats:{count 10, ColNDVs map[4:3]}, schema:{Column: [test.t2.a] Key: [] Unique key: []}}", - "GID:3, logic prop:{stats:{count 16.666666666666668, ColNDVs map[1:2 2:2 4:3]}, schema:{Column: [test.t1.a,test.t1.b,test.t2.a] Key: [] Unique key: []}}", - "GID:4, logic prop:{stats:{count 2, ColNDVs map[7:2]}, schema:{Column: [Column#7] Key: [] Unique key: []}}", - "GID:5, logic prop:{stats:{count 2, ColNDVs map[7:2]}, schema:{Column: [Column#7] Key: [] Unique key: []}}" + "GID:1, GE:DataSource_99{}, logic prop:{stats:{count 5, ColNDVs map[1:2 2:2], GroupNDVs []}, schema:{Column: [test.t1.a,test.t1.b] Key: [] Unique key: []}}", + "GID:2, GE:DataSource_100{}, logic prop:{stats:{count 10, ColNDVs map[4:3], GroupNDVs []}, schema:{Column: [test.t2.a] Key: [] Unique key: []}}", + "GID:3, GE:Join_105{GID:1, GID:2}, logic prop:{stats:{count 16.666666666666668, ColNDVs map[1:2 2:2 4:3], GroupNDVs []}, schema:{Column: [test.t1.a,test.t1.b,test.t2.a] Key: [] Unique key: []}}", + "GID:4, GE:Aggregation_103{GID:3}, logic prop:{stats:{count 2, ColNDVs map[7:2], GroupNDVs []}, schema:{Column: [Column#7] Key: [] Unique key: []}}", + "GID:5, GE:Projection_104{GID:4}, logic prop:{stats:{count 2, ColNDVs map[7:2], GroupNDVs []}, schema:{Column: [Column#7] Key: [] Unique key: []}}" ] }, { "SQL": "select count(1) from t1 left join t2 on t1.a = t2.a group by t1.a, t1.b", "Str": [ - "GID:1, logic prop:{stats:{count 5, ColNDVs map[1:2 2:2]}, schema:{Column: [test.t1.a,test.t1.b] Key: [] Unique key: []}}", - "GID:2, logic prop:{stats:{count 10, ColNDVs map[4:3]}, schema:{Column: [test.t2.a] Key: [] Unique key: []}}", - "GID:3, logic prop:{stats:{count 16.666666666666668, ColNDVs map[1:2 2:2 4:3]}, schema:{Column: [test.t1.a,test.t1.b,test.t2.a] Key: [] Unique key: []}}", - "GID:4, logic prop:{stats:{count 4, ColNDVs map[7:4]}, schema:{Column: [Column#7] Key: [] Unique key: []}}", - "GID:5, logic prop:{stats:{count 4, ColNDVs map[7:4]}, schema:{Column: [Column#7] Key: [] Unique key: []}}" + "GID:1, GE:DataSource_106{}, logic prop:{stats:{count 5, ColNDVs map[1:2 2:2], GroupNDVs [{[1 2] 4}]}, schema:{Column: [test.t1.a,test.t1.b] Key: [] Unique key: []}}", + "GID:2, GE:DataSource_107{}, logic prop:{stats:{count 10, ColNDVs map[4:3], GroupNDVs []}, schema:{Column: [test.t2.a] Key: [] Unique key: []}}", + "GID:3, GE:Join_111{GID:1, GID:2}, logic prop:{stats:{count 16.666666666666668, ColNDVs map[1:2 2:2 4:3], GroupNDVs [{[1 2] 4}]}, schema:{Column: [test.t1.a,test.t1.b,test.t2.a] Key: [] Unique key: []}}", + "GID:4, GE:Aggregation_109{GID:3}, logic prop:{stats:{count 4, ColNDVs map[7:4], GroupNDVs []}, schema:{Column: [Column#7] Key: [] Unique key: []}}", + "GID:5, GE:Projection_110{GID:4}, logic prop:{stats:{count 4, ColNDVs map[7:4], GroupNDVs []}, schema:{Column: [Column#7] Key: [] Unique key: []}}" ] }, { "SQL": "select count(1) from t1 left join t2 on t1.a = t2.a group by t2.a, t2.b", "Str": [ - "GID:1, logic prop:{stats:{count 5, ColNDVs map[1:2]}, schema:{Column: [test.t1.a] Key: [] Unique key: []}}", - "GID:2, logic prop:{stats:{count 10, ColNDVs map[4:3 5:3]}, schema:{Column: [test.t2.a,test.t2.b] Key: [] Unique key: []}}", - "GID:3, logic prop:{stats:{count 16.666666666666668, ColNDVs map[1:2 4:3 5:3]}, schema:{Column: [test.t1.a,test.t2.a,test.t2.b] Key: [] Unique key: []}}", - "GID:4, logic prop:{stats:{count 3, ColNDVs map[7:3]}, schema:{Column: [Column#7] Key: [] Unique key: []}}", - "GID:5, logic prop:{stats:{count 3, ColNDVs map[7:3]}, schema:{Column: [Column#7] Key: [] Unique key: []}}" + "GID:1, GE:DataSource_112{}, logic prop:{stats:{count 5, ColNDVs map[1:2], GroupNDVs []}, schema:{Column: [test.t1.a] Key: [] Unique key: []}}", + "GID:2, GE:DataSource_113{}, logic prop:{stats:{count 10, ColNDVs map[4:3 5:3], GroupNDVs []}, schema:{Column: [test.t2.a,test.t2.b] Key: [] Unique key: []}}", + "GID:3, GE:Join_117{GID:1, GID:2}, logic prop:{stats:{count 16.666666666666668, ColNDVs map[1:2 4:3 5:3], GroupNDVs []}, schema:{Column: [test.t1.a,test.t2.a,test.t2.b] Key: [] Unique key: []}}", + "GID:4, GE:Aggregation_115{GID:3}, logic prop:{stats:{count 3, ColNDVs map[7:3], GroupNDVs []}, schema:{Column: [Column#7] Key: [] Unique key: []}}", + "GID:5, GE:Projection_116{GID:4}, logic prop:{stats:{count 3, ColNDVs map[7:3], GroupNDVs []}, schema:{Column: [Column#7] Key: [] Unique key: []}}" ] }, { "SQL": "select count(1) from t1 right join t2 on t1.a = t2.a group by t1.a, t1.b", "Str": [ - "GID:1, logic prop:{stats:{count 5, ColNDVs map[1:2 2:2]}, schema:{Column: [test.t1.a,test.t1.b] Key: [] Unique key: []}}", - "GID:2, logic prop:{stats:{count 10, ColNDVs map[4:3]}, schema:{Column: [test.t2.a] Key: [] Unique key: []}}", - "GID:3, logic prop:{stats:{count 16.666666666666668, ColNDVs map[1:2 2:2 4:3]}, schema:{Column: [test.t1.a,test.t1.b,test.t2.a] Key: [] Unique key: []}}", - "GID:4, logic prop:{stats:{count 2, ColNDVs map[7:2]}, schema:{Column: [Column#7] Key: [] Unique key: []}}", - "GID:5, logic prop:{stats:{count 2, ColNDVs map[7:2]}, schema:{Column: [Column#7] Key: [] Unique key: []}}" + "GID:1, GE:DataSource_118{}, logic prop:{stats:{count 5, ColNDVs map[1:2 2:2], GroupNDVs []}, schema:{Column: [test.t1.a,test.t1.b] Key: [] Unique key: []}}", + "GID:2, GE:DataSource_119{}, logic prop:{stats:{count 10, ColNDVs map[4:3], GroupNDVs []}, schema:{Column: [test.t2.a] Key: [] Unique key: []}}", + "GID:3, GE:Join_123{GID:1, GID:2}, logic prop:{stats:{count 16.666666666666668, ColNDVs map[1:2 2:2 4:3], GroupNDVs []}, schema:{Column: [test.t1.a,test.t1.b,test.t2.a] Key: [] Unique key: []}}", + "GID:4, GE:Aggregation_121{GID:3}, logic prop:{stats:{count 2, ColNDVs map[7:2], GroupNDVs []}, schema:{Column: [Column#7] Key: [] Unique key: []}}", + "GID:5, GE:Projection_122{GID:4}, logic prop:{stats:{count 2, ColNDVs map[7:2], GroupNDVs []}, schema:{Column: [Column#7] Key: [] Unique key: []}}" ] }, { "SQL": "select count(1) from t1 right join t2 on t1.a = t2.a group by t2.a, t2.b", "Str": [ - "GID:1, logic prop:{stats:{count 5, ColNDVs map[1:2]}, schema:{Column: [test.t1.a] Key: [] Unique key: []}}", - "GID:2, logic prop:{stats:{count 10, ColNDVs map[4:3 5:3]}, schema:{Column: [test.t2.a,test.t2.b] Key: [] Unique key: []}}", - "GID:3, logic prop:{stats:{count 16.666666666666668, ColNDVs map[1:2 4:3 5:3]}, schema:{Column: [test.t1.a,test.t2.a,test.t2.b] Key: [] Unique key: []}}", - "GID:4, logic prop:{stats:{count 9, ColNDVs map[7:9]}, schema:{Column: [Column#7] Key: [] Unique key: []}}", - "GID:5, logic prop:{stats:{count 9, ColNDVs map[7:9]}, schema:{Column: [Column#7] Key: [] Unique key: []}}" + "GID:1, GE:DataSource_124{}, logic prop:{stats:{count 5, ColNDVs map[1:2], GroupNDVs []}, schema:{Column: [test.t1.a] Key: [] Unique key: []}}", + "GID:2, GE:DataSource_125{}, logic prop:{stats:{count 10, ColNDVs map[4:3 5:3], GroupNDVs [{[4 5] 9}]}, schema:{Column: [test.t2.a,test.t2.b] Key: [] Unique key: []}}", + "GID:3, GE:Join_129{GID:1, GID:2}, logic prop:{stats:{count 16.666666666666668, ColNDVs map[1:2 4:3 5:3], GroupNDVs [{[4 5] 9}]}, schema:{Column: [test.t1.a,test.t2.a,test.t2.b] Key: [] Unique key: []}}", + "GID:4, GE:Aggregation_127{GID:3}, logic prop:{stats:{count 9, ColNDVs map[7:9], GroupNDVs []}, schema:{Column: [Column#7] Key: [] Unique key: []}}", + "GID:5, GE:Projection_128{GID:4}, logic prop:{stats:{count 9, ColNDVs map[7:9], GroupNDVs []}, schema:{Column: [Column#7] Key: [] Unique key: []}}" ] }, { "SQL": "select count(tmp.cmp) from (select t1.a as a, t1.b as b, (t1.b in (select t2.b from t2 where t2.a > t1.a)) as cmp from t1) tmp group by tmp.a, tmp.b", "Str": [ - "GID:1, logic prop:{stats:{count 5, ColNDVs map[1:2 2:2]}, schema:{Column: [test.t1.a,test.t1.b] Key: [] Unique key: []}}", - "GID:2, logic prop:{stats:{count 10, ColNDVs map[7:3 8:3]}, schema:{Column: [test.t2.a,test.t2.b] Key: [] Unique key: []}}", - "GID:3, logic prop:{stats:{count 5, ColNDVs map[1:2 2:2 10:2]}, schema:{Column: [test.t1.a,test.t1.b,Column#10] Key: [] Unique key: []}}", - "GID:4, logic prop:{stats:{count 4, ColNDVs map[11:4]}, schema:{Column: [Column#11] Key: [] Unique key: []}}", - "GID:5, logic prop:{stats:{count 4, ColNDVs map[11:4]}, schema:{Column: [Column#11] Key: [] Unique key: []}}" + "GID:1, GE:DataSource_130{}, logic prop:{stats:{count 5, ColNDVs map[1:2 2:2], GroupNDVs [{[1 2] 4}]}, schema:{Column: [test.t1.a,test.t1.b] Key: [] Unique key: []}}", + "GID:2, GE:DataSource_133{}, logic prop:{stats:{count 10, ColNDVs map[7:3 8:3], GroupNDVs []}, schema:{Column: [test.t2.a,test.t2.b] Key: [] Unique key: []}}", + "GID:3, GE:Join_136{GID:1, GID:2}, logic prop:{stats:{count 5, ColNDVs map[1:2 2:2 10:2], GroupNDVs [{[1 2] 4}]}, schema:{Column: [test.t1.a,test.t1.b,Column#10] Key: [] Unique key: []}}", + "GID:4, GE:Aggregation_137{GID:3}, logic prop:{stats:{count 4, ColNDVs map[11:4], GroupNDVs []}, schema:{Column: [Column#11] Key: [] Unique key: []}}", + "GID:5, GE:Projection_138{GID:4}, logic prop:{stats:{count 4, ColNDVs map[11:4], GroupNDVs []}, schema:{Column: [Column#11] Key: [] Unique key: []}}" ] }, { "SQL": "select count(tmp.cmp) from (select t1.a as a, t1.b as b, (t1.b not in (select t2.b from t2 where t2.a > t1.a)) as cmp from t1) tmp group by tmp.a, tmp.b", "Str": [ - "GID:1, logic prop:{stats:{count 5, ColNDVs map[1:2 2:2]}, schema:{Column: [test.t1.a,test.t1.b] Key: [] Unique key: []}}", - "GID:2, logic prop:{stats:{count 10, ColNDVs map[7:3 8:3]}, schema:{Column: [test.t2.a,test.t2.b] Key: [] Unique key: []}}", - "GID:3, logic prop:{stats:{count 5, ColNDVs map[1:2 2:2 10:2]}, schema:{Column: [test.t1.a,test.t1.b,Column#10] Key: [] Unique key: []}}", - "GID:4, logic prop:{stats:{count 4, ColNDVs map[11:4]}, schema:{Column: [Column#11] Key: [] Unique key: []}}", - "GID:5, logic prop:{stats:{count 4, ColNDVs map[11:4]}, schema:{Column: [Column#11] Key: [] Unique key: []}}" + "GID:1, GE:DataSource_139{}, logic prop:{stats:{count 5, ColNDVs map[1:2 2:2], GroupNDVs [{[1 2] 4}]}, schema:{Column: [test.t1.a,test.t1.b] Key: [] Unique key: []}}", + "GID:2, GE:DataSource_142{}, logic prop:{stats:{count 10, ColNDVs map[7:3 8:3], GroupNDVs []}, schema:{Column: [test.t2.a,test.t2.b] Key: [] Unique key: []}}", + "GID:3, GE:Join_145{GID:1, GID:2}, logic prop:{stats:{count 5, ColNDVs map[1:2 2:2 10:2], GroupNDVs [{[1 2] 4}]}, schema:{Column: [test.t1.a,test.t1.b,Column#10] Key: [] Unique key: []}}", + "GID:4, GE:Aggregation_146{GID:3}, logic prop:{stats:{count 4, ColNDVs map[11:4], GroupNDVs []}, schema:{Column: [Column#11] Key: [] Unique key: []}}", + "GID:5, GE:Projection_147{GID:4}, logic prop:{stats:{count 4, ColNDVs map[11:4], GroupNDVs []}, schema:{Column: [Column#11] Key: [] Unique key: []}}" ] }, { "SQL": "select count(1) from (select t1.a as a, t1.b as b from t1 where t1.b in (select t2.b from t2 where t2.a > t1.a)) tmp group by tmp.a, tmp.b", "Str": [ - "GID:1, logic prop:{stats:{count 5, ColNDVs map[1:2 2:2]}, schema:{Column: [test.t1.a,test.t1.b] Key: [] Unique key: []}}", - "GID:2, logic prop:{stats:{count 10, ColNDVs map[4:3 5:3]}, schema:{Column: [test.t2.a,test.t2.b] Key: [] Unique key: []}}", - "GID:3, logic prop:{stats:{count 4, ColNDVs map[1:1.6 2:1.6]}, schema:{Column: [test.t1.a,test.t1.b] Key: [] Unique key: []}}", - "GID:4, logic prop:{stats:{count 1.6, ColNDVs map[7:1.6]}, schema:{Column: [Column#7] Key: [] Unique key: []}}", - "GID:5, logic prop:{stats:{count 1.6, ColNDVs map[7:1.6]}, schema:{Column: [Column#7] Key: [] Unique key: []}}" + "GID:1, GE:DataSource_148{}, logic prop:{stats:{count 5, ColNDVs map[1:2 2:2], GroupNDVs []}, schema:{Column: [test.t1.a,test.t1.b] Key: [] Unique key: []}}", + "GID:2, GE:DataSource_150{}, logic prop:{stats:{count 10, ColNDVs map[4:3 5:3], GroupNDVs []}, schema:{Column: [test.t2.a,test.t2.b] Key: [] Unique key: []}}", + "GID:3, GE:Join_153{GID:1, GID:2}, logic prop:{stats:{count 4, ColNDVs map[1:1.6 2:1.6], GroupNDVs []}, schema:{Column: [test.t1.a,test.t1.b] Key: [] Unique key: []}}", + "GID:4, GE:Aggregation_155{GID:3}, logic prop:{stats:{count 1.6, ColNDVs map[7:1.6], GroupNDVs []}, schema:{Column: [Column#7] Key: [] Unique key: []}}", + "GID:5, GE:Projection_156{GID:4}, logic prop:{stats:{count 1.6, ColNDVs map[7:1.6], GroupNDVs []}, schema:{Column: [Column#7] Key: [] Unique key: []}}" ] }, { "SQL": "select count(1) from (select t1.a as a, t1.b as b from t1 where t1.b not in (select t2.b from t2 where t2.a > t1.a)) tmp group by tmp.a, tmp.b", "Str": [ - "GID:1, logic prop:{stats:{count 5, ColNDVs map[1:2 2:2]}, schema:{Column: [test.t1.a,test.t1.b] Key: [] Unique key: []}}", - "GID:2, logic prop:{stats:{count 10, ColNDVs map[4:3 5:3]}, schema:{Column: [test.t2.a,test.t2.b] Key: [] Unique key: []}}", - "GID:3, logic prop:{stats:{count 4, ColNDVs map[1:1.6 2:1.6]}, schema:{Column: [test.t1.a,test.t1.b] Key: [] Unique key: []}}", - "GID:4, logic prop:{stats:{count 1.6, ColNDVs map[7:1.6]}, schema:{Column: [Column#7] Key: [] Unique key: []}}", - "GID:5, logic prop:{stats:{count 1.6, ColNDVs map[7:1.6]}, schema:{Column: [Column#7] Key: [] Unique key: []}}" + "GID:1, GE:DataSource_157{}, logic prop:{stats:{count 5, ColNDVs map[1:2 2:2], GroupNDVs []}, schema:{Column: [test.t1.a,test.t1.b] Key: [] Unique key: []}}", + "GID:2, GE:DataSource_159{}, logic prop:{stats:{count 10, ColNDVs map[4:3 5:3], GroupNDVs []}, schema:{Column: [test.t2.a,test.t2.b] Key: [] Unique key: []}}", + "GID:3, GE:Join_162{GID:1, GID:2}, logic prop:{stats:{count 4, ColNDVs map[1:1.6 2:1.6], GroupNDVs []}, schema:{Column: [test.t1.a,test.t1.b] Key: [] Unique key: []}}", + "GID:4, GE:Aggregation_164{GID:3}, logic prop:{stats:{count 1.6, ColNDVs map[7:1.6], GroupNDVs []}, schema:{Column: [Column#7] Key: [] Unique key: []}}", + "GID:5, GE:Projection_165{GID:4}, logic prop:{stats:{count 1.6, ColNDVs map[7:1.6], GroupNDVs []}, schema:{Column: [Column#7] Key: [] Unique key: []}}" ] }, { "SQL": "select * from t1 left join (select t2.a as a, t2.b as b, count(1) as cnt from t2 group by t2.a, t2.b) as tmp on t1.a = tmp.a and t1.b = tmp.b", "Str": [ - "GID:1, logic prop:{stats:{count 5, ColNDVs map[1:2 2:2]}, schema:{Column: [test.t1.a,test.t1.b] Key: [] Unique key: []}}", - "GID:2, logic prop:{stats:{count 10, ColNDVs map[4:3 5:3]}, schema:{Column: [test.t2.a,test.t2.b] Key: [] Unique key: []}}", - "GID:3, logic prop:{stats:{count 9, ColNDVs map[4:9 5:9 7:9]}, schema:{Column: [Column#7,test.t2.a,test.t2.b] Key: [[test.t2.a,test.t2.b]] Unique key: []}}", - "GID:4, logic prop:{stats:{count 5, ColNDVs map[1:2 2:2 4:5 5:5 7:5]}, schema:{Column: [test.t1.a,test.t1.b,Column#7,test.t2.a,test.t2.b] Key: [] Unique key: []}}", - "GID:5, logic prop:{stats:{count 5, ColNDVs map[1:2 2:2 4:5 5:5 7:5]}, schema:{Column: [test.t1.a,test.t1.b,test.t2.a,test.t2.b,Column#7] Key: [] Unique key: []}}" + "GID:1, GE:DataSource_166{}, logic prop:{stats:{count 5, ColNDVs map[1:2 2:2], GroupNDVs [{[1 2] 4}]}, schema:{Column: [test.t1.a,test.t1.b] Key: [] Unique key: []}}", + "GID:2, GE:DataSource_167{}, logic prop:{stats:{count 10, ColNDVs map[4:3 5:3], GroupNDVs [{[4 5] 9}]}, schema:{Column: [test.t2.a,test.t2.b] Key: [] Unique key: []}}", + "GID:3, GE:Aggregation_168{GID:2}, logic prop:{stats:{count 9, ColNDVs map[4:9 5:9 7:9], GroupNDVs [{[4 5] 9}]}, schema:{Column: [Column#7,test.t2.a,test.t2.b] Key: [[test.t2.a,test.t2.b]] Unique key: []}}", + "GID:4, GE:Join_172{GID:1, GID:3}, logic prop:{stats:{count 5, ColNDVs map[1:2 2:2 4:5 5:5 7:5], GroupNDVs []}, schema:{Column: [test.t1.a,test.t1.b,Column#7,test.t2.a,test.t2.b] Key: [] Unique key: []}}", + "GID:5, GE:Projection_171{GID:4}, logic prop:{stats:{count 5, ColNDVs map[1:2 2:2 4:5 5:5 7:5], GroupNDVs []}, schema:{Column: [test.t1.a,test.t1.b,test.t2.a,test.t2.b,Column#7] Key: [] Unique key: []}}" ] }, { "SQL": "select count(1) from (select t1.a as a, t1.b as b from t1 limit 3) tmp group by tmp.a, tmp.b", "Str": [ - "GID:1, logic prop:{stats:{count 5, ColNDVs map[1:2 2:2]}, schema:{Column: [test.t1.a,test.t1.b] Key: [] Unique key: []}}", - "GID:2, logic prop:{stats:{count 3, ColNDVs map[1:2 2:2]}, schema:{Column: [test.t1.a,test.t1.b] Key: [] Unique key: []}}", - "GID:3, logic prop:{stats:{count 2, ColNDVs map[4:2]}, schema:{Column: [Column#4] Key: [] Unique key: []}}", - "GID:4, logic prop:{stats:{count 2, ColNDVs map[4:2]}, schema:{Column: [Column#4] Key: [] Unique key: []}}" + "GID:1, GE:DataSource_173{}, logic prop:{stats:{count 5, ColNDVs map[1:2 2:2], GroupNDVs []}, schema:{Column: [test.t1.a,test.t1.b] Key: [] Unique key: []}}", + "GID:2, GE:Limit_179{GID:1}, logic prop:{stats:{count 3, ColNDVs map[1:2 2:2], GroupNDVs []}, schema:{Column: [test.t1.a,test.t1.b] Key: [] Unique key: []}}", + "GID:3, GE:Aggregation_176{GID:2}, logic prop:{stats:{count 2, ColNDVs map[4:2], GroupNDVs []}, schema:{Column: [Column#4] Key: [] Unique key: []}}", + "GID:4, GE:Projection_177{GID:3}, logic prop:{stats:{count 2, ColNDVs map[4:2], GroupNDVs []}, schema:{Column: [Column#4] Key: [] Unique key: []}}" ] }, { "SQL": "select count(tmp.a_sum) from (select t1.a as a, t1.b as b, sum(a) over() as a_sum from t1) tmp group by tmp.a, tmp.b", "Str": [ - "GID:1, logic prop:{stats:{count 5, ColNDVs map[1:2 2:2]}, schema:{Column: [test.t1.a,test.t1.b] Key: [] Unique key: []}}", - "GID:2, logic prop:{stats:{count 5, ColNDVs map[1:2 2:2 5:5]}, schema:{Column: [test.t1.a,test.t1.b,Column#5] Key: [] Unique key: []}}", - "GID:3, logic prop:{stats:{count 4, ColNDVs map[6:4]}, schema:{Column: [Column#6] Key: [] Unique key: []}}", - "GID:4, logic prop:{stats:{count 4, ColNDVs map[6:4]}, schema:{Column: [Column#6] Key: [] Unique key: []}}" + "GID:1, GE:DataSource_180{}, logic prop:{stats:{count 5, ColNDVs map[1:2 2:2], GroupNDVs [{[1 2] 4}]}, schema:{Column: [test.t1.a,test.t1.b] Key: [] Unique key: []}}", + "GID:2, GE:Window_183{GID:1}, logic prop:{stats:{count 5, ColNDVs map[1:2 2:2 5:5], GroupNDVs [{[1 2] 4}]}, schema:{Column: [test.t1.a,test.t1.b,Column#5] Key: [] Unique key: []}}", + "GID:3, GE:Aggregation_185{GID:2}, logic prop:{stats:{count 4, ColNDVs map[6:4], GroupNDVs []}, schema:{Column: [Column#6] Key: [] Unique key: []}}", + "GID:4, GE:Projection_186{GID:3}, logic prop:{stats:{count 4, ColNDVs map[6:4], GroupNDVs []}, schema:{Column: [Column#6] Key: [] Unique key: []}}" ] } ] From f68716a07c7b68bb64f2d41dce0452d41db548e2 Mon Sep 17 00:00:00 2001 From: arenatlx <314806019@qq.com> Date: Fri, 6 Dec 2024 17:11:25 +0800 Subject: [PATCH 10/25] . Signed-off-by: arenatlx <314806019@qq.com> --- pkg/planner/cascades/memo/group.go | 2 +- pkg/planner/cascades/memo/memo.go | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/pkg/planner/cascades/memo/group.go b/pkg/planner/cascades/memo/group.go index 37f0f6301a2a5..e9ad838b5df45 100644 --- a/pkg/planner/cascades/memo/group.go +++ b/pkg/planner/cascades/memo/group.go @@ -154,7 +154,7 @@ func (g *Group) String(w util.StrBufferWriter) { // ForEachGE traverse the inside group expression with f call on them each. func (g *Group) ForEachGE(f func(ge *GroupExpression) bool) { - next := true + var next bool for elem := g.logicalExpressions.Front(); elem != nil; elem = elem.Next() { expr := elem.Value.(*GroupExpression) next = f(expr) diff --git a/pkg/planner/cascades/memo/memo.go b/pkg/planner/cascades/memo/memo.go index a09822a447039..c5c51ab3ffedd 100644 --- a/pkg/planner/cascades/memo/memo.go +++ b/pkg/planner/cascades/memo/memo.go @@ -164,7 +164,7 @@ func (mm *Memo) Init(plan base.LogicalPlan) *GroupExpression { // ForEachGroup traverse the inside group expression with f call on them each. func (mm *Memo) ForEachGroup(f func(g *Group) bool) { - next := true + var next bool for elem := mm.GetGroups().Front(); elem != nil; elem = elem.Next() { expr := elem.Value.(*Group) next = f(expr) From 6b390fa2c9c3aa4c2a3303d591579bbd8a671877 Mon Sep 17 00:00:00 2001 From: arenatlx <314806019@qq.com> Date: Fri, 6 Dec 2024 17:55:58 +0800 Subject: [PATCH 11/25] . Signed-off-by: arenatlx <314806019@qq.com> --- pkg/planner/cascades/memo/memo_test.go | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/pkg/planner/cascades/memo/memo_test.go b/pkg/planner/cascades/memo/memo_test.go index e9d2313cf070e..ee67b5c4f3600 100644 --- a/pkg/planner/cascades/memo/memo_test.go +++ b/pkg/planner/cascades/memo/memo_test.go @@ -17,13 +17,13 @@ package memo import ( "testing" - plannercore "github.com/pingcap/tidb/pkg/planner/core" "github.com/pingcap/tidb/pkg/planner/core/operator/logicalop" + "github.com/pingcap/tidb/pkg/util/mock" "github.com/stretchr/testify/require" ) func TestMemo(t *testing.T) { - ctx := plannercore.MockContext() + ctx := mock.NewContext() ctx.GetSessionVars().MockPlan = true t1 := logicalop.DataSource{}.Init(ctx, 0) t2 := logicalop.DataSource{}.Init(ctx, 0) @@ -46,7 +46,7 @@ func TestMemo(t *testing.T) { } func TestInsertGE(t *testing.T) { - ctx := plannercore.MockContext() + ctx := mock.NewContext() ctx.GetSessionVars().MockPlan = true t1 := logicalop.DataSource{}.Init(ctx, 0) t2 := logicalop.DataSource{}.Init(ctx, 0) From 412568156097b8bb6c8633c1a4464fbb7723b206 Mon Sep 17 00:00:00 2001 From: arenatlx <314806019@qq.com> Date: Fri, 6 Dec 2024 17:59:02 +0800 Subject: [PATCH 12/25] . Signed-off-by: arenatlx <314806019@qq.com> --- pkg/planner/cascades/memo/BUILD.bazel | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pkg/planner/cascades/memo/BUILD.bazel b/pkg/planner/cascades/memo/BUILD.bazel index c2722b9d235d0..f2ae5633c8815 100644 --- a/pkg/planner/cascades/memo/BUILD.bazel +++ b/pkg/planner/cascades/memo/BUILD.bazel @@ -36,9 +36,9 @@ go_test( deps = [ "//pkg/expression", "//pkg/planner/cascades/base", - "//pkg/planner/core", "//pkg/planner/core/operator/logicalop", "//pkg/testkit/testsetup", + "//pkg/util/mock", "@com_github_stretchr_testify//require", "@org_uber_go_goleak//:goleak", ], From c3fca985c690ad319ad3939bd8bf3c456cc40ef2 Mon Sep 17 00:00:00 2001 From: arenatlx <314806019@qq.com> Date: Tue, 10 Dec 2024 12:34:25 +0800 Subject: [PATCH 13/25] . Signed-off-by: arenatlx <314806019@qq.com> --- pkg/planner/core/operator/baseimpl/plan.go | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/pkg/planner/core/operator/baseimpl/plan.go b/pkg/planner/core/operator/baseimpl/plan.go index 49dd28fb811dc..44c4c2ffe705b 100644 --- a/pkg/planner/core/operator/baseimpl/plan.go +++ b/pkg/planner/core/operator/baseimpl/plan.go @@ -40,12 +40,14 @@ type Plan struct { // NewBasePlan creates a new base plan. func NewBasePlan(ctx planctx.PlanContext, tp string, qbBlock int) Plan { - delta := int32(1) + var delta int32 if ctx.GetSessionVars().MockPlan { delta = int32(-1) intest.Assert(ctx.GetSessionVars().PlanID.Load() <= 0) + } else { + delta = int32(1) + intest.Assert(ctx.GetSessionVars().PlanID.Load() >= 0) } - intest.Assert(ctx.GetSessionVars().PlanID.Load() >= 0) id := ctx.GetSessionVars().PlanID.Add(delta) return Plan{ tp: tp, From 0c555178b0fadd5e940b02c1aaae7150d89a9961 Mon Sep 17 00:00:00 2001 From: arenatlx <314806019@qq.com> Date: Tue, 10 Dec 2024 13:07:24 +0800 Subject: [PATCH 14/25] . Signed-off-by: arenatlx <314806019@qq.com> --- pkg/planner/cascades/rule/binder_test.go | 1 + 1 file changed, 1 insertion(+) diff --git a/pkg/planner/cascades/rule/binder_test.go b/pkg/planner/cascades/rule/binder_test.go index 6d3349787aaba..56a6885c2ec27 100644 --- a/pkg/planner/cascades/rule/binder_test.go +++ b/pkg/planner/cascades/rule/binder_test.go @@ -388,6 +388,7 @@ func TestBinderAny(t *testing.T) { func TestBinderMultiAny(t *testing.T) { ctx := mock.NewContext() + ctx.GetSessionVars().MockPlan = true asT1 := pmodel.NewCIStr("t1") asT2 := pmodel.NewCIStr("t2") t1 := logicalop.DataSource{TableAsName: &asT1}.Init(ctx, 0) From 2418df4e2cc8f3f5671e442e6a069d11c3045730 Mon Sep 17 00:00:00 2001 From: arenatlx <314806019@qq.com> Date: Tue, 10 Dec 2024 13:16:56 +0800 Subject: [PATCH 15/25] . Signed-off-by: arenatlx <314806019@qq.com> --- pkg/planner/cascades/memo/group.go | 2 ++ pkg/planner/cascades/memo/group_expr.go | 1 - 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/pkg/planner/cascades/memo/group.go b/pkg/planner/cascades/memo/group.go index e9ad838b5df45..eba930bcf26e1 100644 --- a/pkg/planner/cascades/memo/group.go +++ b/pkg/planner/cascades/memo/group.go @@ -23,6 +23,7 @@ import ( "github.com/pingcap/tidb/pkg/planner/cascades/pattern" "github.com/pingcap/tidb/pkg/planner/cascades/util" "github.com/pingcap/tidb/pkg/planner/property" + "github.com/pingcap/tidb/pkg/util/intest" ) var _ base.HashEquals = &Group{} @@ -129,6 +130,7 @@ func (g *Group) GetFirstElem(operand pattern.Operand) *list.Element { // GetLogicalProperty return this group's logical property. func (g *Group) GetLogicalProperty() *property.LogicalProperty { + intest.Assert(g.logicalProp != nil) return g.logicalProp } diff --git a/pkg/planner/cascades/memo/group_expr.go b/pkg/planner/cascades/memo/group_expr.go index bcb457da8bde2..8b46554456168 100644 --- a/pkg/planner/cascades/memo/group_expr.go +++ b/pkg/planner/cascades/memo/group_expr.go @@ -137,7 +137,6 @@ func (e *GroupExpression) DeriveLogicalProp() (err error) { childSchema := make([]*expression.Schema, 0, len(e.Inputs)) for _, childG := range e.Inputs { childGProp := childG.GetLogicalProperty() - intest.Assert(childGProp != nil) childStats = append(childStats, childGProp.Stats) childSchema = append(childSchema, childGProp.Schema) } From 3714055db6b0203b06f36e7a62b20dfe3fada193 Mon Sep 17 00:00:00 2001 From: arenatlx <314806019@qq.com> Date: Tue, 10 Dec 2024 14:55:35 +0800 Subject: [PATCH 16/25] . Signed-off-by: arenatlx <314806019@qq.com> --- pkg/planner/cascades/memo/group.go | 5 +++++ pkg/planner/cascades/memo/group_expr.go | 2 +- 2 files changed, 6 insertions(+), 1 deletion(-) diff --git a/pkg/planner/cascades/memo/group.go b/pkg/planner/cascades/memo/group.go index eba930bcf26e1..ac115861e7f15 100644 --- a/pkg/planner/cascades/memo/group.go +++ b/pkg/planner/cascades/memo/group.go @@ -128,6 +128,11 @@ func (g *Group) GetFirstElem(operand pattern.Operand) *list.Element { return g.Operand2FirstExpr[operand] } +// HasLogicalProperty check whether current group has the logical property. +func (g *Group) HasLogicalProperty() bool { + return g.logicalProp != nil +} + // GetLogicalProperty return this group's logical property. func (g *Group) GetLogicalProperty() *property.LogicalProperty { intest.Assert(g.logicalProp != nil) diff --git a/pkg/planner/cascades/memo/group_expr.go b/pkg/planner/cascades/memo/group_expr.go index 8b46554456168..a7069d145b8fc 100644 --- a/pkg/planner/cascades/memo/group_expr.go +++ b/pkg/planner/cascades/memo/group_expr.go @@ -130,7 +130,7 @@ func (e *GroupExpression) Init(h base2.Hasher) { // init new group from bottom-up, so we can sure that this new group's children // has already gotten its logical prop. func (e *GroupExpression) DeriveLogicalProp() (err error) { - if e.GetGroup().GetLogicalProperty() != nil { + if e.GetGroup().HasLogicalProperty() { return nil } childStats := make([]*property.StatsInfo, 0, len(e.Inputs)) From 4cbc1e749e2f6c1734ca9ea15b9a43e9481c61bc Mon Sep 17 00:00:00 2001 From: arenatlx <314806019@qq.com> Date: Tue, 10 Dec 2024 15:38:31 +0800 Subject: [PATCH 17/25] fix old test Signed-off-by: arenatlx <314806019@qq.com> --- pkg/planner/core/exhaust_physical_plans_test.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pkg/planner/core/exhaust_physical_plans_test.go b/pkg/planner/core/exhaust_physical_plans_test.go index e66a046dc4633..d304d0ca72e68 100644 --- a/pkg/planner/core/exhaust_physical_plans_test.go +++ b/pkg/planner/core/exhaust_physical_plans_test.go @@ -64,7 +64,7 @@ func prepareForAnalyzeLookUpFilters() *indexJoinContext { do := domain.GetDomain(ctx) do.StatsHandle().Close() }() - ctx.GetSessionVars().PlanID.Store(-1) + ctx.GetSessionVars().PlanID.Store(0) joinNode := logicalop.LogicalJoin{}.Init(ctx.GetPlanCtx(), 0) dataSourceNode := logicalop.DataSource{}.Init(ctx.GetPlanCtx(), 0) dsSchema := expression.NewSchema() From 1590021e27f102d42c1b95152c001d9694f83957 Mon Sep 17 00:00:00 2001 From: arenatlx <314806019@qq.com> Date: Tue, 10 Dec 2024 15:41:57 +0800 Subject: [PATCH 18/25] eliminate the unnecessary tmp variable Signed-off-by: arenatlx <314806019@qq.com> --- pkg/planner/core/operator/baseimpl/plan.go | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/pkg/planner/core/operator/baseimpl/plan.go b/pkg/planner/core/operator/baseimpl/plan.go index 44c4c2ffe705b..6d5af8abdfe59 100644 --- a/pkg/planner/core/operator/baseimpl/plan.go +++ b/pkg/planner/core/operator/baseimpl/plan.go @@ -40,15 +40,14 @@ type Plan struct { // NewBasePlan creates a new base plan. func NewBasePlan(ctx planctx.PlanContext, tp string, qbBlock int) Plan { - var delta int32 + var id int32 if ctx.GetSessionVars().MockPlan { - delta = int32(-1) intest.Assert(ctx.GetSessionVars().PlanID.Load() <= 0) + id = ctx.GetSessionVars().PlanID.Add(-1) } else { - delta = int32(1) intest.Assert(ctx.GetSessionVars().PlanID.Load() >= 0) + id = ctx.GetSessionVars().PlanID.Add(1) } - id := ctx.GetSessionVars().PlanID.Add(delta) return Plan{ tp: tp, id: int(id), From e4d594f070af4f6c906a3720a57f771674560d35 Mon Sep 17 00:00:00 2001 From: arenatlx <314806019@qq.com> Date: Tue, 10 Dec 2024 16:38:01 +0800 Subject: [PATCH 19/25] . Signed-off-by: arenatlx <314806019@qq.com> --- pkg/planner/core/rule_join_reorder_dp_test.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pkg/planner/core/rule_join_reorder_dp_test.go b/pkg/planner/core/rule_join_reorder_dp_test.go index f89c34a8654cb..faba962c9c1d8 100644 --- a/pkg/planner/core/rule_join_reorder_dp_test.go +++ b/pkg/planner/core/rule_join_reorder_dp_test.go @@ -217,7 +217,7 @@ func TestDPReorderAllCartesian(t *testing.T) { defer func() { domain.GetDomain(ctx).StatsHandle().Close() }() - ctx.GetSessionVars().PlanID.Store(-1) + ctx.GetSessionVars().PlanID.Store(0) joinGroup := make([]base.LogicalPlan, 0, 4) joinGroup = append(joinGroup, newDataSource(ctx, "a", 100)) From 59105968c7e5d7e1e696d2e970f0c8da4d307c86 Mon Sep 17 00:00:00 2001 From: arenatlx <314806019@qq.com> Date: Tue, 10 Dec 2024 17:02:29 +0800 Subject: [PATCH 20/25] . Signed-off-by: arenatlx <314806019@qq.com> --- pkg/planner/core/rule_join_reorder_dp_test.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pkg/planner/core/rule_join_reorder_dp_test.go b/pkg/planner/core/rule_join_reorder_dp_test.go index faba962c9c1d8..cae96dec95ae8 100644 --- a/pkg/planner/core/rule_join_reorder_dp_test.go +++ b/pkg/planner/core/rule_join_reorder_dp_test.go @@ -169,7 +169,7 @@ func TestDPReorderTPCHQ5(t *testing.T) { do := domain.GetDomain(ctx) do.StatsHandle().Close() }() - ctx.GetSessionVars().PlanID.Store(-1) + ctx.GetSessionVars().PlanID.Store(0) joinGroups := make([]base.LogicalPlan, 0, 6) joinGroups = append(joinGroups, newDataSource(ctx, "lineitem", 59986052)) joinGroups = append(joinGroups, newDataSource(ctx, "orders", 15000000)) From 32d2325aee494bfa09100f3e3a1bc0db57ea982b Mon Sep 17 00:00:00 2001 From: arenatlx <314806019@qq.com> Date: Tue, 10 Dec 2024 17:18:41 +0800 Subject: [PATCH 21/25] . Signed-off-by: arenatlx <314806019@qq.com> --- pkg/planner/core/rule_join_reorder_dp_test.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/pkg/planner/core/rule_join_reorder_dp_test.go b/pkg/planner/core/rule_join_reorder_dp_test.go index cae96dec95ae8..4b7e33f123f68 100644 --- a/pkg/planner/core/rule_join_reorder_dp_test.go +++ b/pkg/planner/core/rule_join_reorder_dp_test.go @@ -58,12 +58,12 @@ func newMockJoin(ctx base.PlanContext, statsMap map[int]*property.StatsInfo) fun if mj, ok := lChild.(*mockLogicalJoin); ok { retJoin.involvedNodeSet = mj.involvedNodeSet } else { - retJoin.involvedNodeSet = 1 << uint(lChild.ID()) + retJoin.involvedNodeSet = 1 << uint(lChild.ID()-1) } if mj, ok := rChild.(*mockLogicalJoin); ok { retJoin.involvedNodeSet |= mj.involvedNodeSet } else { - retJoin.involvedNodeSet |= 1 << uint(rChild.ID()) + retJoin.involvedNodeSet |= 1 << uint(rChild.ID()-1) } retJoin.SetChildren(lChild, rChild) retJoin.JoinType = joinType From fab10f095ee5b319e8a55073ab759db16d956017 Mon Sep 17 00:00:00 2001 From: arenatlx <314806019@qq.com> Date: Wed, 11 Dec 2024 13:59:20 +0800 Subject: [PATCH 22/25] . Signed-off-by: arenatlx <314806019@qq.com> --- pkg/planner/cascades/memo/group.go | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/pkg/planner/cascades/memo/group.go b/pkg/planner/cascades/memo/group.go index ac115861e7f15..556f1bf1ebf0e 100644 --- a/pkg/planner/cascades/memo/group.go +++ b/pkg/planner/cascades/memo/group.go @@ -17,8 +17,6 @@ package memo import ( "container/list" "fmt" - "strconv" - "github.com/pingcap/tidb/pkg/planner/cascades/base" "github.com/pingcap/tidb/pkg/planner/cascades/pattern" "github.com/pingcap/tidb/pkg/planner/cascades/util" @@ -156,7 +154,7 @@ func (g *Group) SetExplored() { // String implements fmt.Stringer interface. func (g *Group) String(w util.StrBufferWriter) { - w.WriteString(fmt.Sprintf("GID:%s", strconv.Itoa(int(g.groupID)))) + w.WriteString(fmt.Sprintf("GID:%d", int(g.groupID))) } // ForEachGE traverse the inside group expression with f call on them each. From 6316c2bd3c0b906d48276257840156a06e7b7dc0 Mon Sep 17 00:00:00 2001 From: arenatlx <314806019@qq.com> Date: Wed, 11 Dec 2024 14:37:23 +0800 Subject: [PATCH 23/25] . Signed-off-by: arenatlx <314806019@qq.com> --- pkg/planner/cascades/memo/group.go | 1 + 1 file changed, 1 insertion(+) diff --git a/pkg/planner/cascades/memo/group.go b/pkg/planner/cascades/memo/group.go index 556f1bf1ebf0e..c9a7f4a3286d1 100644 --- a/pkg/planner/cascades/memo/group.go +++ b/pkg/planner/cascades/memo/group.go @@ -17,6 +17,7 @@ package memo import ( "container/list" "fmt" + "github.com/pingcap/tidb/pkg/planner/cascades/base" "github.com/pingcap/tidb/pkg/planner/cascades/pattern" "github.com/pingcap/tidb/pkg/planner/cascades/util" From 365588a777e9a77e27fb54153ce4f58183c67516 Mon Sep 17 00:00:00 2001 From: arenatlx <314806019@qq.com> Date: Wed, 11 Dec 2024 17:29:54 +0800 Subject: [PATCH 24/25] intro failpoint Signed-off-by: arenatlx <314806019@qq.com> --- pkg/planner/cascades/memo/BUILD.bazel | 2 + pkg/planner/cascades/memo/group_expr.go | 17 +++-- pkg/planner/cascades/memo/memo_test.go | 11 ++- pkg/planner/cascades/rule/BUILD.bazel | 1 + pkg/planner/cascades/rule/binder_test.go | 67 +++++++++++++------ .../core/operator/baseimpl/BUILD.bazel | 1 - pkg/planner/core/operator/baseimpl/plan.go | 10 +-- 7 files changed, 71 insertions(+), 38 deletions(-) diff --git a/pkg/planner/cascades/memo/BUILD.bazel b/pkg/planner/cascades/memo/BUILD.bazel index f2ae5633c8815..900fa9a0c386a 100644 --- a/pkg/planner/cascades/memo/BUILD.bazel +++ b/pkg/planner/cascades/memo/BUILD.bazel @@ -18,6 +18,7 @@ go_library( "//pkg/planner/core/base", "//pkg/planner/property", "//pkg/util/intest", + "@com_github_pingcap_failpoint//:failpoint", ], ) @@ -39,6 +40,7 @@ go_test( "//pkg/planner/core/operator/logicalop", "//pkg/testkit/testsetup", "//pkg/util/mock", + "@com_github_pingcap_failpoint//:failpoint", "@com_github_stretchr_testify//require", "@org_uber_go_goleak//:goleak", ], diff --git a/pkg/planner/cascades/memo/group_expr.go b/pkg/planner/cascades/memo/group_expr.go index a7069d145b8fc..5135bdc4c53c1 100644 --- a/pkg/planner/cascades/memo/group_expr.go +++ b/pkg/planner/cascades/memo/group_expr.go @@ -15,6 +15,7 @@ package memo import ( + "github.com/pingcap/failpoint" "github.com/pingcap/tidb/pkg/expression" base2 "github.com/pingcap/tidb/pkg/planner/cascades/base" "github.com/pingcap/tidb/pkg/planner/cascades/pattern" @@ -148,11 +149,17 @@ func (e *GroupExpression) DeriveLogicalProp() (err error) { tmpStats := e.LogicalPlan.StatsInfo() // only for those new created logical op from XForm, we should rebuild their stats; // in memo init phase, all logical ops has maintained their stats already, just use them. - if tmpStats == nil && e.LogicalPlan.ID() > 0 { - // here can only derive the basic stats from bottom up, we can't pass any colGroups required by parents. - tmpStats, err = e.LogicalPlan.DeriveStats(childStats, tmpSchema, childSchema, nil) - if err != nil { - return err + if tmpStats == nil { + skipDeriveStats := false + failpoint.Inject("MockPlanSkipMemoDeriveStats", func(val failpoint.Value) { + skipDeriveStats = val.(bool) + }) + if !skipDeriveStats { + // here can only derive the basic stats from bottom up, we can't pass any colGroups required by parents. + tmpStats, err = e.LogicalPlan.DeriveStats(childStats, tmpSchema, childSchema, nil) + if err != nil { + return err + } } } e.GetGroup().GetLogicalProperty().Schema = tmpSchema diff --git a/pkg/planner/cascades/memo/memo_test.go b/pkg/planner/cascades/memo/memo_test.go index ee67b5c4f3600..29b1416a4a09e 100644 --- a/pkg/planner/cascades/memo/memo_test.go +++ b/pkg/planner/cascades/memo/memo_test.go @@ -17,14 +17,18 @@ package memo import ( "testing" + "github.com/pingcap/failpoint" "github.com/pingcap/tidb/pkg/planner/core/operator/logicalop" "github.com/pingcap/tidb/pkg/util/mock" "github.com/stretchr/testify/require" ) func TestMemo(t *testing.T) { + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/pkg/planner/cascades/memo/MockPlanSkipMemoDeriveStats", `return(true)`)) + defer func() { + require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/pkg/planner/cascades/memo/MockPlanSkipMemoDeriveStats")) + }() ctx := mock.NewContext() - ctx.GetSessionVars().MockPlan = true t1 := logicalop.DataSource{}.Init(ctx, 0) t2 := logicalop.DataSource{}.Init(ctx, 0) join := logicalop.LogicalJoin{}.Init(ctx, 0) @@ -46,8 +50,11 @@ func TestMemo(t *testing.T) { } func TestInsertGE(t *testing.T) { + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/pkg/planner/cascades/memo/MockPlanSkipMemoDeriveStats", `return(true)`)) + defer func() { + require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/pkg/planner/cascades/memo/MockPlanSkipMemoDeriveStats")) + }() ctx := mock.NewContext() - ctx.GetSessionVars().MockPlan = true t1 := logicalop.DataSource{}.Init(ctx, 0) t2 := logicalop.DataSource{}.Init(ctx, 0) join := logicalop.LogicalJoin{}.Init(ctx, 0) diff --git a/pkg/planner/cascades/rule/BUILD.bazel b/pkg/planner/cascades/rule/BUILD.bazel index 9937e04420944..3b03895534728 100644 --- a/pkg/planner/cascades/rule/BUILD.bazel +++ b/pkg/planner/cascades/rule/BUILD.bazel @@ -31,6 +31,7 @@ go_test( "//pkg/planner/cascades/util", "//pkg/planner/core/operator/logicalop", "//pkg/util/mock", + "@com_github_pingcap_failpoint//:failpoint", "@com_github_stretchr_testify//require", ], ) diff --git a/pkg/planner/cascades/rule/binder_test.go b/pkg/planner/cascades/rule/binder_test.go index 56a6885c2ec27..49fbac0f4d9b7 100644 --- a/pkg/planner/cascades/rule/binder_test.go +++ b/pkg/planner/cascades/rule/binder_test.go @@ -18,6 +18,7 @@ import ( "bytes" "testing" + "github.com/pingcap/failpoint" pmodel "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/planner/cascades/memo" "github.com/pingcap/tidb/pkg/planner/cascades/pattern" @@ -28,8 +29,11 @@ import ( ) func TestBinderSuccess(t *testing.T) { + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/pkg/planner/cascades/memo/MockPlanSkipMemoDeriveStats", `return(true)`)) + defer func() { + require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/pkg/planner/cascades/memo/MockPlanSkipMemoDeriveStats")) + }() ctx := mock.NewContext() - ctx.GetSessionVars().MockPlan = true t1 := logicalop.DataSource{}.Init(ctx, 0) t2 := logicalop.DataSource{}.Init(ctx, 0) join := logicalop.LogicalJoin{}.Init(ctx, 0) @@ -63,8 +67,11 @@ func TestBinderSuccess(t *testing.T) { } func TestBinderFail(t *testing.T) { + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/pkg/planner/cascades/memo/MockPlanSkipMemoDeriveStats", `return(true)`)) + defer func() { + require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/pkg/planner/cascades/memo/MockPlanSkipMemoDeriveStats")) + }() ctx := mock.NewContext() - ctx.GetSessionVars().MockPlan = true t1 := logicalop.DataSource{}.Init(ctx, 0) t2 := logicalop.DataSource{}.Init(ctx, 0) join := logicalop.LogicalJoin{}.Init(ctx, 0) @@ -88,7 +95,7 @@ func TestBinderFail(t *testing.T) { holder := binder.Next() require.Nil(t, holder) buf.Flush() - require.Equal(t, b.String(), "GE:DataSource_-1{}\n") + require.Equal(t, b.String(), "GE:DataSource_1{}\n") s1 := logicalop.LogicalLimit{}.Init(ctx, 0) s1.SetChildren(t1) @@ -119,12 +126,15 @@ func TestBinderFail(t *testing.T) { holder = binder.Next() require.Nil(t, holder) buf.Flush() - require.Equal(t, b.String(), "GE:Limit_-4{GID:1}\n") + require.Equal(t, b.String(), "GE:Limit_4{GID:1}\n") } func TestBinderTopNode(t *testing.T) { + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/pkg/planner/cascades/memo/MockPlanSkipMemoDeriveStats", `return(true)`)) + defer func() { + require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/pkg/planner/cascades/memo/MockPlanSkipMemoDeriveStats")) + }() ctx := mock.NewContext() - ctx.GetSessionVars().MockPlan = true t1 := logicalop.DataSource{}.Init(ctx, 0) t2 := logicalop.DataSource{}.Init(ctx, 0) join := logicalop.LogicalJoin{}.Init(ctx, 0) @@ -144,8 +154,11 @@ func TestBinderTopNode(t *testing.T) { } func TestBinderOneNode(t *testing.T) { + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/pkg/planner/cascades/memo/MockPlanSkipMemoDeriveStats", `return(true)`)) + defer func() { + require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/pkg/planner/cascades/memo/MockPlanSkipMemoDeriveStats")) + }() ctx := mock.NewContext() - ctx.GetSessionVars().MockPlan = true join := logicalop.LogicalJoin{}.Init(ctx, 0) mm := memo.NewMemo() @@ -161,8 +174,11 @@ func TestBinderOneNode(t *testing.T) { } func TestBinderSubTreeMatch(t *testing.T) { + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/pkg/planner/cascades/memo/MockPlanSkipMemoDeriveStats", `return(true)`)) + defer func() { + require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/pkg/planner/cascades/memo/MockPlanSkipMemoDeriveStats")) + }() ctx := mock.NewContext() - ctx.GetSessionVars().MockPlan = true t1 := logicalop.DataSource{}.Init(ctx, 0) t2 := logicalop.DataSource{}.Init(ctx, 0) join1 := logicalop.LogicalJoin{}.Init(ctx, 0) @@ -205,8 +221,11 @@ func TestBinderSubTreeMatch(t *testing.T) { } func TestBinderMultiNext(t *testing.T) { + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/pkg/planner/cascades/memo/MockPlanSkipMemoDeriveStats", `return(true)`)) + defer func() { + require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/pkg/planner/cascades/memo/MockPlanSkipMemoDeriveStats")) + }() ctx := mock.NewContext() - ctx.GetSessionVars().MockPlan = true asT1 := pmodel.NewCIStr("t1") asT2 := pmodel.NewCIStr("t2") t1 := logicalop.DataSource{TableAsName: &asT1}.Init(ctx, 0) @@ -295,16 +314,19 @@ func TestBinderMultiNext(t *testing.T) { // when G3 is exhausted, and next gE will be nil, and next() loop will enter next round with stack info popped as // G2(id(1)) which is what the third line comes from, and the next round will start from G2.next element starting // as G2(id(4)) which is the prefix of the fourth and fifth stack info. - require.Equal(t, b.String(), "GE:DataSource_-1{} -> GE:DataSource_-2{}\n"+ - "GE:DataSource_-1{} -> GE:DataSource_-5{}\n"+ - "GE:DataSource_-1{}\n"+ - "GE:DataSource_-4{} -> GE:DataSource_-2{}\n"+ - "GE:DataSource_-4{} -> GE:DataSource_-5{}\n") + require.Equal(t, b.String(), "GE:DataSource_1{} -> GE:DataSource_2{}\n"+ + "GE:DataSource_1{} -> GE:DataSource_5{}\n"+ + "GE:DataSource_1{}\n"+ + "GE:DataSource_4{} -> GE:DataSource_2{}\n"+ + "GE:DataSource_4{} -> GE:DataSource_5{}\n") } func TestBinderAny(t *testing.T) { + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/pkg/planner/cascades/memo/MockPlanSkipMemoDeriveStats", `return(true)`)) + defer func() { + require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/pkg/planner/cascades/memo/MockPlanSkipMemoDeriveStats")) + }() ctx := mock.NewContext() - ctx.GetSessionVars().MockPlan = true asT1 := pmodel.NewCIStr("t1") asT2 := pmodel.NewCIStr("t2") t1 := logicalop.DataSource{TableAsName: &asT1}.Init(ctx, 0) @@ -380,15 +402,18 @@ func TestBinderAny(t *testing.T) { // In a conclusion: the Group matched with Any pattern only generate the first group expression since we don't // care what the concrete group expression it is. Because the final generated group expression if any, will be // substituted ANY pattern with the referred group at last not a concrete one group expression inside. - require.Equal(t, b.String(), "GE:DataSource_-1{} -> GE:DataSource_-2{}\n"+ - "GE:DataSource_-1{}\n"+ - "GE:DataSource_-4{} -> GE:DataSource_-2{}\n"+ - "GE:DataSource_-4{}\n") + require.Equal(t, b.String(), "GE:DataSource_1{} -> GE:DataSource_2{}\n"+ + "GE:DataSource_1{}\n"+ + "GE:DataSource_4{} -> GE:DataSource_2{}\n"+ + "GE:DataSource_4{}\n") } func TestBinderMultiAny(t *testing.T) { + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/pkg/planner/cascades/memo/MockPlanSkipMemoDeriveStats", `return(true)`)) + defer func() { + require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/pkg/planner/cascades/memo/MockPlanSkipMemoDeriveStats")) + }() ctx := mock.NewContext() - ctx.GetSessionVars().MockPlan = true asT1 := pmodel.NewCIStr("t1") asT2 := pmodel.NewCIStr("t2") t1 := logicalop.DataSource{TableAsName: &asT1}.Init(ctx, 0) @@ -452,6 +477,6 @@ func TestBinderMultiAny(t *testing.T) { // G2{t1,t3} G3{t2,t4} // ▴ (already matched, pop stack) // final state: empty stack - require.Equal(t, b.String(), "GE:DataSource_-1{} -> GE:DataSource_-2{}\n"+ - "GE:DataSource_-1{}\n") + require.Equal(t, b.String(), "GE:DataSource_1{} -> GE:DataSource_2{}\n"+ + "GE:DataSource_1{}\n") } diff --git a/pkg/planner/core/operator/baseimpl/BUILD.bazel b/pkg/planner/core/operator/baseimpl/BUILD.bazel index ff054bc3209e5..01c216d35fda2 100644 --- a/pkg/planner/core/operator/baseimpl/BUILD.bazel +++ b/pkg/planner/core/operator/baseimpl/BUILD.bazel @@ -11,7 +11,6 @@ go_library( "//pkg/planner/planctx", "//pkg/planner/property", "//pkg/types", - "//pkg/util/intest", "//pkg/util/stringutil", "//pkg/util/tracing", ], diff --git a/pkg/planner/core/operator/baseimpl/plan.go b/pkg/planner/core/operator/baseimpl/plan.go index 6d5af8abdfe59..60459a651d1fa 100644 --- a/pkg/planner/core/operator/baseimpl/plan.go +++ b/pkg/planner/core/operator/baseimpl/plan.go @@ -24,7 +24,6 @@ import ( "github.com/pingcap/tidb/pkg/planner/planctx" "github.com/pingcap/tidb/pkg/planner/property" "github.com/pingcap/tidb/pkg/types" - "github.com/pingcap/tidb/pkg/util/intest" "github.com/pingcap/tidb/pkg/util/stringutil" "github.com/pingcap/tidb/pkg/util/tracing" ) @@ -40,14 +39,7 @@ type Plan struct { // NewBasePlan creates a new base plan. func NewBasePlan(ctx planctx.PlanContext, tp string, qbBlock int) Plan { - var id int32 - if ctx.GetSessionVars().MockPlan { - intest.Assert(ctx.GetSessionVars().PlanID.Load() <= 0) - id = ctx.GetSessionVars().PlanID.Add(-1) - } else { - intest.Assert(ctx.GetSessionVars().PlanID.Load() >= 0) - id = ctx.GetSessionVars().PlanID.Add(1) - } + id := ctx.GetSessionVars().PlanID.Add(1) return Plan{ tp: tp, id: int(id), From d64a273e33950e2176e5f5e8461330d369608f01 Mon Sep 17 00:00:00 2001 From: arenatlx <314806019@qq.com> Date: Wed, 11 Dec 2024 17:33:48 +0800 Subject: [PATCH 25/25] . Signed-off-by: arenatlx <314806019@qq.com> --- pkg/planner/core/exhaust_physical_plans_test.go | 2 +- pkg/planner/core/rule_join_reorder_dp_test.go | 8 ++++---- pkg/sessionctx/variable/session.go | 3 --- 3 files changed, 5 insertions(+), 8 deletions(-) diff --git a/pkg/planner/core/exhaust_physical_plans_test.go b/pkg/planner/core/exhaust_physical_plans_test.go index d304d0ca72e68..e66a046dc4633 100644 --- a/pkg/planner/core/exhaust_physical_plans_test.go +++ b/pkg/planner/core/exhaust_physical_plans_test.go @@ -64,7 +64,7 @@ func prepareForAnalyzeLookUpFilters() *indexJoinContext { do := domain.GetDomain(ctx) do.StatsHandle().Close() }() - ctx.GetSessionVars().PlanID.Store(0) + ctx.GetSessionVars().PlanID.Store(-1) joinNode := logicalop.LogicalJoin{}.Init(ctx.GetPlanCtx(), 0) dataSourceNode := logicalop.DataSource{}.Init(ctx.GetPlanCtx(), 0) dsSchema := expression.NewSchema() diff --git a/pkg/planner/core/rule_join_reorder_dp_test.go b/pkg/planner/core/rule_join_reorder_dp_test.go index 4b7e33f123f68..f89c34a8654cb 100644 --- a/pkg/planner/core/rule_join_reorder_dp_test.go +++ b/pkg/planner/core/rule_join_reorder_dp_test.go @@ -58,12 +58,12 @@ func newMockJoin(ctx base.PlanContext, statsMap map[int]*property.StatsInfo) fun if mj, ok := lChild.(*mockLogicalJoin); ok { retJoin.involvedNodeSet = mj.involvedNodeSet } else { - retJoin.involvedNodeSet = 1 << uint(lChild.ID()-1) + retJoin.involvedNodeSet = 1 << uint(lChild.ID()) } if mj, ok := rChild.(*mockLogicalJoin); ok { retJoin.involvedNodeSet |= mj.involvedNodeSet } else { - retJoin.involvedNodeSet |= 1 << uint(rChild.ID()-1) + retJoin.involvedNodeSet |= 1 << uint(rChild.ID()) } retJoin.SetChildren(lChild, rChild) retJoin.JoinType = joinType @@ -169,7 +169,7 @@ func TestDPReorderTPCHQ5(t *testing.T) { do := domain.GetDomain(ctx) do.StatsHandle().Close() }() - ctx.GetSessionVars().PlanID.Store(0) + ctx.GetSessionVars().PlanID.Store(-1) joinGroups := make([]base.LogicalPlan, 0, 6) joinGroups = append(joinGroups, newDataSource(ctx, "lineitem", 59986052)) joinGroups = append(joinGroups, newDataSource(ctx, "orders", 15000000)) @@ -217,7 +217,7 @@ func TestDPReorderAllCartesian(t *testing.T) { defer func() { domain.GetDomain(ctx).StatsHandle().Close() }() - ctx.GetSessionVars().PlanID.Store(0) + ctx.GetSessionVars().PlanID.Store(-1) joinGroup := make([]base.LogicalPlan, 0, 4) joinGroup = append(joinGroup, newDataSource(ctx, "a", 100)) diff --git a/pkg/sessionctx/variable/session.go b/pkg/sessionctx/variable/session.go index 696c6af2ba8df..0551e777188c8 100644 --- a/pkg/sessionctx/variable/session.go +++ b/pkg/sessionctx/variable/session.go @@ -857,9 +857,6 @@ type SessionVars struct { // SQLCPUUsages records tidb/tikv cpu usages for current sql SQLCPUUsages ppcpuusage.SQLCPUUsages - // MockPlan is tagged in test to generate logical plan with negative plan ids. - MockPlan bool - // PlanID is the unique id of logical and physical plan. PlanID atomic.Int32