Skip to content

Commit

Permalink
first commit for unit testing
Browse files Browse the repository at this point in the history
  • Loading branch information
ghazalfamilyusa committed May 11, 2024
1 parent c73d6c5 commit dcf1bf3
Show file tree
Hide file tree
Showing 3 changed files with 134 additions and 82 deletions.
2 changes: 2 additions & 0 deletions pkg/planner/core/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -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",
Expand Down Expand Up @@ -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"],
Expand Down
112 changes: 30 additions & 82 deletions pkg/planner/core/rule_join_reorder_dp_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -15,61 +15,19 @@
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"
"github.com/pingcap/tidb/pkg/types"
"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:
Expand Down Expand Up @@ -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()

Expand All @@ -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]))
Expand All @@ -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) {
Expand All @@ -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))
}
102 changes: 102 additions & 0 deletions pkg/planner/core/unit_test_util.go
Original file line number Diff line number Diff line change
@@ -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 ""
}

0 comments on commit dcf1bf3

Please sign in to comment.