Skip to content

Commit

Permalink
planner: add simple task stack for memo (#51663)
Browse files Browse the repository at this point in the history
close #51664
  • Loading branch information
AilinKid authored Mar 12, 2024
1 parent 46e95f4 commit db87239
Show file tree
Hide file tree
Showing 3 changed files with 175 additions and 1 deletion.
4 changes: 3 additions & 1 deletion pkg/planner/memo/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -8,6 +8,7 @@ go_library(
"group_expr.go",
"implementation.go",
"pattern.go",
"task.go",
],
importpath = "github.com/pingcap/tidb/pkg/planner/memo",
visibility = ["//visibility:public"],
Expand All @@ -27,10 +28,11 @@ go_test(
"group_test.go",
"main_test.go",
"pattern_test.go",
"task_test.go",
],
embed = [":memo"],
flaky = True,
shard_count = 22,
shard_count = 24,
deps = [
"//pkg/domain",
"//pkg/expression",
Expand Down
77 changes: 77 additions & 0 deletions pkg/planner/memo/task.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,77 @@
// 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.
// 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 memo

import (
"sync"
)

// Task is an interface defined for all type of optimizing work: exploring, implementing, deriving-stats, join-reordering and so on.
type Task interface {
// task self executing logic
execute() error
// task self description string.
desc() string
}

// TaskStackPool is initialized for memory saving by reusing taskStack.
var TaskStackPool = sync.Pool{
New: func() any {
return newTaskStack()
},
}

// TaskStack is used to store the optimizing tasks created before or during the optimizing process.
type TaskStack struct {
tasks []Task
}

func newTaskStack() *TaskStack {
return &TaskStack{
tasks: make([]Task, 0, 4),
}
}

// Destroy indicates that when stack itself is useless like in the end of optimizing phase, we can destroy ourselves.
func (ts *TaskStack) Destroy() {
// when a taskStack itself is useless, we can destroy itself actively.
clear(ts.tasks)
TaskStackPool.Put(ts)
}

// Len indicates the length of current stack.
func (ts *TaskStack) Len() int {
return len(ts.tasks)
}

// Pop indicates to pop one task out of the stack.
func (ts *TaskStack) Pop() Task {
if !ts.Empty() {
tmp := ts.tasks[len(ts.tasks)-1]
ts.tasks = ts.tasks[:len(ts.tasks)-1]
return tmp
}
return nil
}

// Push indicates to push one task into the stack.
func (ts *TaskStack) Push(one Task) {
ts.tasks = append(ts.tasks, one)
}

// Empty indicates whether taskStack is empty.
func (ts *TaskStack) Empty() bool {
return ts.Len() == 0
}
95 changes: 95 additions & 0 deletions pkg/planner/memo/task_test.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,95 @@
// 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.
// 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 memo

import (
"strconv"
"testing"
"unsafe"

"github.com/stretchr/testify/require"
)

type TestTaskImpl struct {
a int64
}

func (t *TestTaskImpl) execute() error {
return nil
}
func (t *TestTaskImpl) desc() string {
return strconv.Itoa(int(t.a))
}

func TestTaskStack(t *testing.T) {
newSS := newTaskStack()
// size of pointer to TaskStack{}
require.Equal(t, int64(unsafe.Sizeof(newSS)), int64(8))
// size of pointer to TaskStack.[]Task, cap + len + addr
require.Equal(t, int64(unsafe.Sizeof(newSS.tasks)), int64(24))
// size of pointer to TaskStack's first element Task[0]
newSS.Push(nil)
newSS.Push(&TestTaskImpl{a: 1})
newSS.Push(nil)
v := unsafe.Sizeof(newSS.tasks[0])
require.Equal(t, int64(v), int64(16))
v = unsafe.Sizeof(newSS.tasks[1])
require.Equal(t, int64(v), int64(16))
}

func TestTaskFunctionality(t *testing.T) {
taskTaskPool := TaskStackPool.Get()
require.Equal(t, len(taskTaskPool.(*TaskStack).tasks), 0)
require.Equal(t, cap(taskTaskPool.(*TaskStack).tasks), 4)
taskStack := taskTaskPool.(*TaskStack)
taskStack.Push(&TestTaskImpl{a: 1})
taskStack.Push(&TestTaskImpl{a: 2})
one := taskStack.Pop()
require.Equal(t, one.desc(), "2")
one = taskStack.Pop()
require.Equal(t, one.desc(), "1")
// empty, pop nil.
one = taskStack.Pop()
require.Nil(t, one)

taskStack.Push(&TestTaskImpl{a: 3})
taskStack.Push(&TestTaskImpl{a: 4})
taskStack.Push(&TestTaskImpl{a: 5})
taskStack.Push(&TestTaskImpl{a: 6})
// no clean, put it back
TaskStackPool.Put(taskTaskPool)

// require again.
taskTaskPool = TaskStackPool.Get()
require.Equal(t, len(taskTaskPool.(*TaskStack).tasks), 4)
require.Equal(t, cap(taskTaskPool.(*TaskStack).tasks), 4)
// clean the stack
one = taskStack.Pop()
require.Equal(t, one.desc(), "6")
one = taskStack.Pop()
require.Equal(t, one.desc(), "5")
one = taskStack.Pop()
require.Equal(t, one.desc(), "4")
one = taskStack.Pop()
require.Equal(t, one.desc(), "3")
one = taskStack.Pop()
require.Nil(t, one)

// self destroy.
taskStack.Destroy()
taskTaskPool = TaskStackPool.Get()
require.Equal(t, len(taskTaskPool.(*TaskStack).tasks), 0)
require.Equal(t, cap(taskTaskPool.(*TaskStack).tasks), 4)
}

0 comments on commit db87239

Please sign in to comment.