Skip to content

Commit

Permalink
planner: move hash encoding logic and HasMaxOneRow to util (pingcap#5…
Browse files Browse the repository at this point in the history
  • Loading branch information
AilinKid authored and RidRisR committed May 23, 2024
1 parent 674d76f commit bad7334
Show file tree
Hide file tree
Showing 7 changed files with 58 additions and 21 deletions.
31 changes: 13 additions & 18 deletions pkg/planner/core/hashcode.go
Original file line number Diff line number Diff line change
Expand Up @@ -19,21 +19,16 @@ import (
"encoding/binary"
"slices"

"github.com/pingcap/tidb/pkg/planner/util"
"github.com/pingcap/tidb/pkg/util/plancodec"
)

func encodeIntAsUint32(result []byte, value int) []byte {
var buf [4]byte
binary.BigEndian.PutUint32(buf[:], uint32(value))
return append(result, buf[:]...)
}

// HashCode implements LogicalPlan interface.
func (p *baseLogicalPlan) HashCode() []byte {
// We use PlanID for the default hash, so if two plans do not have
// the same id, the hash value will never be the same.
result := make([]byte, 0, 4)
result = encodeIntAsUint32(result, p.ID())
result = util.EncodeIntAsUint32(result, p.ID())
return result
}

Expand All @@ -43,12 +38,12 @@ func (p *LogicalProjection) HashCode() []byte {
// Expressions are commonly `Column`s, whose hashcode has the length 9, so
// we pre-alloc 10 bytes for each expr's hashcode.
result := make([]byte, 0, 12+len(p.Exprs)*10)
result = encodeIntAsUint32(result, plancodec.TypeStringToPhysicalID(p.TP()))
result = encodeIntAsUint32(result, p.QueryBlockOffset())
result = encodeIntAsUint32(result, len(p.Exprs))
result = util.EncodeIntAsUint32(result, plancodec.TypeStringToPhysicalID(p.TP()))
result = util.EncodeIntAsUint32(result, p.QueryBlockOffset())
result = util.EncodeIntAsUint32(result, len(p.Exprs))
for _, expr := range p.Exprs {
exprHashCode := expr.HashCode()
result = encodeIntAsUint32(result, len(exprHashCode))
result = util.EncodeIntAsUint32(result, len(exprHashCode))
result = append(result, exprHashCode...)
}
return result
Expand All @@ -58,9 +53,9 @@ func (p *LogicalProjection) HashCode() []byte {
func (p *LogicalTableDual) HashCode() []byte {
// PlanType + SelectOffset + RowCount
result := make([]byte, 0, 12)
result = encodeIntAsUint32(result, plancodec.TypeStringToPhysicalID(p.TP()))
result = encodeIntAsUint32(result, p.QueryBlockOffset())
result = encodeIntAsUint32(result, p.RowCount)
result = util.EncodeIntAsUint32(result, plancodec.TypeStringToPhysicalID(p.TP()))
result = util.EncodeIntAsUint32(result, p.QueryBlockOffset())
result = util.EncodeIntAsUint32(result, p.RowCount)
return result
}

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

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

for _, condHashCode := range condHashCodes {
result = encodeIntAsUint32(result, len(condHashCode))
result = util.EncodeIntAsUint32(result, len(condHashCode))
result = append(result, condHashCode...)
}
return result
Expand Down
6 changes: 5 additions & 1 deletion pkg/planner/core/plan.go
Original file line number Diff line number Diff line change
Expand Up @@ -34,6 +34,10 @@ import (
"github.com/pingcap/tidb/pkg/util/tracing"
)

func init() {
util.HasMaxOneRowUtil = HasMaxOneRow
}

// AsSctx converts PlanContext to sessionctx.Context.
func AsSctx(pctx base.PlanContext) (sessionctx.Context, error) {
sctx, ok := pctx.(sessionctx.Context)
Expand Down Expand Up @@ -475,7 +479,7 @@ func (p *baseLogicalPlan) BuildKeyInfo(_ *expression.Schema, _ []*expression.Sch
for i := range p.children {
childMaxOneRow[i] = p.children[i].MaxOneRow()
}
p.maxOneRow = HasMaxOneRow(p.self, childMaxOneRow)
p.maxOneRow = util.HasMaxOneRowUtil(p.self, childMaxOneRow)
}

// BuildKeyInfo implements LogicalPlan BuildKeyInfo interface.
Expand Down
1 change: 1 addition & 0 deletions pkg/planner/memo/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -16,6 +16,7 @@ go_library(
"//pkg/planner/core/base",
"//pkg/planner/pattern",
"//pkg/planner/property",
"//pkg/planner/util",
],
)

Expand Down
6 changes: 4 additions & 2 deletions pkg/planner/memo/group.go
Original file line number Diff line number Diff line change
Expand Up @@ -19,10 +19,12 @@ import (
"fmt"

"github.com/pingcap/tidb/pkg/expression"
plannercore "github.com/pingcap/tidb/pkg/planner/core"
// import core pkg first to call its init func.
_ "github.com/pingcap/tidb/pkg/planner/core"
"github.com/pingcap/tidb/pkg/planner/core/base"
"github.com/pingcap/tidb/pkg/planner/pattern"
"github.com/pingcap/tidb/pkg/planner/property"
"github.com/pingcap/tidb/pkg/planner/util"
)

// ExploreMark is uses to mark whether a Group or GroupExpr has
Expand Down Expand Up @@ -221,5 +223,5 @@ func (g *Group) BuildKeyInfo() {
g.Prop.Schema.Keys = childSchema[0].Keys
}
e.ExprNode.BuildKeyInfo(g.Prop.Schema, childSchema)
g.Prop.MaxOneRow = e.ExprNode.MaxOneRow() || plannercore.HasMaxOneRow(e.ExprNode, childMaxOneRow)
g.Prop.MaxOneRow = e.ExprNode.MaxOneRow() || util.HasMaxOneRowUtil(e.ExprNode, childMaxOneRow)
}
2 changes: 2 additions & 0 deletions pkg/planner/util/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -5,6 +5,7 @@ go_library(
srcs = [
"byitem.go",
"expression.go",
"func_pointer_misc.go",
"handle_cols.go",
"misc.go",
"null_misc.go",
Expand All @@ -19,6 +20,7 @@ go_library(
"//pkg/parser/model",
"//pkg/parser/mysql",
"//pkg/planner/context",
"//pkg/planner/core/base",
"//pkg/sessionctx/stmtctx",
"//pkg/tablecodec",
"//pkg/types",
Expand Down
25 changes: 25 additions & 0 deletions pkg/planner/util/func_pointer_misc.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,25 @@
// 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 util

import "github.com/pingcap/tidb/pkg/planner/core/base"

// this file is used for passing function pointer at init(){} to avoid some import cycles.

// HasMaxOneRowUtil is used in baseLogicalPlan implementation of LogicalPlan interface, while
// the original HasMaxOneRowUtil has some dependency of original core pkg: like Datasource which
// hasn't been moved out of core pkg, so associative func pointer is introduced.
// todo: (1) arenatlx, remove this func pointer when concrete Logical Operators moved out.
var HasMaxOneRowUtil func(p base.LogicalPlan, childMaxOneRow []bool) bool
8 changes: 8 additions & 0 deletions pkg/planner/util/misc.go
Original file line number Diff line number Diff line change
Expand Up @@ -15,6 +15,7 @@
package util

import (
"encoding/binary"
"fmt"
"time"
"unsafe"
Expand Down Expand Up @@ -88,3 +89,10 @@ func (tr *QueryTimeRange) MemoryUsage() (sum int64) {
}
return emptyQueryTimeRangeSize
}

// EncodeIntAsUint32 is used for LogicalPlan Interface
func EncodeIntAsUint32(result []byte, value int) []byte {
var buf [4]byte
binary.BigEndian.PutUint32(buf[:], uint32(value))
return append(result, buf[:]...)
}

0 comments on commit bad7334

Please sign in to comment.