Skip to content

Commit

Permalink
planner: introduce hashEqual interface for datum. (#55727)
Browse files Browse the repository at this point in the history
ref #51664
  • Loading branch information
AilinKid authored Aug 29, 2024
1 parent aeefe2e commit f5fff6c
Show file tree
Hide file tree
Showing 6 changed files with 117 additions and 0 deletions.
16 changes: 16 additions & 0 deletions pkg/planner/cascades/base/hash_equaler.go
Original file line number Diff line number Diff line change
Expand Up @@ -41,6 +41,8 @@ type Hasher interface {
HashByte(val byte)
HashBytes(val []byte)
Reset()
SetCache([]byte)
Cache() []byte
Sum64() uint64
}

Expand All @@ -53,6 +55,9 @@ type Hash64a uint64
type hasher struct {
// hash stores the hash value as it is incrementally computed.
hash64a Hash64a

// cache is the internal bytes slice that's will be reused for some special tmp encoding like datum.
cache []byte
}

// NewHashEqualer creates a new HashEqualer.
Expand All @@ -65,6 +70,17 @@ func NewHashEqualer() Hasher {
// Reset resets the Hasher to its initial state, reusing the internal bytes slice.
func (h *hasher) Reset() {
h.hash64a = offset64
h.cache = h.cache[:0]
}

// Cache returns the internal bytes slice for re-usage.
func (h *hasher) Cache() []byte {
return h.cache
}

// SetCache sets the internal bytes slice for reu-sage.
func (h *hasher) SetCache(cache []byte) {
h.cache = cache
}

func (h *hasher) Sum64() uint64 {
Expand Down
1 change: 1 addition & 0 deletions pkg/types/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -53,6 +53,7 @@ go_library(
"//pkg/parser/opcode",
"//pkg/parser/terror",
"//pkg/parser/types",
"//pkg/planner/cascades/base",
"//pkg/util/collate",
"//pkg/util/context",
"//pkg/util/dbterror",
Expand Down
43 changes: 43 additions & 0 deletions pkg/types/datum.go
Original file line number Diff line number Diff line change
Expand Up @@ -32,6 +32,7 @@ import (
"github.com/pingcap/tidb/pkg/parser/mysql"
"github.com/pingcap/tidb/pkg/parser/terror"
"github.com/pingcap/tidb/pkg/parser/types"
"github.com/pingcap/tidb/pkg/planner/cascades/base"
"github.com/pingcap/tidb/pkg/util/collate"
"github.com/pingcap/tidb/pkg/util/hack"
"github.com/pingcap/tidb/pkg/util/logutil"
Expand Down Expand Up @@ -651,6 +652,48 @@ func (d *Datum) SetValue(val any, tp *types.FieldType) {
}
}

// Hash64ForDatum is a hash function for initialized by codec package.
var Hash64ForDatum func(h base.Hasher, d *Datum)

// Hash64 implements base.HashEquals<0th> interface.
func (d *Datum) Hash64(h base.Hasher) {
Hash64ForDatum(h, d)
}

// Equals implements base.HashEquals.<1st> interface.
func (d *Datum) Equals(other any) bool {
if other == nil {
return false
}
var d2 *Datum
switch x := other.(type) {
case *Datum:
d2 = x
case Datum:
d2 = &x
default:
return false
}
ok := d.k == d2.k &&
d.decimal == d2.decimal &&
d.length == d2.length &&
d.i == d2.i &&
d.collation == d2.collation &&
string(d.b) == string(d2.b)
if !ok {
return false
}
// compare x
switch d.k {
case KindMysqlDecimal:
return d.GetMysqlDecimal().Compare(d2.GetMysqlDecimal()) == 0
case KindMysqlTime:
return d.GetMysqlTime().Compare(d2.GetMysqlTime()) == 0
default:
return true
}
}

// Compare compares datum to another datum.
// Notes: don't rely on datum.collation to get the collator, it's tend to buggy.
func (d *Datum) Compare(ctx Context, ad *Datum, comparer collate.Collator) (int, error) {
Expand Down
2 changes: 2 additions & 0 deletions pkg/util/codec/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -14,6 +14,7 @@ go_library(
deps = [
"//pkg/parser/mysql",
"//pkg/parser/terror",
"//pkg/planner/cascades/base",
"//pkg/types",
"//pkg/util/chunk",
"//pkg/util/collate",
Expand Down Expand Up @@ -42,6 +43,7 @@ go_test(
"//pkg/errctx",
"//pkg/parser/mysql",
"//pkg/parser/terror",
"//pkg/planner/cascades/base",
"//pkg/testkit/testsetup",
"//pkg/types",
"//pkg/util/benchdaily",
Expand Down
15 changes: 15 additions & 0 deletions pkg/util/codec/codec.go
Original file line number Diff line number Diff line change
Expand Up @@ -25,6 +25,7 @@ import (
"github.com/pingcap/errors"
"github.com/pingcap/tidb/pkg/parser/mysql"
"github.com/pingcap/tidb/pkg/parser/terror"
"github.com/pingcap/tidb/pkg/planner/cascades/base"
"github.com/pingcap/tidb/pkg/types"
"github.com/pingcap/tidb/pkg/util/chunk"
"github.com/pingcap/tidb/pkg/util/collate"
Expand Down Expand Up @@ -1586,6 +1587,20 @@ func ConvertByCollationStr(str string, tp *types.FieldType) string {
return string(hack.String(collator.Key(str)))
}

// Hash64 is for datum hash64 calculation.
func Hash64(h base.Hasher, d *types.Datum) {
// let h.cache to receive datum hash value, which is potentially expendable.
// clean the cache before using it.
b := h.Cache()[:0]
b = HashCode(b, *d)
h.HashBytes(b)
h.SetCache(b)
}

func init() {
types.Hash64ForDatum = Hash64
}

// HashCode encodes a Datum into a unique byte slice.
// It is mostly the same as EncodeValue, but it doesn't contain truncation or verification logic in order to make the encoding lossless.
func HashCode(b []byte, d types.Datum) []byte {
Expand Down
40 changes: 40 additions & 0 deletions pkg/util/codec/codec_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -27,6 +27,7 @@ import (
"github.com/pingcap/tidb/pkg/errctx"
"github.com/pingcap/tidb/pkg/parser/mysql"
"github.com/pingcap/tidb/pkg/parser/terror"
"github.com/pingcap/tidb/pkg/planner/cascades/base"
"github.com/pingcap/tidb/pkg/types"
"github.com/pingcap/tidb/pkg/util/chunk"
"github.com/pingcap/tidb/pkg/util/collate"
Expand Down Expand Up @@ -1293,3 +1294,42 @@ func TestHashChunkColumns(t *testing.T) {
require.Equal(t, rowHash[2].Sum64(), vecHash[2].Sum64())
}
}

func TestDatumHashEquals(t *testing.T) {
now := time.Now()
tests := []struct {
d1 types.Datum
d2 types.Datum
}{
{types.NewIntDatum(1), types.NewIntDatum(1)},
{types.NewUintDatum(1), types.NewUintDatum(1)},
{types.NewFloat64Datum(1.1), types.NewFloat64Datum(1.1)},
{types.NewStringDatum("abc"), types.NewStringDatum("abc")},
{types.NewBytesDatum([]byte("abc")), types.NewBytesDatum([]byte("abc"))},
{types.NewMysqlEnumDatum(types.Enum{Name: "a", Value: 1}), types.NewMysqlEnumDatum(types.Enum{Name: "a", Value: 1})},
{types.NewMysqlSetDatum(types.Set{Name: "a", Value: 1}, "a"), types.NewMysqlSetDatum(types.Set{Name: "a", Value: 1}, "a")},
{types.NewBinaryLiteralDatum([]byte{0x01}), types.NewBinaryLiteralDatum([]byte{0x01})},
{types.NewMysqlBitDatum(types.NewBinaryLiteralFromUint(1, -1)), types.NewMysqlBitDatum(types.NewBinaryLiteralFromUint(1, -1))},
{types.NewTimeDatum(types.NewTime(types.FromGoTime(now), mysql.TypeDatetime, 6)), types.NewTimeDatum(types.NewTime(types.FromGoTime(now), mysql.TypeDatetime, 6))},
{types.NewDurationDatum(types.Duration{Duration: time.Second}), types.NewDurationDatum(types.Duration{Duration: time.Second})},
{types.NewJSONDatum(types.CreateBinaryJSON("a")), types.NewJSONDatum(types.CreateBinaryJSON("a"))},
{types.NewTimeDatum(types.NewTime(types.FromGoTime(now), mysql.TypeDatetime, 6)), types.NewTimeDatum(types.NewTime(types.FromGoTime(time.Now()), mysql.TypeDatetime, 6))},
}
hasher1 := base.NewHashEqualer()
hasher2 := base.NewHashEqualer()
for _, tt := range tests[:(len(tests) - 1)] {
hasher1.Reset()
hasher2.Reset()
tt.d1.Hash64(hasher1)
tt.d2.Hash64(hasher2)
require.Equal(t, hasher1.Sum64(), hasher2.Sum64())
require.True(t, tt.d1.Equals(tt.d2))
}
// the last test case is for the case that two datums are not equal
hasher1.Reset()
hasher2.Reset()
tests[len(tests)-1].d1.Hash64(hasher1)
tests[len(tests)-1].d2.Hash64(hasher2)
require.NotEqual(t, hasher1.Sum64(), hasher2.Sum64())
require.False(t, tests[len(tests)-1].d1.Equals(tests[len(tests)-1].d2))
}

0 comments on commit f5fff6c

Please sign in to comment.