Skip to content

Commit ce74191

Browse files
authored
planner: move hash encoding logic and HasMaxOneRow to util (#53161)
ref #51664, ref #52714
1 parent 09c8f96 commit ce74191

File tree

7 files changed

+58
-21
lines changed

7 files changed

+58
-21
lines changed

pkg/planner/core/hashcode.go

Lines changed: 13 additions & 18 deletions
Original file line numberDiff line numberDiff line change
@@ -19,21 +19,16 @@ import (
1919
"encoding/binary"
2020
"slices"
2121

22+
"github.com/pingcap/tidb/pkg/planner/util"
2223
"github.com/pingcap/tidb/pkg/util/plancodec"
2324
)
2425

25-
func encodeIntAsUint32(result []byte, value int) []byte {
26-
var buf [4]byte
27-
binary.BigEndian.PutUint32(buf[:], uint32(value))
28-
return append(result, buf[:]...)
29-
}
30-
3126
// HashCode implements LogicalPlan interface.
3227
func (p *baseLogicalPlan) HashCode() []byte {
3328
// We use PlanID for the default hash, so if two plans do not have
3429
// the same id, the hash value will never be the same.
3530
result := make([]byte, 0, 4)
36-
result = encodeIntAsUint32(result, p.ID())
31+
result = util.EncodeIntAsUint32(result, p.ID())
3732
return result
3833
}
3934

@@ -43,12 +38,12 @@ func (p *LogicalProjection) HashCode() []byte {
4338
// Expressions are commonly `Column`s, whose hashcode has the length 9, so
4439
// we pre-alloc 10 bytes for each expr's hashcode.
4540
result := make([]byte, 0, 12+len(p.Exprs)*10)
46-
result = encodeIntAsUint32(result, plancodec.TypeStringToPhysicalID(p.TP()))
47-
result = encodeIntAsUint32(result, p.QueryBlockOffset())
48-
result = encodeIntAsUint32(result, len(p.Exprs))
41+
result = util.EncodeIntAsUint32(result, plancodec.TypeStringToPhysicalID(p.TP()))
42+
result = util.EncodeIntAsUint32(result, p.QueryBlockOffset())
43+
result = util.EncodeIntAsUint32(result, len(p.Exprs))
4944
for _, expr := range p.Exprs {
5045
exprHashCode := expr.HashCode()
51-
result = encodeIntAsUint32(result, len(exprHashCode))
46+
result = util.EncodeIntAsUint32(result, len(exprHashCode))
5247
result = append(result, exprHashCode...)
5348
}
5449
return result
@@ -58,9 +53,9 @@ func (p *LogicalProjection) HashCode() []byte {
5853
func (p *LogicalTableDual) HashCode() []byte {
5954
// PlanType + SelectOffset + RowCount
6055
result := make([]byte, 0, 12)
61-
result = encodeIntAsUint32(result, plancodec.TypeStringToPhysicalID(p.TP()))
62-
result = encodeIntAsUint32(result, p.QueryBlockOffset())
63-
result = encodeIntAsUint32(result, p.RowCount)
56+
result = util.EncodeIntAsUint32(result, plancodec.TypeStringToPhysicalID(p.TP()))
57+
result = util.EncodeIntAsUint32(result, p.QueryBlockOffset())
58+
result = util.EncodeIntAsUint32(result, p.RowCount)
6459
return result
6560
}
6661

@@ -70,9 +65,9 @@ func (p *LogicalSelection) HashCode() []byte {
7065
// Conditions are commonly `ScalarFunction`s, whose hashcode usually has a
7166
// length larger than 20, so we pre-alloc 25 bytes for each expr's hashcode.
7267
result := make([]byte, 0, 12+len(p.Conditions)*25)
73-
result = encodeIntAsUint32(result, plancodec.TypeStringToPhysicalID(p.TP()))
74-
result = encodeIntAsUint32(result, p.QueryBlockOffset())
75-
result = encodeIntAsUint32(result, len(p.Conditions))
68+
result = util.EncodeIntAsUint32(result, plancodec.TypeStringToPhysicalID(p.TP()))
69+
result = util.EncodeIntAsUint32(result, p.QueryBlockOffset())
70+
result = util.EncodeIntAsUint32(result, len(p.Conditions))
7671

7772
condHashCodes := make([][]byte, len(p.Conditions))
7873
for i, expr := range p.Conditions {
@@ -82,7 +77,7 @@ func (p *LogicalSelection) HashCode() []byte {
8277
slices.SortFunc(condHashCodes, func(i, j []byte) int { return bytes.Compare(i, j) })
8378

8479
for _, condHashCode := range condHashCodes {
85-
result = encodeIntAsUint32(result, len(condHashCode))
80+
result = util.EncodeIntAsUint32(result, len(condHashCode))
8681
result = append(result, condHashCode...)
8782
}
8883
return result

pkg/planner/core/plan.go

Lines changed: 5 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -34,6 +34,10 @@ import (
3434
"github.com/pingcap/tidb/pkg/util/tracing"
3535
)
3636

37+
func init() {
38+
util.HasMaxOneRowUtil = HasMaxOneRow
39+
}
40+
3741
// AsSctx converts PlanContext to sessionctx.Context.
3842
func AsSctx(pctx base.PlanContext) (sessionctx.Context, error) {
3943
sctx, ok := pctx.(sessionctx.Context)
@@ -475,7 +479,7 @@ func (p *baseLogicalPlan) BuildKeyInfo(_ *expression.Schema, _ []*expression.Sch
475479
for i := range p.children {
476480
childMaxOneRow[i] = p.children[i].MaxOneRow()
477481
}
478-
p.maxOneRow = HasMaxOneRow(p.self, childMaxOneRow)
482+
p.maxOneRow = util.HasMaxOneRowUtil(p.self, childMaxOneRow)
479483
}
480484

481485
// BuildKeyInfo implements LogicalPlan BuildKeyInfo interface.

pkg/planner/memo/BUILD.bazel

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -16,6 +16,7 @@ go_library(
1616
"//pkg/planner/core/base",
1717
"//pkg/planner/pattern",
1818
"//pkg/planner/property",
19+
"//pkg/planner/util",
1920
],
2021
)
2122

pkg/planner/memo/group.go

Lines changed: 4 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -19,10 +19,12 @@ import (
1919
"fmt"
2020

2121
"github.com/pingcap/tidb/pkg/expression"
22-
plannercore "github.com/pingcap/tidb/pkg/planner/core"
22+
// import core pkg first to call its init func.
23+
_ "github.com/pingcap/tidb/pkg/planner/core"
2324
"github.com/pingcap/tidb/pkg/planner/core/base"
2425
"github.com/pingcap/tidb/pkg/planner/pattern"
2526
"github.com/pingcap/tidb/pkg/planner/property"
27+
"github.com/pingcap/tidb/pkg/planner/util"
2628
)
2729

2830
// ExploreMark is uses to mark whether a Group or GroupExpr has
@@ -221,5 +223,5 @@ func (g *Group) BuildKeyInfo() {
221223
g.Prop.Schema.Keys = childSchema[0].Keys
222224
}
223225
e.ExprNode.BuildKeyInfo(g.Prop.Schema, childSchema)
224-
g.Prop.MaxOneRow = e.ExprNode.MaxOneRow() || plannercore.HasMaxOneRow(e.ExprNode, childMaxOneRow)
226+
g.Prop.MaxOneRow = e.ExprNode.MaxOneRow() || util.HasMaxOneRowUtil(e.ExprNode, childMaxOneRow)
225227
}

pkg/planner/util/BUILD.bazel

Lines changed: 2 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -5,6 +5,7 @@ go_library(
55
srcs = [
66
"byitem.go",
77
"expression.go",
8+
"func_pointer_misc.go",
89
"handle_cols.go",
910
"misc.go",
1011
"null_misc.go",
@@ -19,6 +20,7 @@ go_library(
1920
"//pkg/parser/model",
2021
"//pkg/parser/mysql",
2122
"//pkg/planner/context",
23+
"//pkg/planner/core/base",
2224
"//pkg/sessionctx/stmtctx",
2325
"//pkg/tablecodec",
2426
"//pkg/types",

pkg/planner/util/func_pointer_misc.go

Lines changed: 25 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,25 @@
1+
// Copyright 2024 PingCAP, Inc.
2+
//
3+
// Licensed under the Apache License, Version 2.0 (the "License");
4+
// you may not use this file except in compliance with the License.
5+
// You may obtain a copy of the License at
6+
//
7+
// http://www.apache.org/licenses/LICENSE-2.0
8+
//
9+
// Unless required by applicable law or agreed to in writing, software
10+
// distributed under the License is distributed on an "AS IS" BASIS,
11+
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12+
// See the License for the specific language governing permissions and
13+
// limitations under the License.
14+
15+
package util
16+
17+
import "github.com/pingcap/tidb/pkg/planner/core/base"
18+
19+
// this file is used for passing function pointer at init(){} to avoid some import cycles.
20+
21+
// HasMaxOneRowUtil is used in baseLogicalPlan implementation of LogicalPlan interface, while
22+
// the original HasMaxOneRowUtil has some dependency of original core pkg: like Datasource which
23+
// hasn't been moved out of core pkg, so associative func pointer is introduced.
24+
// todo: (1) arenatlx, remove this func pointer when concrete Logical Operators moved out.
25+
var HasMaxOneRowUtil func(p base.LogicalPlan, childMaxOneRow []bool) bool

pkg/planner/util/misc.go

Lines changed: 8 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -15,6 +15,7 @@
1515
package util
1616

1717
import (
18+
"encoding/binary"
1819
"fmt"
1920
"time"
2021
"unsafe"
@@ -88,3 +89,10 @@ func (tr *QueryTimeRange) MemoryUsage() (sum int64) {
8889
}
8990
return emptyQueryTimeRangeSize
9091
}
92+
93+
// EncodeIntAsUint32 is used for LogicalPlan Interface
94+
func EncodeIntAsUint32(result []byte, value int) []byte {
95+
var buf [4]byte
96+
binary.BigEndian.PutUint32(buf[:], uint32(value))
97+
return append(result, buf[:]...)
98+
}

0 commit comments

Comments
 (0)