Skip to content

Commit

Permalink
util: let TypeFloat should be decoded as Float32 in Chunk (#35978) (#…
Browse files Browse the repository at this point in the history
  • Loading branch information
ti-srebot authored Aug 3, 2022
1 parent 392a844 commit dc157fc
Show file tree
Hide file tree
Showing 8 changed files with 781 additions and 0 deletions.
1 change: 1 addition & 0 deletions .github/licenserc.yml
Original file line number Diff line number Diff line change
Expand Up @@ -31,4 +31,5 @@ header:
- 'dumpling/'
- 'tidb-binlog/driver/example'
- 'tidb-binlog/proto/go-binlog/secondary_binlog.pb.go'
- '**/*.sql'
comment: on-failure
21 changes: 21 additions & 0 deletions executor/analyze_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@ import (
"context"
"encoding/json"
"fmt"
"io/ioutil"
"strconv"
"strings"
"sync"
Expand Down Expand Up @@ -3389,3 +3390,23 @@ PARTITION BY RANGE ( a ) (
tbl := h.GetTableStats(tableInfo)
require.Equal(t, int64(6), tbl.Columns[tableInfo.Columns[0].ID].Histogram.NDV)
}

func TestAnalyzePartitionTableForFloat(t *testing.T) {
store, clean := testkit.CreateMockStore(t)
defer clean()
tk := testkit.NewTestKit(t, store)
tk.MustExec("set @@tidb_partition_prune_mode='dynamic'")
tk.MustExec("use test")
tk.MustExec("CREATE TABLE t1 ( id bigint(20) unsigned NOT NULL AUTO_INCREMENT, num float(9,8) DEFAULT NULL, PRIMARY KEY (id) ) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin PARTITION BY HASH (id) PARTITIONS 128;")
// To reproduce the error we meet in https://github.com/pingcap/tidb/issues/35910, we should use the data provided in this issue
b, err := ioutil.ReadFile("testdata/analyze_test_data.sql")
require.NoError(t, err)
sqls := strings.Split(string(b), ";")
for _, sql := range sqls {
if len(sql) < 1 {
continue
}
tk.MustExec(sql)
}
tk.MustExec("analyze table t1")
}
726 changes: 726 additions & 0 deletions executor/testdata/analyze_test_data.sql

Large diffs are not rendered by default.

2 changes: 2 additions & 0 deletions statistics/cmsketch.go
Original file line number Diff line number Diff line change
Expand Up @@ -767,6 +767,8 @@ func MergePartTopN2GlobalTopN(sc *stmtctx.StatementContext, version int, topNs [
if types.IsTypeTime(hists[0].Tp.GetType()) {
// handle datetime values specially since they are encoded to int and we'll get int values if using DecodeOne.
_, d, err = codec.DecodeAsDateTime(val.Encoded, hists[0].Tp.GetType(), sc.TimeZone)
} else if types.IsTypeFloat(hists[0].Tp.GetType()) {
_, d, err = codec.DecodeAsFloat32(val.Encoded, hists[0].Tp.GetType())
} else {
_, d, err = codec.DecodeOne(val.Encoded)
}
Expand Down
2 changes: 2 additions & 0 deletions statistics/histogram.go
Original file line number Diff line number Diff line change
Expand Up @@ -2120,6 +2120,8 @@ func MergePartitionHist2GlobalHist(sc *stmtctx.StatementContext, hists []*Histog
if types.IsTypeTime(hists[0].Tp.GetType()) {
// handle datetime values specially since they are encoded to int and we'll get int values if using DecodeOne.
_, d, err = codec.DecodeAsDateTime(meta.Encoded, hists[0].Tp.GetType(), sc.TimeZone)
} else if types.IsTypeFloat(hists[0].Tp.GetType()) {
_, d, err = codec.DecodeAsFloat32(meta.Encoded, hists[0].Tp.GetType())
} else {
_, d, err = codec.DecodeOne(meta.Encoded)
}
Expand Down
6 changes: 6 additions & 0 deletions types/datum.go
Original file line number Diff line number Diff line change
Expand Up @@ -183,6 +183,12 @@ func (d *Datum) SetFloat32(f float32) {
d.i = int64(math.Float64bits(float64(f)))
}

// SetFloat32FromF64 sets float32 values from f64
func (d *Datum) SetFloat32FromF64(f float64) {
d.k = KindFloat32
d.i = int64(math.Float64bits(f))
}

// GetString gets string value.
func (d *Datum) GetString() string {
return string(hack.String(d.b))
Expand Down
5 changes: 5 additions & 0 deletions types/etc.go
Original file line number Diff line number Diff line change
Expand Up @@ -66,6 +66,11 @@ func IsTypeTime(tp byte) bool {
return tp == mysql.TypeDatetime || tp == mysql.TypeDate || tp == mysql.TypeTimestamp
}

// IsTypeFloat indicates whether the type is TypeFloat
func IsTypeFloat(tp byte) bool {
return tp == mysql.TypeFloat
}

// IsTypeInteger returns a boolean indicating whether the tp is integer type.
func IsTypeInteger(tp byte) bool {
switch tp {
Expand Down
18 changes: 18 additions & 0 deletions util/codec/codec.go
Original file line number Diff line number Diff line change
Expand Up @@ -762,6 +762,8 @@ func DecodeRange(b []byte, size int, idxColumnTypes []byte, loc *time.Location)
if types.IsTypeTime(idxColumnTypes[i]) {
// handle datetime values specially since they are encoded to int and we'll get int values if using DecodeOne.
b, d, err = DecodeAsDateTime(b, idxColumnTypes[i], loc)
} else if types.IsTypeFloat(idxColumnTypes[i]) {
b, d, err = DecodeAsFloat32(b, idxColumnTypes[i])
} else {
b, d, err = DecodeOne(b)
}
Expand Down Expand Up @@ -899,6 +901,22 @@ func DecodeAsDateTime(b []byte, tp byte, loc *time.Location) (remain []byte, d t
return b, d, nil
}

// DecodeAsFloat32 decodes value for mysql.TypeFloat
func DecodeAsFloat32(b []byte, tp byte) (remain []byte, d types.Datum, err error) {
if len(b) < 1 || tp != mysql.TypeFloat {
return nil, d, errors.New("invalid encoded key")
}
flag := b[0]
b = b[1:]
if flag != floatFlag {
return b, d, errors.Errorf("invalid encoded key flag %v for DecodeAsFloat32", flag)
}
var v float64
b, v, err = DecodeFloat(b)
d.SetFloat32FromF64(v)
return b, d, nil
}

// CutOne cuts the first encoded value from b.
// It will return the first encoded item and the remains as byte slice.
func CutOne(b []byte) (data []byte, remain []byte, err error) {
Expand Down

0 comments on commit dc157fc

Please sign in to comment.