Skip to content

Commit

Permalink
planner: add auto hash64 and equals generator for logical operators. (#…
Browse files Browse the repository at this point in the history
  • Loading branch information
AilinKid authored Oct 31, 2024
1 parent c240fdf commit e92e2d0
Show file tree
Hide file tree
Showing 12 changed files with 417 additions and 50 deletions.
1 change: 1 addition & 0 deletions build/nogo_config.json
Original file line number Diff line number Diff line change
Expand Up @@ -524,6 +524,7 @@
"pkg/parser/ast/": "ignore parser/ast code",
"pkg/parser/test_driver/": "ignore parser/test_driver code",
"pkg/planner/core/plan_clone_generated.go": "ignore plan_clone_generated code",
"pkg/planner/core/operator/logicalop/hash64_equals_generated.go": "ignore hash64_equals_generated code",
".*_test\\.go$": "ignore generated code",
".*_generated\\.go$": "ignore generated code",
".*mock.go$": "ignore generated code",
Expand Down
33 changes: 33 additions & 0 deletions pkg/planner/core/generator/hash64_equals/BUILD.bazel
Original file line number Diff line number Diff line change
@@ -0,0 +1,33 @@
load("@io_bazel_rules_go//go:def.bzl", "go_binary", "go_library", "go_test")

go_library(
name = "hash64_equals_lib",
srcs = ["hash64_equals_generator.go"],
importpath = "github.com/pingcap/tidb/pkg/planner/core/generator/hash64_equals",
visibility = ["//visibility:private"],
deps = [
"//pkg/planner/cascades/base",
"//pkg/planner/core/operator/logicalop",
],
)

go_binary(
name = "hash64_equals",
embed = [":hash64_equals_lib"],
visibility = ["//visibility:public"],
)

go_test(
name = "hash64_equals_test",
timeout = "short",
srcs = ["hash64_equals_test.go"],
data = [
"//pkg/planner/core/operator/logicalop:generator_files",
],
embed = [":hash64_equals_lib"],
flaky = True,
deps = [
"//pkg/util",
"@com_github_stretchr_testify//require",
],
)
163 changes: 163 additions & 0 deletions pkg/planner/core/generator/hash64_equals/hash64_equals_generator.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,163 @@
// 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 main

import (
"bytes"
"fmt"
"go/format"
"log"
"os"
"reflect"

"github.com/pingcap/tidb/pkg/planner/cascades/base"
"github.com/pingcap/tidb/pkg/planner/core/operator/logicalop"
)

// GenHash64Equals4LogicalOps generates Hash64(xxx) and Equals(xxx) for all logical plan nodes in hash64_equals_generated.go.
// Using code-gen is safer than writing by hand, for example, if someone adds a new field to a struct,
// the code-gen can update the Clone method correctly and automatically.
// To update hash64_equals_generated.go, please run TestUpdateHash64EqualsCode manually.
// This function relies on Golang field tags to determine whether to hash64/equals involve a field or not.
// If a field is tagged with `hash64-equals`, then it will be computed in hash64 and equals func.
// If a field is not tagged, then it will be skipped.
func GenHash64Equals4LogicalOps() ([]byte, error) {
var structures = []any{logicalop.LogicalJoin{}}
c := new(cc)
c.write(codeGenHash64EqualsPrefix)
for _, s := range structures {
code, err := genHash64EqualsForLogicalOps(s)
if err != nil {
return nil, err
}
c.write("%s", string(code))
}
return c.format()
}

var hashEqualsType = reflect.TypeOf((*base.HashEquals)(nil)).Elem()

func genHash64EqualsForLogicalOps(x any) ([]byte, error) {
c := new(cc)
vType := reflect.TypeOf(x)
c.write("// Hash64 implements the Hash64Equals interface.")
c.write("func (op *%v) Hash64(h base.Hasher) {", vType.Name())
c.write("h.HashString(%v)", logicalOpName2PlanCodecString(vType.Name()))
for i := 0; i < vType.NumField(); i++ {
f := vType.Field(i)
if !isHash64EqualsField(f) {
continue
}
callName := "op." + vType.Field(i).Name
// if a field is a pointer, we should encode the Nil/NotNil flag inside hash64.
if f.Type.Kind() == reflect.Pointer || f.Type.Kind() == reflect.Slice {
c.write("if %v == nil { h.HashByte(base.NilFlag) } else {", callName)
c.write("h.HashByte(base.NotNilFlag)")
c.Hash64Element(f.Type, callName)
c.write("}")
} else {
c.Hash64Element(f.Type, callName)
}
}
c.write("}")
return c.format()
}

func logicalOpName2PlanCodecString(name string) string {
switch name {
case "LogicalJoin":
return "plancodec.TypeJoin"
default:
return ""
}
}

func isHash64EqualsField(fType reflect.StructField) bool {
return fType.Tag.Get("hash64-equals") == "true"
}

func (c *cc) Hash64Element(fType reflect.Type, callName string) {
switch fType.Kind() {
case reflect.Slice:
c.write("h.HashInt(len(%v))", callName)
c.write("for _, one := range %v {", callName)
// one more round
c.Hash64Element(fType.Elem(), "one")
c.write("}")
case reflect.String:
c.write("h.HashString(%v)", callName)
case reflect.Bool:
c.write("h.HashBool(%v)", callName)
case reflect.Int, reflect.Int8, reflect.Int16, reflect.Int32, reflect.Int64:
c.write("h.HashInt64(int64(%v))", callName)
case reflect.Uint, reflect.Uint8, reflect.Uint16, reflect.Uint32, reflect.Uint64:
c.write("h.HashUint64(uint64(%v))", callName)
case reflect.Float32, reflect.Float64:
c.write("h.HashFloat64(float64(%v))", callName)
default:
if fType.Implements(hashEqualsType) {
c.write("%v.Hash64(h)", callName)
} else {
panic("doesn't support element type" + fType.Kind().String())
}
}
}

type cc struct {
buffer bytes.Buffer
}

func (c *cc) write(format string, args ...any) {
c.buffer.WriteString(fmt.Sprintf(format, args...))
c.buffer.WriteString("\n")
}

func (c *cc) format() ([]byte, error) {
return format.Source(c.buffer.Bytes())
}

const codeGenHash64EqualsPrefix = `// 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.
// Code generated by hash64_equals_generator; DO NOT EDIT IT DIRECTLY.
package logicalop
import (
"github.com/pingcap/tidb/pkg/planner/cascades/base"
"github.com/pingcap/tidb/pkg/util/plancodec"
)
`

func main() {
fileData, err := GenHash64Equals4LogicalOps()
if err != nil {
log.Fatalln("failed to generate hash64_equals_generated.go", err)
}
if err := os.WriteFile("hash64_equals_generated.go", fileData, 0644); err != nil {
log.Fatalln("failed to write hash64_equals_generated.go", err)
}
}
122 changes: 122 additions & 0 deletions pkg/planner/core/generator/hash64_equals/hash64_equals_test.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,122 @@
// 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 main

import (
"bufio"
"bytes"
"io"
"os"
"reflect"
"strings"
"testing"

"github.com/pingcap/tidb/pkg/util"
"github.com/stretchr/testify/require"
)

type Test interface {
Hello() string
}

type A struct {
}

type B1 struct {
b *A
}

func (*B1) Hello() string {
return "B1"
}

type B2 struct {
b A
}

func (*B2) Hello() string {
return "B2"
}

func TestGenHash64EqualsField(t *testing.T) {
vType1 := reflect.TypeOf(B1{})
vType2 := reflect.TypeOf(B2{})
f1 := vType1.Field(0)
f2 := vType2.Field(0)
k1 := f1.Type.Kind()
k2 := f2.Type.Kind()
require.Equal(t, k1, reflect.Pointer)
require.Equal(t, k2, reflect.Struct)

vValue1 := reflect.ValueOf(B1{})
// vValue2 := reflect.ValueOf(B2{})
require.True(t, vValue1.Field(0).IsNil())
// we can't call IsNil on a non-pointer field.
// require.False(t, vValue2.Field(0).IsNil())

vType1 = reflect.TypeOf(&B1{})
vType2 = reflect.TypeOf(&B2{})
// pointer do not have a filed.
// f1 := vType1.Field(0)
// f2 := vType2.Field(0)
k1 = vType1.Kind()
k2 = vType2.Kind()
require.Equal(t, k1, reflect.Pointer)
require.Equal(t, k2, reflect.Pointer)

vValue1 = reflect.ValueOf(&B1{})
vValue2 := reflect.ValueOf(&B2{})
// value still pointer, no fields.
// require.Equal(t, 0, vValue1.NumField())
// require.Equal(t, 0, vValue2.NumField())

vValue1 = reflect.Indirect(vValue1)
vValue2 = reflect.Indirect(vValue2)
require.Equal(t, 1, vValue1.NumField())
require.Equal(t, 1, vValue2.NumField())

TestType := reflect.TypeOf((*Test)(nil)).Elem()
require.True(t, vType1.Implements(TestType))
require.True(t, vType2.Implements(TestType))
}

func TestHash64Equals(t *testing.T) {
updatedCode, err := GenHash64Equals4LogicalOps()
if err != nil {
t.Errorf("Generate CloneForPlanCache code error: %v", err)
return
}
currentCode, err := os.ReadFile("../../operator/logicalop/hash64_equals_generated.go")
if err != nil {
t.Errorf("Read current hash64_equals_generated.go code error: %v", err)
return
}
updateLines := bufio.NewReader(strings.NewReader(string(updatedCode)))
currentLines := bufio.NewReader(strings.NewReader(string(currentCode)))
for {
line1, err1 := util.ReadLine(updateLines, 1024)
line2, err2 := util.ReadLine(currentLines, 1024)
if err1 == nil && err2 != nil || err1 != nil && err2 == nil || err1 != nil && err2 != nil && err1.Error() != err2.Error() || !bytes.Equal(line1, line2) {
t.Errorf("line unmatched, line1: %s, line2: %s", string(line1), string(line2))
break
}
if err1 == io.EOF && err2 == io.EOF {
break
}
}
if !bytes.Equal(updatedCode, currentCode) {
t.Errorf("hash64_equals_generated.go should be updated, please run 'make gogenerate' to update it.")
}
}
Original file line number Diff line number Diff line change
@@ -1,26 +1,26 @@
load("@io_bazel_rules_go//go:def.bzl", "go_binary", "go_library", "go_test")

go_library(
name = "generator_lib",
name = "plan_cache_lib",
srcs = ["plan_clone_generator.go"],
importpath = "github.com/pingcap/tidb/pkg/planner/core/generator",
importpath = "github.com/pingcap/tidb/pkg/planner/core/generator/plan_cache",
visibility = ["//visibility:private"],
deps = ["//pkg/planner/core"],
)

go_binary(
name = "generator",
embed = [":generator_lib"],
name = "plan_cache",
embed = [":plan_cache_lib"],
visibility = ["//visibility:public"],
)

go_test(
name = "generator_test",
name = "plan_cache_test",
timeout = "short",
srcs = ["plan_clone_test.go"],
data = [
"//pkg/planner/core:generator_files",
],
embed = [":generator_lib"],
embed = [":plan_cache_lib"],
flaky = True,
)
Original file line number Diff line number Diff line change
Expand Up @@ -42,7 +42,7 @@ func GenPlanCloneForPlanCacheCode() ([]byte, error) {
core.PhysicalIndexJoin{}, core.PhysicalIndexHashJoin{}, core.PhysicalIndexLookUpReader{}, core.PhysicalIndexMergeReader{},
core.Update{}, core.Delete{}, core.Insert{}, core.PhysicalLock{}, core.PhysicalUnionScan{}, core.PhysicalUnionAll{}}
c := new(codeGen)
c.write(codeGenPrefix)
c.write(codeGenPlanCachePrefix)
for _, s := range structures {
code, err := genPlanCloneForPlanCache(s)
if err != nil {
Expand Down Expand Up @@ -204,7 +204,7 @@ func (c *codeGen) format() ([]byte, error) {
return format.Source(c.buffer.Bytes())
}

const codeGenPrefix = `// Copyright 2024 PingCAP, Inc.
const codeGenPlanCachePrefix = `// 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.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -26,7 +26,7 @@ func TestPlanClone(t *testing.T) {
t.Errorf("Generate CloneForPlanCache code error: %v", err)
return
}
currentCode, err := os.ReadFile("../plan_clone_generated.go")
currentCode, err := os.ReadFile("../../plan_clone_generated.go")
if err != nil {
t.Errorf("Read current plan_clone_generated.go code error: %v", err)
return
Expand Down
Loading

0 comments on commit e92e2d0

Please sign in to comment.