stats: use an alternative function to read the bound from BLOB stored in mysql.stats_buckets. (#59791)

close pingcap/tidb#59759
This commit is contained in:
YangKeao
2025-04-15 16:57:06 +08:00
committed by GitHub
parent 6ec0bd2c6f
commit 25dde1c45d
5 changed files with 95 additions and 5 deletions

View File

@ -9,7 +9,7 @@ go_test(
],
flaky = True,
race = "on",
shard_count = 34,
shard_count = 35,
deps = [
"//pkg/config",
"//pkg/domain",

View File

@ -1512,3 +1512,49 @@ func TestStatsCacheUpdateTimeout(t *testing.T) {
require.Equal(t, 6, int(globalStats2.RealtimeCount))
require.Equal(t, 2, int(globalStats2.ModifyCount))
}
func TestLoadStatsForBitColumn(t *testing.T) {
store, dom := testkit.CreateMockStoreAndDomain(t)
tk := testkit.NewTestKit(t, store)
tk.MustExec("use test")
testCases := []struct {
len int
lowerBound string
expectedLowerBoundHex string
upperBound string
expectedUpperBoundHex string
}{
// 0 -> 0 -> "0" -> 30
{1, "0", "30", "1", "31"},
{2, "2", "32", "3", "33"},
// "0" -> 48 -> "48" -> 3438
{6, `"0"`, "3438", `"1"`, "3439"},
// "a" -> 97 -> "97" -> 3937
{7, `"a"`, "3937", `"b"`, "3938"},
}
for i, testCase := range testCases {
tableName := fmt.Sprintf("t%d", i)
tk.MustExec(fmt.Sprintf("create table %s(a bit(%d));", tableName, testCase.len))
tbl, err := dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr(tableName))
require.NoError(t, err)
tk.MustExec(fmt.Sprintf("insert into %s values (%s), (%s);", tableName, testCase.lowerBound, testCase.upperBound))
tk.MustExec(fmt.Sprintf("analyze table %s all columns with 0 topn;", tableName))
h := dom.StatsHandle()
_, err = h.TableStatsFromStorage(tbl.Meta(), tbl.Meta().ID, true, 0)
require.NoError(t, err)
tk.MustQuery(
fmt.Sprintf("SELECT hex(lower_bound), hex(upper_bound) FROM mysql.stats_buckets WHERE table_id = %d ORDER BY lower_bound", tbl.Meta().ID),
).Check(testkit.Rows(
fmt.Sprintf("%s %s", testCase.expectedLowerBoundHex, testCase.expectedLowerBoundHex),
fmt.Sprintf("%s %s", testCase.expectedUpperBoundHex, testCase.expectedUpperBoundHex),
))
tk.MustExec("drop table " + tableName)
}
}

View File

@ -159,12 +159,12 @@ func HistogramFromStorageWithPriority(
if tp.EvalType() == types.ETString && tp.GetType() != mysql.TypeEnum && tp.GetType() != mysql.TypeSet {
tp = types.NewFieldType(mysql.TypeBlob)
}
lowerBound, err = d.ConvertTo(statistics.UTCWithAllowInvalidDateCtx, tp)
lowerBound, err = convertBoundFromBlob(statistics.UTCWithAllowInvalidDateCtx, d, tp)
if err != nil {
return nil, errors.Trace(err)
}
d = rows[i].GetDatum(3, &fields[3].Column.FieldType)
upperBound, err = d.ConvertTo(statistics.UTCWithAllowInvalidDateCtx, tp)
upperBound, err = convertBoundFromBlob(statistics.UTCWithAllowInvalidDateCtx, d, tp)
if err != nil {
return nil, errors.Trace(err)
}
@ -920,3 +920,37 @@ func StatsMetaByTableIDFromStorage(sctx sessionctx.Context, tableID int64, snaps
count = rows[0].GetInt64(2)
return
}
// convertBoundFromBlob reads the bound from blob. The `blob` is read from the `mysql.stats_buckets` table.
// The `convertBoundFromBlob(convertBoundToBlob(a))` should be equal to `a`.
// TODO: add a test to make sure that this assumption is correct.
func convertBoundFromBlob(ctx types.Context, blob types.Datum, tp *types.FieldType) (types.Datum, error) {
// For `BIT` type, when converting to `BLOB`, it's formated as an integer (when it's possible). Therefore, we should try to
// parse it as an integer first.
if tp.GetType() == mysql.TypeBit {
var ret types.Datum
// The implementation of converting BIT to BLOB will try to format it as an integer first. Theoretically, it should
// always be able to format the integer because the `BIT` length is limited to 64. Therefore, this err should never
// happen.
uintValue, err := strconv.ParseUint(string(blob.GetBytes()), 10, 64)
intest.AssertNoError(err)
if err != nil {
// Fail to parse, return the original blob as BIT directly.
ret.SetBinaryLiteral(types.BinaryLiteral(blob.GetBytes()))
return ret, nil
}
// part of the code is copied from `(*Datum).convertToMysqlBit`.
if tp.GetFlen() < 64 && uintValue >= 1<<(uint64(tp.GetFlen())) {
logutil.BgLogger().Warn("bound in stats exceeds the bit length", zap.Uint64("bound", uintValue), zap.Int("flen", tp.GetFlen()))
err = types.ErrDataTooLong.GenWithStack("Data Too Long, field len %d", tp.GetFlen())
intest.Assert(false, "bound in stats exceeds the bit length")
uintValue = (1 << (uint64(tp.GetFlen()))) - 1
}
byteSize := (tp.GetFlen() + 7) >> 3
ret.SetMysqlBit(types.NewBinaryLiteralFromUint(uintValue, byteSize))
return ret, errors.Trace(err)
}
return blob.ConvertTo(ctx, tp)
}

View File

@ -93,12 +93,12 @@ func saveBucketsToStorage(sctx sessionctx.Context, tableID int64, isIndex int, h
count -= hg.Buckets[j-1].Count
}
var upperBound types.Datum
upperBound, err = hg.GetUpper(j).ConvertTo(sc.TypeCtx(), types.NewFieldType(mysql.TypeBlob))
upperBound, err = convertBoundToBlob(sc.TypeCtx(), *hg.GetUpper(j))
if err != nil {
return
}
var lowerBound types.Datum
lowerBound, err = hg.GetLower(j).ConvertTo(sc.TypeCtx(), types.NewFieldType(mysql.TypeBlob))
lowerBound, err = convertBoundToBlob(sc.TypeCtx(), *hg.GetLower(j))
if err != nil {
return
}
@ -535,3 +535,10 @@ func InsertTableStats2KV(
}
return startTS, nil
}
// convertBoundToBlob converts the bound to blob. The `blob` will be used to store in the `mysql.stats_buckets` table.
// The `convertBoundFromBlob(convertBoundToBlob(a))` should be equal to `a`.
// TODO: add a test to make sure that this assumption is correct.
func convertBoundToBlob(ctx types.Context, d types.Datum) (types.Datum, error) {
return d.ConvertTo(ctx, types.NewFieldType(mysql.TypeBlob))
}

View File

@ -35,6 +35,7 @@ import (
"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/intest"
"github.com/pingcap/tidb/pkg/util/logutil"
"go.uber.org/zap"
)
@ -1188,6 +1189,8 @@ func (d *Datum) convertToString(ctx Context, target *FieldType) (Datum, error) {
// https://github.com/pingcap/tidb/issues/31124.
// Consider converting to uint first.
val, err := d.GetBinaryLiteral().ToInt(ctx)
// The length of BIT is limited to 64, so this function will never fail / truncated.
intest.AssertNoError(err)
if err != nil {
s = d.GetBinaryLiteral().ToString()
} else {