From dcf1bf3fe61b7f5d2b4f18d3f07c3626e267d912 Mon Sep 17 00:00:00 2001 From: ghazalfamilyusa Date: Sat, 11 May 2024 14:29:58 -0700 Subject: [PATCH] first commit for unit testing --- pkg/planner/core/BUILD.bazel | 2 + pkg/planner/core/rule_join_reorder_dp_test.go | 112 +++++------------- pkg/planner/core/unit_test_util.go | 102 ++++++++++++++++ 3 files changed, 134 insertions(+), 82 deletions(-) create mode 100644 pkg/planner/core/unit_test_util.go diff --git a/pkg/planner/core/BUILD.bazel b/pkg/planner/core/BUILD.bazel index e9b9b99b70b0b..ba97ceaf7c22f 100644 --- a/pkg/planner/core/BUILD.bazel +++ b/pkg/planner/core/BUILD.bazel @@ -3,6 +3,7 @@ load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") go_library( name = "core", srcs = [ + "unit_test_util.go", "access_object.go", "collect_column_stats_usage.go", "common_plans.go", @@ -230,6 +231,7 @@ go_test( "rule_join_reorder_dp_test.go", "runtime_filter_generator_test.go", "stringer_test.go", + "unit_test_util.go", ], data = glob(["testdata/**"]), embed = [":core"], diff --git a/pkg/planner/core/rule_join_reorder_dp_test.go b/pkg/planner/core/rule_join_reorder_dp_test.go index 90d04ce992dd0..4a3fd0429d3e9 100644 --- a/pkg/planner/core/rule_join_reorder_dp_test.go +++ b/pkg/planner/core/rule_join_reorder_dp_test.go @@ -15,13 +15,12 @@ package core import ( - "fmt" "testing" + "context" "github.com/pingcap/tidb/pkg/domain" "github.com/pingcap/tidb/pkg/expression" "github.com/pingcap/tidb/pkg/parser/ast" - "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/planner/core/base" "github.com/pingcap/tidb/pkg/planner/property" @@ -29,47 +28,6 @@ import ( "github.com/stretchr/testify/require" ) -type mockLogicalJoin struct { - logicalSchemaProducer - involvedNodeSet int - statsMap map[int]*property.StatsInfo - JoinType JoinType -} - -func (mj mockLogicalJoin) init(ctx base.PlanContext) *mockLogicalJoin { - mj.baseLogicalPlan = newBaseLogicalPlan(ctx, "MockLogicalJoin", &mj, 0) - return &mj -} - -// RecursiveDeriveStats implements LogicalPlan interface. -func (mj *mockLogicalJoin) RecursiveDeriveStats(_ [][]*expression.Column) (*property.StatsInfo, error) { - if mj.StatsInfo() == nil { - mj.SetStats(mj.statsMap[mj.involvedNodeSet]) - } - return mj.statsMap[mj.involvedNodeSet], nil -} - -func newMockJoin(ctx base.PlanContext, statsMap map[int]*property.StatsInfo) func(lChild, rChild base.LogicalPlan, _ []*expression.ScalarFunction, _, _, _ []expression.Expression, joinType JoinType) base.LogicalPlan { - return func(lChild, rChild base.LogicalPlan, _ []*expression.ScalarFunction, _, _, _ []expression.Expression, joinType JoinType) base.LogicalPlan { - retJoin := mockLogicalJoin{}.init(ctx) - retJoin.schema = expression.MergeSchema(lChild.Schema(), rChild.Schema()) - retJoin.statsMap = statsMap - if mj, ok := lChild.(*mockLogicalJoin); ok { - retJoin.involvedNodeSet = mj.involvedNodeSet - } else { - retJoin.involvedNodeSet = 1 << uint(lChild.ID()) - } - if mj, ok := rChild.(*mockLogicalJoin); ok { - retJoin.involvedNodeSet |= mj.involvedNodeSet - } else { - retJoin.involvedNodeSet |= 1 << uint(rChild.ID()) - } - retJoin.SetChildren(lChild, rChild) - retJoin.JoinType = joinType - return retJoin - } -} - func makeStatsMapForTPCHQ5() map[int]*property.StatsInfo { // Labeled as lineitem -> 0, orders -> 1, customer -> 2, supplier 3, nation 4, region 5 // This graph can be shown as following: @@ -135,31 +93,6 @@ func makeStatsMapForTPCHQ5() map[int]*property.StatsInfo { return statsMap } -func newDataSource(ctx base.PlanContext, name string, count int) base.LogicalPlan { - ds := DataSource{}.Init(ctx, 0) - tan := model.NewCIStr(name) - ds.TableAsName = &tan - ds.schema = expression.NewSchema() - ds.schema.Append(&expression.Column{ - UniqueID: ctx.GetSessionVars().PlanColumnID.Add(1), - RetType: types.NewFieldType(mysql.TypeLonglong), - }) - ds.SetStats(&property.StatsInfo{ - RowCount: float64(count), - }) - return ds -} - -func planToString(plan base.LogicalPlan) string { - switch x := plan.(type) { - case *mockLogicalJoin: - return fmt.Sprintf("MockJoin{%v, %v}", planToString(x.children[0]), planToString(x.children[1])) - case *DataSource: - return x.TableAsName.L - } - return "" -} - func TestDPReorderTPCHQ5(t *testing.T) { statsMap := makeStatsMapForTPCHQ5() @@ -170,12 +103,12 @@ func TestDPReorderTPCHQ5(t *testing.T) { }() 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)) - joinGroups = append(joinGroups, newDataSource(ctx, "customer", 1500000)) - joinGroups = append(joinGroups, newDataSource(ctx, "supplier", 100000)) - joinGroups = append(joinGroups, newDataSource(ctx, "nation", 25)) - joinGroups = append(joinGroups, newDataSource(ctx, "region", 5)) + joinGroups = append(joinGroups, MockDataSource(ctx, "lineitem", 59986052)) + joinGroups = append(joinGroups, MockDataSource(ctx, "orders", 15000000)) + joinGroups = append(joinGroups, MockDataSource(ctx, "customer", 1500000)) + joinGroups = append(joinGroups, MockDataSource(ctx, "supplier", 100000)) + joinGroups = append(joinGroups, MockDataSource(ctx, "nation", 25)) + joinGroups = append(joinGroups, MockDataSource(ctx, "region", 5)) var eqConds []expression.Expression eqConds = append(eqConds, expression.NewFunctionInternal(ctx, ast.EQ, types.NewFieldType(mysql.TypeTiny), joinGroups[0].Schema().Columns[0], joinGroups[1].Schema().Columns[0])) @@ -200,13 +133,13 @@ func TestDPReorderTPCHQ5(t *testing.T) { } solver := &joinReorderDPSolver{ baseSingleGroupJoinOrderSolver: baseGroupSolver, - newJoin: newMockJoin(ctx, statsMap), + newJoin: NewMockJoinForJoinOrder(ctx, statsMap), } result, err := solver.solve(joinGroups, nil) require.NoError(t, err) expected := "MockJoin{supplier, MockJoin{lineitem, MockJoin{orders, MockJoin{customer, MockJoin{nation, region}}}}}" - require.Equal(t, expected, planToString(result)) + require.Equal(t, expected, PlanToString(result)) } func TestDPReorderAllCartesian(t *testing.T) { @@ -219,20 +152,35 @@ func TestDPReorderAllCartesian(t *testing.T) { ctx.GetSessionVars().PlanID.Store(-1) joinGroup := make([]base.LogicalPlan, 0, 4) - joinGroup = append(joinGroup, newDataSource(ctx, "a", 100)) - joinGroup = append(joinGroup, newDataSource(ctx, "b", 100)) - joinGroup = append(joinGroup, newDataSource(ctx, "c", 100)) - joinGroup = append(joinGroup, newDataSource(ctx, "d", 100)) + joinGroup = append(joinGroup, MockDataSource(ctx, "a", 100)) + joinGroup = append(joinGroup, MockDataSource(ctx, "b", 100)) + joinGroup = append(joinGroup, MockDataSource(ctx, "c", 100)) + joinGroup = append(joinGroup, MockDataSource(ctx, "d", 100)) solver := &joinReorderDPSolver{ baseSingleGroupJoinOrderSolver: &baseSingleGroupJoinOrderSolver{ ctx: ctx, basicJoinGroupInfo: &basicJoinGroupInfo{}, }, - newJoin: newMockJoin(ctx, statsMap), + newJoin: NewMockJoinForJoinOrder(ctx, statsMap), } result, err := solver.solve(joinGroup, nil) require.NoError(t, err) expected := "MockJoin{MockJoin{a, b}, MockJoin{c, d}}" - require.Equal(t, expected, planToString(result)) + require.Equal(t, expected, PlanToString(result)) +} + +func TestLogicalPlanner(t *testing.T) { + ctx := MockContext() + defer func() { + do := domain.GetDomain(ctx) + do.StatsHandle().Close() + }() + leftChild := MockDataSource(ctx, "lineitem", 59986052) + rightChild := MockDataSource(ctx, "orders", 15000000) + plan := MockJoin(ctx, leftChild, rightChild, InnerJoin) + optimizedPlan, err := logicalOptimize(context.TODO(), 0, plan) + require.NoError(t, err) + expected := "MockJoin{lineitem, orders}" + require.Equal(t, expected, PlanToString(optimizedPlan)) } diff --git a/pkg/planner/core/unit_test_util.go b/pkg/planner/core/unit_test_util.go new file mode 100644 index 0000000000000..dad57c0894e2e --- /dev/null +++ b/pkg/planner/core/unit_test_util.go @@ -0,0 +1,102 @@ +// Copyright 2018 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 core + +import ( + "fmt" + + "github.com/pingcap/tidb/pkg/expression" + "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/mysql" + "github.com/pingcap/tidb/pkg/planner/core/base" + "github.com/pingcap/tidb/pkg/planner/property" + "github.com/pingcap/tidb/pkg/types" +) + +type mockLogicalJoinForJoinOrder struct { + logicalSchemaProducer + involvedNodeSet int + statsMap map[int]*property.StatsInfo + JoinType JoinType +} + +func (mj mockLogicalJoinForJoinOrder) init(ctx base.PlanContext) *mockLogicalJoinForJoinOrder { + mj.baseLogicalPlan = newBaseLogicalPlan(ctx, "MockLogicalJoin", &mj, 0) + return &mj +} + +// NewMockJoinForJoinOrder creates a LogicalJoin node for join order testing testing. +func NewMockJoinForJoinOrder(ctx base.PlanContext, statsMap map[int]*property.StatsInfo) func(lChild, rChild base.LogicalPlan, _ []*expression.ScalarFunction, _, _, _ []expression.Expression, joinType JoinType) base.LogicalPlan { + return func(lChild, rChild base.LogicalPlan, _ []*expression.ScalarFunction, _, _, _ []expression.Expression, joinType JoinType) base.LogicalPlan { + retJoin := mockLogicalJoinForJoinOrder{}.init(ctx) + retJoin.schema = expression.MergeSchema(lChild.Schema(), rChild.Schema()) + retJoin.statsMap = statsMap + if mj, ok := lChild.(*mockLogicalJoinForJoinOrder); ok { + retJoin.involvedNodeSet = mj.involvedNodeSet + } else { + retJoin.involvedNodeSet = 1 << uint(lChild.ID()) + } + if mj, ok := rChild.(*mockLogicalJoinForJoinOrder); ok { + retJoin.involvedNodeSet |= mj.involvedNodeSet + } else { + retJoin.involvedNodeSet |= 1 << uint(rChild.ID()) + } + retJoin.SetChildren(lChild, rChild) + retJoin.JoinType = joinType + return retJoin + } +} + +// MockDataSource make logical DataSource. +func MockJoin(ctx base.PlanContext, leftPlan base.LogicalPlan, rightPlan base.LogicalPlan, joinType JoinType) base.LogicalPlan { + joinPlan := LogicalJoin{}.Init(ctx, 0) + joinPlan.SetChildren(leftPlan, rightPlan) + joinPlan.SetSchema(expression.MergeSchema(leftPlan.Schema(), rightPlan.Schema())) + joinPlan.names = make([]*types.FieldName, leftPlan.Schema().Len()+rightPlan.Schema().Len()) + copy(joinPlan.names, leftPlan.OutputNames()) + copy(joinPlan.names[leftPlan.Schema().Len():], rightPlan.OutputNames()) + joinPlan.JoinType = joinType + return joinPlan +} + +// MockDataSource make logical DataSource. +func MockDataSource(ctx base.PlanContext, name string, count int) base.LogicalPlan { + ds := DataSource{}.Init(ctx, 0) + tan := model.NewCIStr(name) + ds.TableAsName = &tan + ds.schema = expression.NewSchema() + ds.schema.Append(&expression.Column{ + UniqueID: ctx.GetSessionVars().PlanColumnID.Add(1), + RetType: types.NewFieldType(mysql.TypeLonglong), + }) + ds.SetStats(&property.StatsInfo{ + RowCount: float64(count), + }) + return ds +} + +// PlanToString serializes some logical plan nodes to string. +func PlanToString(plan base.LogicalPlan) string { + switch x := plan.(type) { + case *mockLogicalJoinForJoinOrder: + return fmt.Sprintf("MockJoin{%v, %v}", PlanToString(x.children[0]), PlanToString(x.children[1])) + case *LogicalJoin: + return fmt.Sprintf("MockJoin{%v, %v}", PlanToString(x.children[0]), PlanToString(x.children[1])) + case *DataSource: + return x.TableAsName.L + } + return "" +} +