diff --git a/disttask/framework/planner/BUILD.bazel b/disttask/framework/planner/BUILD.bazel index f4d62a1c10a71..7bd1172250a5c 100644 --- a/disttask/framework/planner/BUILD.bazel +++ b/disttask/framework/planner/BUILD.bazel @@ -1,4 +1,4 @@ -load("@io_bazel_rules_go//go:def.bzl", "go_library") +load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") go_library( name = "planner", @@ -13,3 +13,21 @@ go_library( "//sessionctx", ], ) + +go_test( + name = "planner_test", + timeout = "short", + srcs = [ + "plan_test.go", + "planner_test.go", + ], + embed = [":planner"], + flaky = True, + deps = [ + "//disttask/framework/storage", + "//testkit", + "@com_github_ngaut_pools//:pools", + "@com_github_stretchr_testify//require", + "@com_github_tikv_client_go_v2//util", + ], +) diff --git a/disttask/framework/planner/plan_test.go b/disttask/framework/planner/plan_test.go new file mode 100644 index 0000000000000..1c1a7c2ec1739 --- /dev/null +++ b/disttask/framework/planner/plan_test.go @@ -0,0 +1,36 @@ +// Copyright 2023 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 planner + +import ( + "testing" + + "github.com/stretchr/testify/require" +) + +type mockOperator struct{} + +func (o mockOperator) ToSubtaskMeta(PlanCtx) ([]byte, error) { + return []byte("mock"), nil +} + +func TestPhysicalPlan(t *testing.T) { + plan := &PhysicalPlan{} + planCtx := PlanCtx{} + plan.AddProcessor(ProcessorSpec{Operator: mockOperator{}, Step: 1}) + subtaskMetas, err := plan.ToSubtaskMetas(planCtx, 1) + require.NoError(t, err) + require.Equal(t, [][]byte{[]byte("mock")}, subtaskMetas) +} diff --git a/disttask/framework/planner/planner_test.go b/disttask/framework/planner/planner_test.go new file mode 100644 index 0000000000000..c212dae792730 --- /dev/null +++ b/disttask/framework/planner/planner_test.go @@ -0,0 +1,67 @@ +// Copyright 2023 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 planner_test + +import ( + "context" + "testing" + "time" + + "github.com/ngaut/pools" + "github.com/pingcap/tidb/disttask/framework/planner" + "github.com/pingcap/tidb/disttask/framework/storage" + "github.com/pingcap/tidb/testkit" + "github.com/stretchr/testify/require" + "github.com/tikv/client-go/v2/util" +) + +type mockPlan struct{} + +func (mockPlan) ToTaskMeta() ([]byte, error) { + return []byte("mock"), nil +} + +func (mockPlan) FromTaskMeta([]byte) error { + return nil +} + +func (mockPlan) ToPhysicalPlan(planner.PlanCtx) (*planner.PhysicalPlan, error) { + return &planner.PhysicalPlan{}, nil +} + +func TestPlanner(t *testing.T) { + ctx := context.Background() + store := testkit.CreateMockStore(t) + gtk := testkit.NewTestKit(t, store) + pool := pools.NewResourcePool(func() (pools.Resource, error) { + return gtk.Session(), nil + }, 1, 1, time.Second) + defer pool.Close() + mgr := storage.NewTaskManager(util.WithInternalSourceType(ctx, "taskManager"), pool) + storage.SetTaskManager(mgr) + + p := &planner.Planner{} + pCtx := planner.PlanCtx{ + Ctx: ctx, + SessionCtx: gtk.Session(), + TaskKey: "1", + TaskType: "example", + ThreadCnt: 1, + } + plan := &mockPlan{} + taskID, err := p.Run(pCtx, plan) + require.NoError(t, err) + require.Equal(t, int64(1), taskID) +}