Skip to content
Merged
Show file tree
Hide file tree
Changes from 1 commit
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
3 changes: 2 additions & 1 deletion pkg/ttl/cache/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -50,12 +50,13 @@ go_test(
],
embed = [":cache"],
flaky = True,
shard_count = 17,
shard_count = 18,
deps = [
"//pkg/infoschema",
"//pkg/kv",
"//pkg/parser/ast",
"//pkg/parser/model",
"//pkg/parser/mysql",
"//pkg/server",
"//pkg/session",
"//pkg/store/helper",
Expand Down
230 changes: 181 additions & 49 deletions pkg/ttl/cache/split_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -26,6 +26,7 @@ import (
"github.com/pingcap/tidb/pkg/infoschema"
"github.com/pingcap/tidb/pkg/kv"
"github.com/pingcap/tidb/pkg/parser/model"
"github.com/pingcap/tidb/pkg/parser/mysql"
"github.com/pingcap/tidb/pkg/store/helper"
"github.com/pingcap/tidb/pkg/tablecodec"
"github.com/pingcap/tidb/pkg/testkit"
Expand Down Expand Up @@ -250,21 +251,21 @@ func createTTLTableWithSQL(t *testing.T, tk *testkit.TestKit, name string, sql s
return ttlTbl
}

func checkRange(t *testing.T, r cache.ScanRange, start, end types.Datum) {
func checkRange(t *testing.T, r cache.ScanRange, start, end types.Datum, msgAndArgs ...any) {
if start.IsNull() {
require.Nil(t, r.Start)
require.Nil(t, r.Start, msgAndArgs...)
} else {
require.Equal(t, 1, len(r.Start))
require.Equal(t, start.Kind(), r.Start[0].Kind())
require.Equal(t, start.GetValue(), r.Start[0].GetValue())
require.Equal(t, 1, len(r.Start), msgAndArgs...)
require.Equal(t, start.Kind(), r.Start[0].Kind(), msgAndArgs...)
require.Equal(t, start.GetValue(), r.Start[0].GetValue(), msgAndArgs...)
}

if end.IsNull() {
require.Nil(t, r.End)
require.Nil(t, r.End, msgAndArgs...)
} else {
require.Equal(t, 1, len(r.End))
require.Equal(t, end.Kind(), r.End[0].Kind())
require.Equal(t, end.GetValue(), r.End[0].GetValue())
require.Equal(t, 1, len(r.End), msgAndArgs...)
require.Equal(t, end.Kind(), r.End[0].Kind(), msgAndArgs...)
require.Equal(t, end.GetValue(), r.End[0].GetValue(), msgAndArgs...)
}
}

Expand Down Expand Up @@ -516,47 +517,133 @@ func TestSplitTTLScanRangesWithBytes(t *testing.T) {
createTTLTable(t, tk, "t3", "varchar(32) CHARACTER SET BINARY"),
createTTLTable(t, tk, "t4", "bit(32)"),
create2PKTTLTable(t, tk, "t5", "binary(32)"),
createTTLTable(t, tk, "t6", "char(32) CHARACTER SET UTF8MB4"),
create2PKTTLTable(t, tk, "t7", "char(32) CHARACTER SET gbk"),
}

cases := []struct {
name string
regionEdges []kv.Handle
splitCnt int
binaryExpected [][]types.Datum
stringExpected [][]types.Datum
}{
{
name: "2 regions with binary split",
regionEdges: []kv.Handle{
bytesHandle(t, []byte{1, 2, 3}),
},
splitCnt: 4,
binaryExpected: [][]types.Datum{
{types.Datum{}, types.NewBytesDatum([]byte{1, 2, 3})},
{types.NewBytesDatum([]byte{1, 2, 3}), types.Datum{}},
},
stringExpected: [][]types.Datum{
{types.Datum{}, types.Datum{}},
},
},
{
name: "6 regions with binary split",
regionEdges: []kv.Handle{
bytesHandle(t, []byte{1, 2, 3}),
bytesHandle(t, []byte{1, 2, 3, 4}),
bytesHandle(t, []byte{1, 2, 3, 4, 5}),
bytesHandle(t, []byte{1, 2, 4}),
bytesHandle(t, []byte{1, 2, 5}),
},
splitCnt: 4,
binaryExpected: [][]types.Datum{
{types.Datum{}, types.NewBytesDatum([]byte{1, 2, 3, 4})},
{types.NewBytesDatum([]byte{1, 2, 3, 4}), types.NewBytesDatum([]byte{1, 2, 4})},
{types.NewBytesDatum([]byte{1, 2, 4}), types.NewBytesDatum([]byte{1, 2, 5})},
{types.NewBytesDatum([]byte{1, 2, 5}), types.Datum{}},
},
stringExpected: [][]types.Datum{
{types.Datum{}, types.Datum{}},
},
},
{
name: "2 regions with utf8 split",
regionEdges: []kv.Handle{
bytesHandle(t, []byte("中文")),
},
splitCnt: 4,
binaryExpected: [][]types.Datum{
{types.Datum{}, types.NewBytesDatum([]byte("中文"))},
{types.NewBytesDatum([]byte("中文")), types.Datum{}},
},
stringExpected: [][]types.Datum{
{types.Datum{}, types.Datum{}},
},
},
{
name: "several regions with mixed split",
regionEdges: []kv.Handle{
bytesHandle(t, []byte("abc")),
bytesHandle(t, []byte("ab\x7f0")),
bytesHandle(t, []byte("ab\xff0")),
bytesHandle(t, []byte("ac\x001")),
bytesHandle(t, []byte("ad\x0a1")),
bytesHandle(t, []byte("ad23")),
bytesHandle(t, []byte("ad230\xff")),
bytesHandle(t, []byte("befh")),
bytesHandle(t, []byte("中文")),
},
splitCnt: 10,
binaryExpected: [][]types.Datum{
{types.Datum{}, types.NewBytesDatum([]byte("abc"))},
{types.NewBytesDatum([]byte("abc")), types.NewBytesDatum([]byte("ab\x7f0"))},
{types.NewBytesDatum([]byte("ab\x7f0")), types.NewBytesDatum([]byte("ab\xff0"))},
{types.NewBytesDatum([]byte("ab\xff0")), types.NewBytesDatum([]byte("ac\x001"))},
{types.NewBytesDatum([]byte("ac\x001")), types.NewBytesDatum([]byte("ad\x0a1"))},
{types.NewBytesDatum([]byte("ad\x0a1")), types.NewBytesDatum([]byte("ad23"))},
{types.NewBytesDatum([]byte("ad23")), types.NewBytesDatum([]byte("ad230\xff"))},
{types.NewBytesDatum([]byte("ad230\xff")), types.NewBytesDatum([]byte("befh"))},
{types.NewBytesDatum([]byte("befh")), types.NewBytesDatum([]byte("中文"))},
{types.NewBytesDatum([]byte("中文")), types.Datum{}},
},
stringExpected: [][]types.Datum{
{types.Datum{}, types.NewStringDatum("abc")},
{types.NewStringDatum("abc"), types.NewStringDatum("ac")},
{types.NewStringDatum("ac"), types.NewStringDatum("ad\n1")},
{types.NewStringDatum("ad\n1"), types.NewStringDatum("ad23")},
{types.NewStringDatum("ad23"), types.NewStringDatum("ad230")},
{types.NewStringDatum("ad230"), types.NewStringDatum("befh")},
{types.NewStringDatum("befh"), types.Datum{}},
},
},
}

tikvStore := newMockTiKVStore(t)
for _, tbl := range tbls {
// test only one region
tikvStore.clearRegions()
ranges, err := tbl.SplitScanRanges(context.TODO(), tikvStore, 4)
require.NoError(t, err)
require.Equal(t, 1, len(ranges))
checkRange(t, ranges[0], types.Datum{}, types.Datum{})

// test share regions with other table
tikvStore.clearRegions()
tikvStore.addRegion(
tablecodec.GenTablePrefix(tbl.ID-1),
tablecodec.GenTablePrefix(tbl.ID+1),
)
ranges, err = tbl.SplitScanRanges(context.TODO(), tikvStore, 4)
require.NoError(t, err)
require.Equal(t, 1, len(ranges))
checkRange(t, ranges[0], types.Datum{}, types.Datum{})
for _, c := range cases {
tikvStore.clearRegions()
require.Greater(t, len(c.regionEdges), 0)
for i, edge := range c.regionEdges {
if i == 0 {
tikvStore.addRegionBeginWithTablePrefix(tbl.ID, edge)
} else {
tikvStore.addRegionWithTablePrefix(tbl.ID, c.regionEdges[i-1], edge)
}
}
tikvStore.addRegionEndWithTablePrefix(c.regionEdges[len(c.regionEdges)-1], tbl.ID)
ranges, err := tbl.SplitScanRanges(context.TODO(), tikvStore, c.splitCnt)
require.NoError(t, err)

keyTp := tbl.KeyColumnTypes[0]
var expected [][]types.Datum
if keyTp.GetType() == mysql.TypeBit || mysql.HasBinaryFlag(keyTp.GetFlag()) {
expected = c.binaryExpected
} else {
expected = c.stringExpected
}

// test one table has multiple regions
tikvStore.clearRegions()
tikvStore.addRegionBeginWithTablePrefix(tbl.ID, bytesHandle(t, []byte{1, 2, 3}))
tikvStore.addRegionWithTablePrefix(
tbl.ID, bytesHandle(t, []byte{1, 2, 3}), bytesHandle(t, []byte{1, 2, 3, 4}))
tikvStore.addRegionWithTablePrefix(
tbl.ID, bytesHandle(t, []byte{1, 2, 3, 4}), bytesHandle(t, []byte{1, 2, 3, 4, 5}))
tikvStore.addRegionWithTablePrefix(
tbl.ID, bytesHandle(t, []byte{1, 2, 3, 4, 5}), bytesHandle(t, []byte{1, 2, 4}))
tikvStore.addRegionWithTablePrefix(
tbl.ID, bytesHandle(t, []byte{1, 2, 4}), bytesHandle(t, []byte{1, 2, 5}))
tikvStore.addRegionEndWithTablePrefix(bytesHandle(t, []byte{1, 2, 5}), tbl.ID)
ranges, err = tbl.SplitScanRanges(context.TODO(), tikvStore, 4)
require.NoError(t, err)
require.Equal(t, 4, len(ranges))
checkRange(t, ranges[0], types.Datum{}, types.NewBytesDatum([]byte{1, 2, 3, 4}))
checkRange(t, ranges[1], types.NewBytesDatum([]byte{1, 2, 3, 4}), types.NewBytesDatum([]byte{1, 2, 4}))
checkRange(t, ranges[2], types.NewBytesDatum([]byte{1, 2, 4}), types.NewBytesDatum([]byte{1, 2, 5}))
checkRange(t, ranges[3], types.NewBytesDatum([]byte{1, 2, 5}), types.Datum{})
require.Equal(t, len(expected), len(ranges), "tbl: %s, case: %s", tbl.Name, c.name)
for i, r := range ranges {
checkRange(t, r, expected[i][0], expected[i][1],
"tbl: %s, case: %s, i: %d", tbl.Name, c.name, i)
}
}
}
}

Expand All @@ -565,10 +652,10 @@ func TestNoTTLSplitSupportTables(t *testing.T) {
tk := testkit.NewTestKit(t, store)

tbls := []*cache.PhysicalTable{
createTTLTable(t, tk, "t1", "char(32) CHARACTER SET UTF8MB4"),
createTTLTable(t, tk, "t2", "varchar(32) CHARACTER SET UTF8MB4"),
createTTLTable(t, tk, "t4", "decimal(32, 2)"),
create2PKTTLTable(t, tk, "t5", "char(32) CHARACTER SET UTF8MB4"),
createTTLTable(t, tk, "t1", "decimal(32, 2)"),
createTTLTable(t, tk, "t2", "date"),
createTTLTable(t, tk, "t3", "datetime"),
createTTLTable(t, tk, "t4", "timestamp"),
}

tikvStore := newMockTiKVStore(t)
Expand Down Expand Up @@ -827,6 +914,51 @@ func TestGetNextBytesHandleDatum(t *testing.T) {
}
}

func TestGetASCIIPrefixDatumFromBytes(t *testing.T) {
cases := []struct {
bytes []byte
expected string
}{
{bytes: nil, expected: ""},
{bytes: []byte{}, expected: ""},
{bytes: []byte{0}, expected: ""},
{bytes: []byte{1}, expected: ""},
{bytes: []byte{8}, expected: ""},
{bytes: []byte{9}, expected: "\t"},
{bytes: []byte{10}, expected: "\n"},
{bytes: []byte{11}, expected: ""},
{bytes: []byte{12}, expected: ""},
{bytes: []byte{13}, expected: "\r"},
{bytes: []byte{14}, expected: ""},
{bytes: []byte{0x19}, expected: ""},
{bytes: []byte{0x20}, expected: " "},
{bytes: []byte{0x21}, expected: "!"},
{bytes: []byte{0x7D}, expected: "}"},
{bytes: []byte{0x7E}, expected: "~"},
{bytes: []byte{0x7F}, expected: ""},
{bytes: []byte{0xFF}, expected: ""},
{bytes: []byte{0x0, 'a', 'b'}, expected: ""},
{bytes: []byte{0xFF, 'a', 'b'}, expected: ""},
{bytes: []byte{'0', '1', 0x0, 'a', 'b'}, expected: "01"},
{bytes: []byte{'0', '1', 0x15, 'a', 'b'}, expected: "01"},
{bytes: []byte{'0', '1', 0xFF, 'a', 'b'}, expected: "01"},
{bytes: []byte{'a', 'b', 0x0}, expected: "ab"},
{bytes: []byte{'a', 'b', 0x15}, expected: "ab"},
{bytes: []byte{'a', 'b', 0xFF}, expected: "ab"},
{bytes: []byte("ab\rcd\tef\nAB!~GH()tt ;;"), expected: "ab\rcd\tef\nAB!~GH()tt ;;"},
{bytes: []byte("中文"), expected: ""},
{bytes: []byte("cn中文"), expected: "cn"},
{bytes: []byte("😀"), expected: ""},
{bytes: []byte("emoji😀"), expected: "emoji"},
}

for i, c := range cases {
d := cache.GetASCIIPrefixDatumFromBytes(c.bytes)
require.Equalf(t, types.KindString, d.Kind(), "i: %d", i)
require.Equalf(t, c.expected, d.GetString(), "i: %d, bs: %v", i, c.bytes)
}
}

func TestGetNextIntHandle(t *testing.T) {
tblID := int64(7)
cases := []struct {
Expand Down
51 changes: 41 additions & 10 deletions pkg/ttl/cache/table.go
Original file line number Diff line number Diff line change
Expand Up @@ -287,15 +287,13 @@ func (t *PhysicalTable) SplitScanRanges(ctx context.Context, store kv.Storage, s
switch ft.GetType() {
case mysql.TypeTiny, mysql.TypeShort, mysql.TypeLong, mysql.TypeLonglong, mysql.TypeInt24:
if len(t.KeyColumns) > 1 {
return t.splitCommonHandleRanges(ctx, tikvStore, splitCnt, true, mysql.HasUnsignedFlag(ft.GetFlag()))
return t.splitCommonHandleRanges(ctx, tikvStore, splitCnt, true, mysql.HasUnsignedFlag(ft.GetFlag()), false)
}
return t.splitIntRanges(ctx, tikvStore, splitCnt)
case mysql.TypeBit:
return t.splitCommonHandleRanges(ctx, tikvStore, splitCnt, false, false)
return t.splitCommonHandleRanges(ctx, tikvStore, splitCnt, false, false, true)
case mysql.TypeString, mysql.TypeVarString, mysql.TypeVarchar:
if mysql.HasBinaryFlag(ft.GetFlag()) {
return t.splitCommonHandleRanges(ctx, tikvStore, splitCnt, false, false)
}
return t.splitCommonHandleRanges(ctx, tikvStore, splitCnt, false, false, mysql.HasBinaryFlag(ft.GetFlag()))
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Should it only accept utf8mb4_bin (or other binary collations)?

For utf8mb4_unicode_ci (or other UCA collations), there is no direct connection between the character and the weight. It's meaningless to get the ASCII prefix.

Copy link
Member

@YangKeao YangKeao Aug 26, 2024

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

For example, the ASCII string a has weight 0x0E33.

Copy link
Collaborator Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Done, PTAL again

}
return []ScanRange{newFullRange()}, nil
}
Expand Down Expand Up @@ -365,7 +363,7 @@ func (t *PhysicalTable) splitIntRanges(ctx context.Context, store tikv.Storage,
}

func (t *PhysicalTable) splitCommonHandleRanges(
ctx context.Context, store tikv.Storage, splitCnt int, isInt bool, unsigned bool,
ctx context.Context, store tikv.Storage, splitCnt int, isInt bool, unsigned bool, binary bool,
) ([]ScanRange, error) {
recordPrefix := tablecodec.GenTableRecordPrefix(t.ID)
startKey, endKey := recordPrefix, recordPrefix.PrefixNext()
Expand All @@ -381,20 +379,26 @@ func (t *PhysicalTable) splitCommonHandleRanges(
scanRanges := make([]ScanRange, 0, len(keyRanges))
curScanStart := nullDatum()
for i, keyRange := range keyRanges {
if i != 0 && curScanStart.IsNull() {
break
}

curScanEnd := nullDatum()
if i != len(keyRanges)-1 {
if isInt {
curScanEnd = GetNextIntDatumFromCommonHandle(keyRange.EndKey, recordPrefix, unsigned)
} else {
curScanEnd = GetNextBytesHandleDatum(keyRange.EndKey, recordPrefix)
if !binary {
curScanEnd = GetASCIIPrefixDatumFromBytes(curScanEnd.GetBytes())
}

// "" is the smallest value for string/[]byte, skip to add it to ranges.
if len(curScanEnd.GetBytes()) == 0 {
continue
}
}
}

if !curScanStart.IsNull() && !curScanEnd.IsNull() {
// Sometimes curScanStart >= curScanEnd because the edge datum is an approximate value.
// At this time, we should skip this range to ensure the incremental of ranges.
cmp, err := curScanStart.Compare(types.StrictContext, &curScanEnd, collate.GetBinaryCollator())
intest.AssertNoError(err)
if err != nil {
Expand All @@ -407,6 +411,9 @@ func (t *PhysicalTable) splitCommonHandleRanges(
}

scanRanges = append(scanRanges, newDatumRange(curScanStart, curScanEnd))
if curScanEnd.IsNull() {
break
}
curScanStart = curScanEnd
}
return scanRanges, nil
Expand Down Expand Up @@ -648,3 +655,27 @@ func GetNextBytesHandleDatum(key kv.Key, recordPrefix []byte) (d types.Datum) {
d.SetBytes(val)
return d
}

// GetASCIIPrefixDatumFromBytes is used to convert bytes to string datum which only contains ASCII prefix string.
// The ASCII prefix string only contains visible characters and `\t`, `\n`, `\r`.
// "abc" -> "abc"
// "\0abc" -> ""
// "ab\x01c" -> "ab"
// "ab\xffc" -> "ab"
// "ab\rc\xff" -> "ab\rc"
func GetASCIIPrefixDatumFromBytes(bs []byte) types.Datum {
for i, c := range bs {
if c >= 0x20 && c <= 0x7E {
// visible characters from ` ` to `~`
continue
}

if c == '\t' || c == '\n' || c == '\r' {
continue
}

bs = bs[:i]
break
}
return types.NewStringDatum(string(bs))
}
Loading