Skip to content

Commit

Permalink
Merge branch 'master' into issue39892
Browse files Browse the repository at this point in the history
  • Loading branch information
AilinKid authored Dec 14, 2022
2 parents 8708b2d + b41be06 commit 41453a2
Show file tree
Hide file tree
Showing 14 changed files with 328 additions and 20 deletions.
14 changes: 14 additions & 0 deletions .github/dependabot.yml
Original file line number Diff line number Diff line change
@@ -0,0 +1,14 @@
version: 2
updates:
- package-ecosystem: "gomod"
directory: "/"
schedule:
interval: "weekly"
day: "friday"
time: "18:00"
timezone: "Asia/Shanghai"
allow:
- dependency-name: "golang.org/*"
- dependency-name: "github.com/golangci/golangci-lint"
open-pull-requests-limit: 2

2 changes: 1 addition & 1 deletion executor/showtest/show_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1515,7 +1515,7 @@ func TestShowBuiltin(t *testing.T) {
res := tk.MustQuery("show builtins;")
require.NotNil(t, res)
rows := res.Rows()
const builtinFuncNum = 283
const builtinFuncNum = 284
require.Equal(t, builtinFuncNum, len(rows))
require.Equal(t, rows[0][0].(string), "abs")
require.Equal(t, rows[builtinFuncNum-1][0].(string), "yearweek")
Expand Down
1 change: 1 addition & 0 deletions expression/builtin.go
Original file line number Diff line number Diff line change
Expand Up @@ -874,6 +874,7 @@ var funcs = map[string]functionClass{
ast.JSONObject: &jsonObjectFunctionClass{baseFunctionClass{ast.JSONObject, 0, -1}},
ast.JSONArray: &jsonArrayFunctionClass{baseFunctionClass{ast.JSONArray, 0, -1}},
ast.JSONContains: &jsonContainsFunctionClass{baseFunctionClass{ast.JSONContains, 2, 3}},
ast.JSONOverlaps: &jsonOverlapsFunctionClass{baseFunctionClass{ast.JSONOverlaps, 2, 2}},
ast.JSONContainsPath: &jsonContainsPathFunctionClass{baseFunctionClass{ast.JSONContainsPath, 3, -1}},
ast.JSONValid: &jsonValidFunctionClass{baseFunctionClass{ast.JSONValid, 1, 1}},
ast.JSONArrayAppend: &jsonArrayAppendFunctionClass{baseFunctionClass{ast.JSONArrayAppend, 3, -1}},
Expand Down
58 changes: 58 additions & 0 deletions expression/builtin_json.go
Original file line number Diff line number Diff line change
Expand Up @@ -44,6 +44,7 @@ var (
_ functionClass = &jsonObjectFunctionClass{}
_ functionClass = &jsonArrayFunctionClass{}
_ functionClass = &jsonContainsFunctionClass{}
_ functionClass = &jsonOverlapsFunctionClass{}
_ functionClass = &jsonContainsPathFunctionClass{}
_ functionClass = &jsonValidFunctionClass{}
_ functionClass = &jsonArrayAppendFunctionClass{}
Expand Down Expand Up @@ -72,6 +73,7 @@ var (
_ builtinFunc = &builtinJSONRemoveSig{}
_ builtinFunc = &builtinJSONMergeSig{}
_ builtinFunc = &builtinJSONContainsSig{}
_ builtinFunc = &builtinJSONOverlapsSig{}
_ builtinFunc = &builtinJSONStorageSizeSig{}
_ builtinFunc = &builtinJSONDepthSig{}
_ builtinFunc = &builtinJSONSearchSig{}
Expand Down Expand Up @@ -820,6 +822,62 @@ func (b *builtinJSONContainsSig) evalInt(row chunk.Row) (res int64, isNull bool,
return 0, false, nil
}

type jsonOverlapsFunctionClass struct {
baseFunctionClass
}

type builtinJSONOverlapsSig struct {
baseBuiltinFunc
}

func (b *builtinJSONOverlapsSig) Clone() builtinFunc {
newSig := &builtinJSONOverlapsSig{}
newSig.cloneFrom(&b.baseBuiltinFunc)
return newSig
}

func (c *jsonOverlapsFunctionClass) verifyArgs(args []Expression) error {
if err := c.baseFunctionClass.verifyArgs(args); err != nil {
return err
}
if evalType := args[0].GetType().EvalType(); evalType != types.ETJson && evalType != types.ETString {
return types.ErrInvalidJSONData.GenWithStackByArgs(1, "json_overlaps")
}
if evalType := args[1].GetType().EvalType(); evalType != types.ETJson && evalType != types.ETString {
return types.ErrInvalidJSONData.GenWithStackByArgs(2, "json_overlaps")
}
return nil
}

func (c *jsonOverlapsFunctionClass) getFunction(ctx sessionctx.Context, args []Expression) (builtinFunc, error) {
if err := c.verifyArgs(args); err != nil {
return nil, err
}

argTps := []types.EvalType{types.ETJson, types.ETJson}
bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETInt, argTps...)
if err != nil {
return nil, err
}
sig := &builtinJSONOverlapsSig{bf}
return sig, nil
}

func (b *builtinJSONOverlapsSig) evalInt(row chunk.Row) (res int64, isNull bool, err error) {
obj, isNull, err := b.args[0].EvalJSON(b.ctx, row)
if isNull || err != nil {
return res, isNull, err
}
target, isNull, err := b.args[1].EvalJSON(b.ctx, row)
if isNull || err != nil {
return res, isNull, err
}
if types.OverlapsBinaryJSON(obj, target) {
return 1, false, nil
}
return 0, false, nil
}

type jsonValidFunctionClass struct {
baseFunctionClass
}
Expand Down
65 changes: 65 additions & 0 deletions expression/builtin_json_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -466,6 +466,71 @@ func TestJSONContains(t *testing.T) {
}
}

func TestJSONOverlaps(t *testing.T) {
ctx := createContext(t)
fc := funcs[ast.JSONOverlaps]
tbl := []struct {
input []any
expected any
err error
}{
{[]any{`[1,2,[1,3]]`, `a:1`}, 1, types.ErrInvalidJSONText},
{[]any{`a:1`, `1`}, 1, types.ErrInvalidJSONText},
{[]any{nil, `1`}, nil, nil},
{[]any{`1`, nil}, nil, nil},

{[]any{`[1, 2]`, `[2,3]`}, 1, nil},
{[]any{`[1, 2]`, `[2]`}, 1, nil},
{[]any{`[1, 2]`, `2`}, 1, nil},
{[]any{`[{"a":1}]`, `{"a":1}`}, 1, nil},
{[]any{`[{"a":1}]`, `{"a":1,"b":2}`}, 0, nil},
{[]any{`[{"a":1}]`, `{"a":2}`}, 0, nil},
{[]any{`{"a":[1,2]}`, `{"a":[1]}`}, 0, nil},
{[]any{`{"a":[1,2]}`, `{"a":[2,1]}`}, 0, nil},
{[]any{`[1,1,1]`, `1`}, 1, nil},
{[]any{`1`, `1`}, 1, nil},
{[]any{`0`, `1`}, 0, nil},
{[]any{`[[1,2], 3]`, `[1,[2,3]]`}, 0, nil},
{[]any{`[[1,2], 3]`, `[1,3]`}, 1, nil},
{[]any{`{"a":1,"b":10,"d":10}`, `{"a":5,"e":10,"f":1,"d":20}`}, 0, nil},
{[]any{`[4,5,"6",7]`, `6`}, 0, nil},
{[]any{`[4,5,6,7]`, `"6"`}, 0, nil},

{[]any{`[2,3]`, `[1, 2]`}, 1, nil},
{[]any{`[2]`, `[1, 2]`}, 1, nil},
{[]any{`2`, `[1, 2]`}, 1, nil},
{[]any{`{"a":1}`, `[{"a":1}]`}, 1, nil},
{[]any{`{"a":1,"b":2}`, `[{"a":1}]`}, 0, nil},
{[]any{`{"a":2}`, `[{"a":1}]`}, 0, nil},
{[]any{`{"a":[1]}`, `{"a":[1,2]}`}, 0, nil},
{[]any{`{"a":[2,1]}`, `{"a":[1,2]}`}, 0, nil},
{[]any{`1`, `[1,1,1]`}, 1, nil},
{[]any{`1`, `1`}, 1, nil},
{[]any{`1`, `0`}, 0, nil},
{[]any{`[1,[2,3]]`, `[[1,2], 3]`}, 0, nil},
{[]any{`[1,3]`, `[[1,2], 3]`}, 1, nil},
{[]any{`{"a":5,"e":10,"f":1,"d":20}`, `{"a":1,"b":10,"d":10}`}, 0, nil},
{[]any{`6`, `[4,5,"6",7]`}, 0, nil},
{[]any{`"6"`, `[4,5,6,7]`}, 0, nil},
}
for _, tt := range tbl {
args := types.MakeDatums(tt.input...)
f, err := fc.getFunction(ctx, datumsToConstants(args))
require.NoError(t, err, tt.input)
d, err := evalBuiltinFunc(f, chunk.Row{})
if tt.err == nil {
require.NoError(t, err, tt.input)
if tt.expected == nil {
require.True(t, d.IsNull(), tt.input)
} else {
require.Equal(t, int64(tt.expected.(int)), d.GetInt64(), tt.input)
}
} else {
require.True(t, tt.err.(*terror.Error).Equal(err), tt.input)
}
}
}

func TestJSONContainsPath(t *testing.T) {
ctx := createContext(t)
fc := funcs[ast.JSONContainsPath]
Expand Down
45 changes: 45 additions & 0 deletions expression/builtin_json_vec.go
Original file line number Diff line number Diff line change
Expand Up @@ -359,6 +359,51 @@ func (b *builtinJSONContainsSig) vecEvalInt(input *chunk.Chunk, result *chunk.Co
return nil
}

func (b *builtinJSONOverlapsSig) vectorized() bool {
return true
}

func (b *builtinJSONOverlapsSig) vecEvalInt(input *chunk.Chunk, result *chunk.Column) error {
nr := input.NumRows()

objCol, err := b.bufAllocator.get()
if err != nil {
return err
}
defer b.bufAllocator.put(objCol)

if err := b.args[0].VecEvalJSON(b.ctx, input, objCol); err != nil {
return err
}

targetCol, err := b.bufAllocator.get()
if err != nil {
return err
}
defer b.bufAllocator.put(targetCol)

if err := b.args[1].VecEvalJSON(b.ctx, input, targetCol); err != nil {
return err
}

result.ResizeInt64(nr, false)
resI64s := result.Int64s()

result.MergeNulls(objCol, targetCol)
for i := 0; i < nr; i++ {
if result.IsNull(i) {
continue
}
if types.OverlapsBinaryJSON(objCol.GetJSON(i), targetCol.GetJSON(i)) {
resI64s[i] = 1
} else {
resI64s[i] = 0
}
}

return nil
}

func (b *builtinJSONQuoteSig) vectorized() bool {
return true
}
Expand Down
7 changes: 7 additions & 0 deletions expression/integration_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -2721,6 +2721,13 @@ func TestFuncJSON(t *testing.T) {
// #16267
tk.MustQuery(`select json_array(922337203685477580) = json_array(922337203685477581);`).Check(testkit.Rows("0"))

tk.MustQuery("select json_overlaps('[[1,2], 3]', '[1, 3]');").Check(testkit.Rows("1"))
tk.MustQuery("select json_overlaps('[{\"a\":1}]', '{\"a\":1}');").Check(testkit.Rows("1"))
tk.MustQuery("select json_overlaps('{\"a\":1}', '[{\"a\":1}]');").Check(testkit.Rows("1"))
tk.MustQuery("select json_overlaps('[1,[2,3]]', '[[1,2], 3]');").Check(testkit.Rows("0"))
tk.MustQuery("select json_overlaps('{\"a\":[1,2]}', '{\"a\":[2,1]}');").Check(testkit.Rows("0"))
tk.MustQuery("select json_overlaps('{\"a\":[1,2]}', '{\"a\":[2,1]}');").Check(testkit.Rows("0"))

// #10461
tk.MustExec("drop table if exists tx1")
tk.MustExec("create table tx1(id int key, a double, b double, c double, d double)")
Expand Down
4 changes: 2 additions & 2 deletions metrics/grafana/tidb.json
Original file line number Diff line number Diff line change
Expand Up @@ -17776,9 +17776,9 @@
"targets": [
{
"exemplar": true,
"expr": "sum(tidb_server_ttl_job_status{k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}) by (type)",
"expr": "sum(tidb_server_ttl_job_status{k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}) by (type, instance)",
"interval": "",
"legendFormat": "{{ type }}",
"legendFormat": "{{ instance }} {{ type }}",
"queryType": "randomWalk",
"refId": "A"
}
Expand Down
1 change: 1 addition & 0 deletions parser/ast/functions.go
Original file line number Diff line number Diff line change
Expand Up @@ -331,6 +331,7 @@ const (
JSONInsert = "json_insert"
JSONReplace = "json_replace"
JSONRemove = "json_remove"
JSONOverlaps = "json_overlaps"
JSONContains = "json_contains"
JSONMemberOf = "json_memberof"
JSONContainsPath = "json_contains_path"
Expand Down
30 changes: 29 additions & 1 deletion ttl/cache/split_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -233,6 +233,10 @@ func createTTLTable(t *testing.T, tk *testkit.TestKit, name string, option strin
return createTTLTableWithSQL(t, tk, name, fmt.Sprintf("create table test.%s(id %s primary key, t timestamp) TTL = `t` + interval 1 day", name, option))
}

func create2PKTTLTable(t *testing.T, tk *testkit.TestKit, name string, option string) *cache.PhysicalTable {
return createTTLTableWithSQL(t, tk, name, fmt.Sprintf("create table test.%s(id %s, id2 int, t timestamp, primary key(id, id2)) TTL = `t` + interval 1 day", name, option))
}

func createTTLTableWithSQL(t *testing.T, tk *testkit.TestKit, name string, sql string) *cache.PhysicalTable {
tk.MustExec(sql)
is, ok := tk.Session().GetDomainInfoSchema().(infoschema.InfoSchema)
Expand Down Expand Up @@ -273,6 +277,7 @@ func TestSplitTTLScanRangesWithSignedInt(t *testing.T) {
createTTLTable(t, tk, "t4", "int"),
createTTLTable(t, tk, "t5", "bigint"),
createTTLTable(t, tk, "t6", ""), // no clustered
create2PKTTLTable(t, tk, "t7", "tinyint"),
}

tikvStore := newMockTiKVStore(t)
Expand Down Expand Up @@ -334,6 +339,7 @@ func TestSplitTTLScanRangesWithUnsignedInt(t *testing.T) {
createTTLTable(t, tk, "t3", "mediumint unsigned"),
createTTLTable(t, tk, "t4", "int unsigned"),
createTTLTable(t, tk, "t5", "bigint unsigned"),
create2PKTTLTable(t, tk, "t6", "tinyint unsigned"),
}

tikvStore := newMockTiKVStore(t)
Expand Down Expand Up @@ -397,6 +403,7 @@ func TestSplitTTLScanRangesWithBytes(t *testing.T) {
createTTLTable(t, tk, "t2", "char(32) CHARACTER SET BINARY"),
createTTLTable(t, tk, "t3", "varchar(32) CHARACTER SET BINARY"),
createTTLTable(t, tk, "t4", "bit(32)"),
create2PKTTLTable(t, tk, "t5", "binary(32)"),
}

tikvStore := newMockTiKVStore(t)
Expand Down Expand Up @@ -446,6 +453,7 @@ func TestNoTTLSplitSupportTables(t *testing.T) {
createTTLTable(t, tk, "t2", "varchar(32) CHARACTER SET UTF8MB4"),
createTTLTable(t, tk, "t3", "double"),
createTTLTable(t, tk, "t4", "decimal(32, 2)"),
create2PKTTLTable(t, tk, "t5", "char(32) CHARACTER SET UTF8MB4"),
}

tikvStore := newMockTiKVStore(t)
Expand Down Expand Up @@ -526,6 +534,14 @@ func TestGetNextBytesHandleDatum(t *testing.T) {
key: buildBytesRowKey([]byte{1, 2, 3, 4, 5, 6, 7, 8, 0}),
result: []byte{1, 2, 3, 4, 5, 6, 7, 8, 0},
},
{
key: append(buildBytesRowKey([]byte{1, 2, 3, 4, 5, 6, 7, 8, 0}), 0),
result: []byte{1, 2, 3, 4, 5, 6, 7, 8, 0, 0},
},
{
key: append(buildBytesRowKey([]byte{1, 2, 3, 4, 5, 6, 7, 8, 0}), 1),
result: []byte{1, 2, 3, 4, 5, 6, 7, 8, 0, 0},
},
{
key: []byte{},
result: []byte{},
Expand Down Expand Up @@ -613,7 +629,7 @@ func TestGetNextBytesHandleDatum(t *testing.T) {
bs[len(bs)-10] = 254
return bs
},
result: []byte{1, 2, 3, 4, 5, 6, 7},
result: []byte{1, 2, 3, 4, 5, 6, 7, 0},
},
{
// recordPrefix + bytesFlag + [1, 2, 3, 4, 5, 6, 7, 0, 253, 9, 0, 0, 0, 0, 0, 0, 0, 248]
Expand Down Expand Up @@ -718,6 +734,18 @@ func TestGetNextIntHandle(t *testing.T) {
key: tablecodec.EncodeRowKeyWithHandle(tblID, kv.IntHandle(math.MinInt64)),
result: math.MinInt64,
},
{
key: append(tablecodec.EncodeRowKeyWithHandle(tblID, kv.IntHandle(7)), 0),
result: 8,
},
{
key: append(tablecodec.EncodeRowKeyWithHandle(tblID, kv.IntHandle(math.MaxInt64)), 0),
isNull: true,
},
{
key: append(tablecodec.EncodeRowKeyWithHandle(tblID, kv.IntHandle(math.MinInt64)), 0),
result: math.MinInt64 + 1,
},
{
key: []byte{},
result: math.MinInt64,
Expand Down
13 changes: 8 additions & 5 deletions ttl/cache/table.go
Original file line number Diff line number Diff line change
Expand Up @@ -166,9 +166,9 @@ func NewPhysicalTable(schema model.CIStr, tbl *model.TableInfo, partition model.
}, nil
}

// ValidateKey validates a key
func (t *PhysicalTable) ValidateKey(key []types.Datum) error {
if len(t.KeyColumns) != len(key) {
// ValidateKeyPrefix validates a key prefix
func (t *PhysicalTable) ValidateKeyPrefix(key []types.Datum) error {
if len(key) > len(t.KeyColumns) {
return errors.Errorf("invalid key length: %d, expected %d", len(key), len(t.KeyColumns))
}
return nil
Expand Down Expand Up @@ -198,7 +198,7 @@ func (t *PhysicalTable) EvalExpireTime(ctx context.Context, se session.Session,

// SplitScanRanges split ranges for TTL scan
func (t *PhysicalTable) SplitScanRanges(ctx context.Context, store kv.Storage, splitCnt int) ([]ScanRange, error) {
if len(t.KeyColumns) != 1 || splitCnt <= 1 {
if len(t.KeyColumns) < 1 || splitCnt <= 1 {
return []ScanRange{newFullRange()}, nil
}

Expand Down Expand Up @@ -431,7 +431,10 @@ func GetNextBytesHandleDatum(key kv.Key, recordPrefix []byte) (d types.Datum) {
return d
}

if _, v, err := codec.DecodeOne(encodedVal); err == nil {
if remain, v, err := codec.DecodeOne(encodedVal); err == nil {
if len(remain) > 0 {
v.SetBytes(kv.Key(v.GetBytes()).Next())
}
return v
}

Expand Down
Loading

0 comments on commit 41453a2

Please sign in to comment.