From a594287e9f402037b06930026906547000006bb6 Mon Sep 17 00:00:00 2001 From: Ping Yu Date: Thu, 18 Apr 2019 19:48:01 +0800 Subject: [PATCH] expression/json: add builtin function JSON_SEARCH (#8704) --- expression/builtin_json.go | 117 +++++++++++++++- expression/builtin_json_test.go | 74 +++++++++++ expression/distsql_builtin.go | 2 + expression/distsql_builtin_test.go | 31 ++++- types/json/binary_functions.go | 206 +++++++++++++++++++++++++++++ types/json/binary_test.go | 160 ++++++++++++++++++++++ types/json/path_expr.go | 48 +++++++ types/json/path_expr_test.go | 65 +++++++++ 8 files changed, 699 insertions(+), 4 deletions(-) diff --git a/expression/builtin_json.go b/expression/builtin_json.go index f2ba584a75d78..0693a43fde382 100644 --- a/expression/builtin_json.go +++ b/expression/builtin_json.go @@ -22,6 +22,7 @@ import ( "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/types/json" "github.com/pingcap/tidb/util/chunk" + "github.com/pingcap/tidb/util/stringutil" "github.com/pingcap/tipb/go-tipb" ) @@ -66,6 +67,7 @@ var ( _ builtinFunc = &builtinJSONMergeSig{} _ builtinFunc = &builtinJSONContainsSig{} _ builtinFunc = &builtinJSONDepthSig{} + _ builtinFunc = &builtinJSONSearchSig{} _ builtinFunc = &builtinJSONKeysSig{} _ builtinFunc = &builtinJSONKeys2ArgsSig{} _ builtinFunc = &builtinJSONLengthSig{} @@ -876,8 +878,121 @@ type jsonSearchFunctionClass struct { baseFunctionClass } +type builtinJSONSearchSig struct { + baseBuiltinFunc +} + +func (b *builtinJSONSearchSig) Clone() builtinFunc { + newSig := &builtinJSONSearchSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + func (c *jsonSearchFunctionClass) getFunction(ctx sessionctx.Context, args []Expression) (builtinFunc, error) { - return nil, errFunctionNotExists.GenWithStackByArgs("FUNCTION", "JSON_SEARCH") + if err := c.verifyArgs(args); err != nil { + return nil, err + } + // json_doc, one_or_all, search_str[, escape_char[, path] ...]) + argTps := make([]types.EvalType, 0, len(args)) + argTps = append(argTps, types.ETJson) + for range args[1:] { + argTps = append(argTps, types.ETString) + } + bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETJson, argTps...) + sig := &builtinJSONSearchSig{bf} + sig.setPbCode(tipb.ScalarFuncSig_JsonSearchSig) + return sig, nil +} + +func (b *builtinJSONSearchSig) evalJSON(row chunk.Row) (res json.BinaryJSON, isNull bool, err error) { + // json_doc + var obj json.BinaryJSON + obj, isNull, err = b.args[0].EvalJSON(b.ctx, row) + if isNull || err != nil { + return res, isNull, err + } + + // one_or_all + var containType string + containType, isNull, err = b.args[1].EvalString(b.ctx, row) + if isNull || err != nil { + return res, isNull, err + } + if containType != json.ContainsPathAll && containType != json.ContainsPathOne { + return res, true, errors.AddStack(json.ErrInvalidJSONContainsPathType) + } + + // search_str & escape_char + var searchStr string + searchStr, isNull, err = b.args[2].EvalString(b.ctx, row) + if isNull || err != nil { + return res, isNull, err + } + escape := byte('\\') + if len(b.args) >= 4 { + var escapeStr string + escapeStr, isNull, err = b.args[3].EvalString(b.ctx, row) + if err != nil { + return res, isNull, err + } + if isNull || len(escapeStr) == 0 { + escape = byte('\\') + } else if len(escapeStr) == 1 { + escape = byte(escapeStr[0]) + } else { + return res, true, errIncorrectArgs.GenWithStackByArgs("ESCAPE") + } + } + patChars, patTypes := stringutil.CompilePattern(searchStr, escape) + + // result + result := make([]interface{}, 0) + + // walk json_doc + walkFn := func(fullpath json.PathExpression, bj json.BinaryJSON) (stop bool, err error) { + if bj.TypeCode == json.TypeCodeString && stringutil.DoMatch(string(bj.GetString()), patChars, patTypes) { + result = append(result, fullpath.String()) + if containType == json.ContainsPathOne { + return true, nil + } + } + return false, nil + } + if len(b.args) >= 5 { // path... + pathExprs := make([]json.PathExpression, 0, len(b.args)-4) + for i := 4; i < len(b.args); i++ { + var s string + s, isNull, err = b.args[i].EvalString(b.ctx, row) + if isNull || err != nil { + return res, isNull, err + } + var pathExpr json.PathExpression + pathExpr, err = json.ParseJSONPathExpr(s) + if err != nil { + return res, true, err + } + pathExprs = append(pathExprs, pathExpr) + } + err = obj.Walk(walkFn, pathExprs...) + if err != nil { + return res, true, err + } + } else { + err = obj.Walk(walkFn) + if err != nil { + return res, true, err + } + } + + // return + switch len(result) { + case 0: + return res, true, nil + case 1: + return json.CreateBinary(result[0]), false, nil + default: + return json.CreateBinary(result), false, nil + } } type jsonStorageSizeFunctionClass struct { diff --git a/expression/builtin_json_test.go b/expression/builtin_json_test.go index ebb1a6dcfe9f3..b544e36cc81d8 100644 --- a/expression/builtin_json_test.go +++ b/expression/builtin_json_test.go @@ -773,3 +773,77 @@ func (s *testEvaluatorSuite) TestJSONArrayAppend(c *C) { c.Assert(json.CompareBinary(j1, d.GetMysqlJSON()), Equals, 0, comment) } } + +func (s *testEvaluatorSuite) TestJSONSearch(c *C) { + defer testleak.AfterTest(c)() + fc := funcs[ast.JSONSearch] + jsonString := `["abc", [{"k": "10"}, "def"], {"x":"abc"}, {"y":"bcd"}]` + jsonString2 := `["abc", [{"k": "10"}, "def"], {"x":"ab%d"}, {"y":"abcd"}]` + tbl := []struct { + input []interface{} + expected interface{} + success bool + }{ + // simple case + {[]interface{}{jsonString, `one`, `abc`}, `"$[0]"`, true}, + {[]interface{}{jsonString, `all`, `abc`}, `["$[0]", "$[2].x"]`, true}, + {[]interface{}{jsonString, `all`, `ghi`}, nil, true}, + {[]interface{}{jsonString, `all`, `10`}, `"$[1][0].k"`, true}, + {[]interface{}{jsonString, `all`, `10`, nil, `$`}, `"$[1][0].k"`, true}, + {[]interface{}{jsonString, `all`, `10`, nil, `$[*]`}, `"$[1][0].k"`, true}, + {[]interface{}{jsonString, `all`, `10`, nil, `$**.k`}, `"$[1][0].k"`, true}, + {[]interface{}{jsonString, `all`, `10`, nil, `$[*][0].k`}, `"$[1][0].k"`, true}, + {[]interface{}{jsonString, `all`, `10`, nil, `$[1]`}, `"$[1][0].k"`, true}, + {[]interface{}{jsonString, `all`, `10`, nil, `$[1][0]`}, `"$[1][0].k"`, true}, + {[]interface{}{jsonString, `all`, `abc`, nil, `$[2]`}, `"$[2].x"`, true}, + {[]interface{}{jsonString, `all`, `abc`, nil, `$[2]`, `$[0]`}, `["$[2].x", "$[0]"]`, true}, + {[]interface{}{jsonString, `all`, `abc`, nil, `$[2]`, `$[2]`}, `"$[2].x"`, true}, + + // search pattern + {[]interface{}{jsonString, `all`, `%a%`}, `["$[0]", "$[2].x"]`, true}, + {[]interface{}{jsonString, `all`, `%b%`}, `["$[0]", "$[2].x", "$[3].y"]`, true}, + {[]interface{}{jsonString, `all`, `%b%`, nil, `$[0]`}, `"$[0]"`, true}, + {[]interface{}{jsonString, `all`, `%b%`, nil, `$[2]`}, `"$[2].x"`, true}, + {[]interface{}{jsonString, `all`, `%b%`, nil, `$[1]`}, nil, true}, + {[]interface{}{jsonString, `all`, `%b%`, ``, `$[1]`}, nil, true}, + {[]interface{}{jsonString, `all`, `%b%`, nil, `$[3]`}, `"$[3].y"`, true}, + {[]interface{}{jsonString2, `all`, `ab_d`}, `["$[2].x", "$[3].y"]`, true}, + + // escape char + {[]interface{}{jsonString2, `all`, `ab%d`}, `["$[2].x", "$[3].y"]`, true}, + {[]interface{}{jsonString2, `all`, `ab\%d`}, `"$[2].x"`, true}, + {[]interface{}{jsonString2, `all`, `ab|%d`, `|`}, `"$[2].x"`, true}, + + // error handle + {[]interface{}{nil, `all`, `abc`}, nil, true}, // NULL json + {[]interface{}{`a`, `all`, `abc`}, nil, false}, // non json + {[]interface{}{jsonString, `wrong`, `abc`}, nil, false}, // wrong one_or_all + {[]interface{}{jsonString, `all`, nil}, nil, true}, // NULL search_str + {[]interface{}{jsonString, `all`, `abc`, `??`}, nil, false}, // wrong escape_char + {[]interface{}{jsonString, `all`, `abc`, nil, nil}, nil, true}, // NULL path + {[]interface{}{jsonString, `all`, `abc`, nil, `$xx`}, nil, false}, // wrong path + } + for _, t := range tbl { + args := types.MakeDatums(t.input...) + f, err := fc.getFunction(s.ctx, s.datumsToConstants(args)) + c.Assert(err, IsNil) + d, err := evalBuiltinFunc(f, chunk.Row{}) + if t.success { + c.Assert(err, IsNil) + switch x := t.expected.(type) { + case string: + var j1, j2 json.BinaryJSON + j1, err = json.ParseBinaryFromString(x) + c.Assert(err, IsNil) + j2 = d.GetMysqlJSON() + cmp := json.CompareBinary(j1, j2) + //fmt.Println(j1, j2) + c.Assert(cmp, Equals, 0) + case nil: + c.Assert(d.IsNull(), IsTrue) + } + } else { + c.Assert(err, NotNil) + } + } +} diff --git a/expression/distsql_builtin.go b/expression/distsql_builtin.go index 7135905d80462..96ac0397d0cd5 100644 --- a/expression/distsql_builtin.go +++ b/expression/distsql_builtin.go @@ -444,6 +444,8 @@ func getSignatureByPB(ctx sessionctx.Context, sigCode tipb.ScalarFuncSig, tp *ti f = &builtinJSONLengthSig{base} case tipb.ScalarFuncSig_JsonDepthSig: f = &builtinJSONDepthSig{base} + case tipb.ScalarFuncSig_JsonSearchSig: + f = &builtinJSONSearchSig{base} case tipb.ScalarFuncSig_InInt: f = &builtinInIntSig{base} diff --git a/expression/distsql_builtin_test.go b/expression/distsql_builtin_test.go index 34545f024ca1b..840fd5e89c755 100644 --- a/expression/distsql_builtin_test.go +++ b/expression/distsql_builtin_test.go @@ -17,6 +17,7 @@ import ( "time" . "github.com/pingcap/check" + "github.com/pingcap/parser/charset" "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/types" @@ -108,6 +109,17 @@ func (s *testEvalSuite) TestEval(c *C) { ), types.NewIntDatum(3), }, + { + scalarFunctionExpr(tipb.ScalarFuncSig_JsonSearchSig, + toPBFieldType(newJSONFieldType()), + jsonDatumExpr(c, `["abc", [{"k": "10"}, "def"], {"x":"abc"}, {"y":"bcd"}]`), + datumExpr(c, types.NewBytesDatum([]byte(`all`))), + datumExpr(c, types.NewBytesDatum([]byte(`10`))), + datumExpr(c, types.NewBytesDatum([]byte(`\`))), + datumExpr(c, types.NewBytesDatum([]byte(`$**.k`))), + ), + newJSONDatum(c, `"$[1][0].k"`), + }, } sc := new(stmtctx.StatementContext) for _, tt := range tests { @@ -178,12 +190,15 @@ func datumExpr(c *C, d types.Datum) *tipb.Expr { return expr } -func jsonDatumExpr(c *C, s string) *tipb.Expr { - var d types.Datum +func newJSONDatum(c *C, s string) (d types.Datum) { j, err := json.ParseBinaryFromString(s) c.Assert(err, IsNil) d.SetMysqlJSON(j) - return datumExpr(c, d) + return d +} + +func jsonDatumExpr(c *C, s string) *tipb.Expr { + return datumExpr(c, newJSONDatum(c, s)) } func columnExpr(columnID int64) *tipb.Expr { @@ -214,6 +229,16 @@ func newIntFieldType() *types.FieldType { } } +func newJSONFieldType() *types.FieldType { + return &types.FieldType{ + Tp: mysql.TypeJSON, + Flen: types.UnspecifiedLength, + Decimal: 0, + Charset: charset.CharsetBin, + Collate: charset.CollationBin, + } +} + func scalarFunctionExpr(sigCode tipb.ScalarFuncSig, retType *tipb.FieldType, args ...*tipb.Expr) *tipb.Expr { return &tipb.Expr{ Tp: tipb.ExprType_ScalarFunc, diff --git a/types/json/binary_functions.go b/types/json/binary_functions.go index e20b745658ab8..9cc87569e3542 100644 --- a/types/json/binary_functions.go +++ b/types/json/binary_functions.go @@ -149,6 +149,67 @@ func decodeEscapedUnicode(s []byte) (char [4]byte, size int, err error) { return } +// quoteString escapes interior quote and other characters for JSON_QUOTE +// https://dev.mysql.com/doc/refman/5.7/en/json-creation-functions.html#function_json-quote +// TODO: add JSON_QUOTE builtin +func quoteString(s string) string { + var escapeByteMap = map[byte]string{ + '\\': "\\\\", + '"': "\\\"", + '\b': "\\b", + '\f': "\\f", + '\n': "\\n", + '\r': "\\r", + '\t': "\\t", + } + + ret := new(bytes.Buffer) + ret.WriteByte('"') + + start := 0 + hasEscaped := false + + for i := 0; i < len(s); { + if b := s[i]; b < utf8.RuneSelf { + escaped, ok := escapeByteMap[b] + if ok { + if start < i { + ret.WriteString(s[start:i]) + } + hasEscaped = true + ret.WriteString(escaped) + i++ + start = i + } else { + i++ + } + } else { + c, size := utf8.DecodeRune([]byte(s[i:])) + if c == utf8.RuneError && size == 1 { // refer to codes of `binary.marshalStringTo` + if start < i { + ret.WriteString(s[start:i]) + } + hasEscaped = true + ret.WriteString(`\ufffd`) + i += size + start = i + continue + } + i += size + } + } + + if start < len(s) { + ret.WriteString(s[start:]) + } + + if hasEscaped { + ret.WriteByte('"') + return ret.String() + } + return ret.String()[1:] +} + // Extract receives several path expressions as arguments, matches them in bj, and returns: // ret: target JSON matched any path expressions. maybe autowrapped as an array. // found: true if any path expressions matched. @@ -778,3 +839,148 @@ func (bj BinaryJSON) GetElemDepth() int { return 1 } } + +// extractCallbackFn: the type of CALLBACK function for extractToCallback +type extractCallbackFn func(fullpath PathExpression, bj BinaryJSON) (stop bool, err error) + +// extractToCallback: callback alternative of extractTo +// would be more effective when walk through the whole JSON is unnecessary +// NOTICE: path [0] & [*] for JSON object other than array is INVALID, which is different from extractTo. +func (bj BinaryJSON) extractToCallback(pathExpr PathExpression, callbackFn extractCallbackFn, fullpath PathExpression) (stop bool, err error) { + if len(pathExpr.legs) == 0 { + return callbackFn(fullpath, bj) + } + + currentLeg, subPathExpr := pathExpr.popOneLeg() + if currentLeg.typ == pathLegIndex && bj.TypeCode == TypeCodeArray { + elemCount := bj.GetElemCount() + if currentLeg.arrayIndex == arrayIndexAsterisk { + for i := 0; i < elemCount; i++ { + //buf = bj.arrayGetElem(i).extractTo(buf, subPathExpr) + path := fullpath.pushBackOneIndexLeg(i) + stop, err = bj.arrayGetElem(i).extractToCallback(subPathExpr, callbackFn, path) + if stop || err != nil { + return + } + } + } else if currentLeg.arrayIndex < elemCount { + //buf = bj.arrayGetElem(currentLeg.arrayIndex).extractTo(buf, subPathExpr) + path := fullpath.pushBackOneIndexLeg(currentLeg.arrayIndex) + stop, err = bj.arrayGetElem(currentLeg.arrayIndex).extractToCallback(subPathExpr, callbackFn, path) + if stop || err != nil { + return + } + } + } else if currentLeg.typ == pathLegKey && bj.TypeCode == TypeCodeObject { + elemCount := bj.GetElemCount() + if currentLeg.dotKey == "*" { + for i := 0; i < elemCount; i++ { + //buf = bj.objectGetVal(i).extractTo(buf, subPathExpr) + path := fullpath.pushBackOneKeyLeg(string(bj.objectGetKey(i))) + stop, err = bj.objectGetVal(i).extractToCallback(subPathExpr, callbackFn, path) + if stop || err != nil { + return + } + } + } else { + child, ok := bj.objectSearchKey(hack.Slice(currentLeg.dotKey)) + if ok { + //buf = child.extractTo(buf, subPathExpr) + path := fullpath.pushBackOneKeyLeg(currentLeg.dotKey) + stop, err = child.extractToCallback(subPathExpr, callbackFn, path) + if stop || err != nil { + return + } + } + } + } else if currentLeg.typ == pathLegDoubleAsterisk { + //buf = bj.extractTo(buf, subPathExpr) + stop, err = bj.extractToCallback(subPathExpr, callbackFn, fullpath) + if stop || err != nil { + return + } + + if bj.TypeCode == TypeCodeArray { + elemCount := bj.GetElemCount() + for i := 0; i < elemCount; i++ { + //buf = bj.arrayGetElem(i).extractTo(buf, pathExpr) + path := fullpath.pushBackOneIndexLeg(i) + stop, err = bj.arrayGetElem(i).extractToCallback(pathExpr, callbackFn, path) + if stop || err != nil { + return + } + } + } else if bj.TypeCode == TypeCodeObject { + elemCount := bj.GetElemCount() + for i := 0; i < elemCount; i++ { + //buf = bj.objectGetVal(i).extractTo(buf, pathExpr) + path := fullpath.pushBackOneKeyLeg(string(bj.objectGetKey(i))) + stop, err = bj.objectGetVal(i).extractToCallback(pathExpr, callbackFn, path) + if stop || err != nil { + return + } + } + } + } + return false, nil +} + +// BinaryJSONWalkFunc is used as callback function for BinaryJSON.Walk +type BinaryJSONWalkFunc func(fullpath PathExpression, bj BinaryJSON) (stop bool, err error) + +// Walk traverse BinaryJSON objects +func (bj BinaryJSON) Walk(walkFn BinaryJSONWalkFunc, pathExprList ...PathExpression) (err error) { + pathSet := make(map[string]bool) + + var doWalk extractCallbackFn + doWalk = func(fullpath PathExpression, bj BinaryJSON) (stop bool, err error) { + pathStr := fullpath.String() + if _, ok := pathSet[pathStr]; ok { + return false, nil + } + + stop, err = walkFn(fullpath, bj) + pathSet[pathStr] = true + if stop || err != nil { + return + } + + if bj.TypeCode == TypeCodeArray { + elemCount := bj.GetElemCount() + for i := 0; i < elemCount; i++ { + path := fullpath.pushBackOneIndexLeg(i) + stop, err = doWalk(path, bj.arrayGetElem(i)) + if stop || err != nil { + return + } + } + } else if bj.TypeCode == TypeCodeObject { + elemCount := bj.GetElemCount() + for i := 0; i < elemCount; i++ { + path := fullpath.pushBackOneKeyLeg(string(bj.objectGetKey(i))) + stop, err = doWalk(path, bj.objectGetVal(i)) + if stop || err != nil { + return + } + } + } + return false, nil + } + + fullpath := PathExpression{legs: make([]pathLeg, 0, 32), flags: pathExpressionFlag(0)} + if len(pathExprList) > 0 { + for _, pathExpr := range pathExprList { + var stop bool + stop, err = bj.extractToCallback(pathExpr, doWalk, fullpath) + if stop || err != nil { + return err + } + } + } else { + _, err = doWalk(fullpath, bj) + if err != nil { + return + } + } + return nil +} diff --git a/types/json/binary_test.go b/types/json/binary_test.go index 73d30c8fc6f2c..6ab74d0739602 100644 --- a/types/json/binary_test.go +++ b/types/json/binary_test.go @@ -132,6 +132,27 @@ func (s *testJSONSuite) TestBinaryJSONUnquote(c *C) { } } +func (s *testJSONSuite) TestQuoteString(c *C) { + var tests = []struct { + j string + quoted string + }{ + {j: "3", quoted: `3`}, + {j: "hello, \"escaped quotes\" world", quoted: `"hello, \"escaped quotes\" world"`}, + {j: "你", quoted: `你`}, + {j: "true", quoted: `true`}, + {j: "null", quoted: `null`}, + {j: `"`, quoted: `"\""`}, + {j: `'`, quoted: `'`}, + {j: `''`, quoted: `''`}, + {j: ``, quoted: ``}, + {j: "\\ \" \b \f \n \r \t", quoted: `"\\ \" \b \f \n \r \t"`}, + } + for _, tt := range tests { + c.Assert(quoteString(tt.j), Equals, tt.quoted) + } +} + func (s *testJSONSuite) TestBinaryJSONModify(c *C) { c.Parallel() var tests = []struct { @@ -420,3 +441,142 @@ func (s *testJSONSuite) TestFunctions(c *C) { c.Assert(n, Equals, 0) c.Assert(err, ErrorMatches, "Cant peek from empty bytes") } + +func (s *testJSONSuite) TestBinaryJSONExtractCallback(c *C) { + bj1 := mustParseBinaryFromString(c, `{"\"hello\"": "world", "a": [1, "2", {"aa": "bb"}, 4.0, {"aa": "cc"}], "b": true, "c": ["d"]}`) + bj2 := mustParseBinaryFromString(c, `[{"a": 1, "b": true}, 3, 3.5, "hello, world", null, true]`) + + type ExpectedPair struct { + path string + bj BinaryJSON + } + var tests = []struct { + bj BinaryJSON + pathExpr string + expected []ExpectedPair + }{ + {bj1, "$.a", []ExpectedPair{ + {"$.a", mustParseBinaryFromString(c, `[1, "2", {"aa": "bb"}, 4.0, {"aa": "cc"}]`)}, + }}, + {bj2, "$.a", []ExpectedPair{}}, + {bj1, "$[0]", []ExpectedPair{}}, // in extractToCallback/Walk/Search, DON'T autowraped bj as an array. + {bj2, "$[0]", []ExpectedPair{ + {"$[0]", mustParseBinaryFromString(c, `{"a": 1, "b": true}`)}, + }}, + {bj1, "$.a[2].aa", []ExpectedPair{ + {"$.a[2].aa", mustParseBinaryFromString(c, `"bb"`)}, + }}, + {bj1, "$.a[*].aa", []ExpectedPair{ + {"$.a[2].aa", mustParseBinaryFromString(c, `"bb"`)}, + {"$.a[4].aa", mustParseBinaryFromString(c, `"cc"`)}, + }}, + {bj1, "$.*[0]", []ExpectedPair{ + // {"$.\"hello\"[0]", mustParseBinaryFromString(c, `"world"`)}, // NO autowraped as an array. + {"$.a[0]", mustParseBinaryFromString(c, `1`)}, + // {"$.b[0]", mustParseBinaryFromString(c, `true`)}, // NO autowraped as an array. + {"$.c[0]", mustParseBinaryFromString(c, `"d"`)}, + }}, + {bj1, `$.a[*]."aa"`, []ExpectedPair{ + {"$.a[2].aa", mustParseBinaryFromString(c, `"bb"`)}, + {"$.a[4].aa", mustParseBinaryFromString(c, `"cc"`)}, + }}, + {bj1, `$."\"hello\""`, []ExpectedPair{ + {`$."\"hello\""`, mustParseBinaryFromString(c, `"world"`)}, + }}, + {bj1, `$**[1]`, []ExpectedPair{ + {`$.a[1]`, mustParseBinaryFromString(c, `"2"`)}, + }}, + } + + for _, tt := range tests { + pe, err := ParseJSONPathExpr(tt.pathExpr) + c.Assert(err, IsNil) + + count := 0 + cb := func(fullpath PathExpression, bj BinaryJSON) (stop bool, err error) { + c.Assert(count, Less, len(tt.expected)) + if count < len(tt.expected) { + c.Assert(fullpath.String(), Equals, tt.expected[count].path) + c.Assert(bj.String(), Equals, tt.expected[count].bj.String()) + } + count++ + return false, nil + } + fullpath := PathExpression{legs: make([]pathLeg, 0), flags: pathExpressionFlag(0)} + _, err = tt.bj.extractToCallback(pe, cb, fullpath) + c.Assert(err, IsNil) + c.Assert(count, Equals, len(tt.expected)) + } +} + +func (s *testJSONSuite) TestBinaryJSONWalk(c *C) { + bj1 := mustParseBinaryFromString(c, `["abc", [{"k": "10"}, "def"], {"x":"abc"}, {"y":"bcd"}]`) + bj2 := mustParseBinaryFromString(c, `{}`) + + type ExpectedPair struct { + path string + bj BinaryJSON + } + var tests = []struct { + bj BinaryJSON + paths []string + expected []ExpectedPair + }{ + {bj1, []string{}, []ExpectedPair{ + {`$`, mustParseBinaryFromString(c, `["abc", [{"k": "10"}, "def"], {"x":"abc"}, {"y":"bcd"}]`)}, + {`$[0]`, mustParseBinaryFromString(c, `"abc"`)}, + {`$[1]`, mustParseBinaryFromString(c, `[{"k": "10"}, "def"]`)}, + {`$[1][0]`, mustParseBinaryFromString(c, `{"k": "10"}`)}, + {`$[1][0].k`, mustParseBinaryFromString(c, `"10"`)}, + {`$[1][1]`, mustParseBinaryFromString(c, `"def"`)}, + {`$[2]`, mustParseBinaryFromString(c, `{"x":"abc"}`)}, + {`$[2].x`, mustParseBinaryFromString(c, `"abc"`)}, + {`$[3]`, mustParseBinaryFromString(c, `{"y":"bcd"}`)}, + {`$[3].y`, mustParseBinaryFromString(c, `"bcd"`)}, + }}, + {bj1, []string{`$[1]`}, []ExpectedPair{ + {`$[1]`, mustParseBinaryFromString(c, `[{"k": "10"}, "def"]`)}, + {`$[1][0]`, mustParseBinaryFromString(c, `{"k": "10"}`)}, + {`$[1][0].k`, mustParseBinaryFromString(c, `"10"`)}, + {`$[1][1]`, mustParseBinaryFromString(c, `"def"`)}, + }}, + {bj1, []string{`$[1]`, `$[1]`}, []ExpectedPair{ // test for unique + {`$[1]`, mustParseBinaryFromString(c, `[{"k": "10"}, "def"]`)}, + {`$[1][0]`, mustParseBinaryFromString(c, `{"k": "10"}`)}, + {`$[1][0].k`, mustParseBinaryFromString(c, `"10"`)}, + {`$[1][1]`, mustParseBinaryFromString(c, `"def"`)}, + }}, + {bj1, []string{`$.m`}, []ExpectedPair{}}, + {bj2, []string{}, []ExpectedPair{ + {`$`, mustParseBinaryFromString(c, `{}`)}, + }}, + } + + for _, tt := range tests { + count := 0 + cb := func(fullpath PathExpression, bj BinaryJSON) (stop bool, err error) { + c.Assert(count, Less, len(tt.expected)) + if count < len(tt.expected) { + c.Assert(fullpath.String(), Equals, tt.expected[count].path) + c.Assert(bj.String(), Equals, tt.expected[count].bj.String()) + } + count++ + return false, nil + } + + var err error + if len(tt.paths) > 0 { + peList := make([]PathExpression, 0, len(tt.paths)) + for _, path := range tt.paths { + pe, errPath := ParseJSONPathExpr(path) + c.Assert(errPath, IsNil) + peList = append(peList, pe) + } + err = tt.bj.Walk(cb, peList...) + } else { + err = tt.bj.Walk(cb) + } + c.Assert(err, IsNil) + c.Assert(count, Equals, len(tt.expected)) + } +} diff --git a/types/json/path_expr.go b/types/json/path_expr.go index 7ed5276231ae8..ede4ce11aa89b 100644 --- a/types/json/path_expr.go +++ b/types/json/path_expr.go @@ -117,6 +117,30 @@ func (pe PathExpression) popOneLastLeg() (PathExpression, pathLeg) { return PathExpression{legs: pe.legs[:lastLegIdx]}, lastLeg } +// pushBackOneIndexLeg pushback one leg of INDEX type +func (pe PathExpression) pushBackOneIndexLeg(index int) PathExpression { + newPe := PathExpression{ + legs: append(pe.legs, pathLeg{typ: pathLegIndex, arrayIndex: index}), + flags: pe.flags, + } + if index == -1 { + newPe.flags |= pathExpressionContainsAsterisk + } + return newPe +} + +// pushBackOneKeyLeg pushback one leg of KEY type +func (pe PathExpression) pushBackOneKeyLeg(key string) PathExpression { + newPe := PathExpression{ + legs: append(pe.legs, pathLeg{typ: pathLegKey, dotKey: key}), + flags: pe.flags, + } + if key == "*" { + newPe.flags |= pathExpressionContainsAsterisk + } + return newPe +} + // ContainsAnyAsterisk returns true if pe contains any asterisk. func (pe PathExpression) ContainsAnyAsterisk() bool { return pe.flags.containsAnyAsterisk() @@ -212,3 +236,27 @@ func isBlank(c rune) bool { } return false } + +func (pe PathExpression) String() string { + var s strings.Builder + + s.WriteString("$") + for _, leg := range pe.legs { + switch leg.typ { + case pathLegIndex: + if leg.arrayIndex == -1 { + s.WriteString("[*]") + } else { + s.WriteString("[") + s.WriteString(strconv.Itoa(leg.arrayIndex)) + s.WriteString("]") + } + case pathLegKey: + s.WriteString(".") + s.WriteString(quoteString(leg.dotKey)) + case pathLegDoubleAsterisk: + s.WriteString("**") + } + } + return s.String() +} diff --git a/types/json/path_expr_test.go b/types/json/path_expr_test.go index 4525b7fe064ee..5efc648283d0a 100644 --- a/types/json/path_expr_test.go +++ b/types/json/path_expr_test.go @@ -62,3 +62,68 @@ func (s *testJSONSuite) TestValidatePathExpr(c *C) { } } } + +func (s *testJSONSuite) TestPathExprToString(c *C) { + var tests = []struct { + exprString string + }{ + {"$.a[1]"}, + {"$.a[*]"}, + {"$.*[2]"}, + {"$**.a[3]"}, + {`$."\"hello\""`}, + } + for _, tt := range tests { + pe, err := ParseJSONPathExpr(tt.exprString) + c.Assert(err, IsNil) + c.Assert(pe.String(), Equals, tt.exprString) + } +} + +func (s *testJSONSuite) TestPushBackOneIndexLeg(c *C) { + var tests = []struct { + exprString string + index int + expected string + containsAnyAsterisk bool + }{ + {"$", 1, "$[1]", false}, + {"$.a[1]", 1, "$.a[1][1]", false}, + {"$.a[1]", -1, "$.a[1][*]", true}, + {"$.a[*]", 10, "$.a[*][10]", true}, + {"$.*[2]", 2, "$.*[2][2]", true}, + {"$**.a[3]", 3, "$**.a[3][3]", true}, + } + for _, tt := range tests { + pe, err := ParseJSONPathExpr(tt.exprString) + c.Assert(err, IsNil) + + pe = pe.pushBackOneIndexLeg(tt.index) + c.Assert(pe.String(), Equals, tt.expected) + c.Assert(pe.ContainsAnyAsterisk(), Equals, tt.containsAnyAsterisk) + } +} + +func (s *testJSONSuite) TestPushBackOneKeyLeg(c *C) { + var tests = []struct { + exprString string + key string + expected string + containsAnyAsterisk bool + }{ + {"$", "aa", "$.aa", false}, + {"$.a[1]", "aa", "$.a[1].aa", false}, + {"$.a[1]", "*", "$.a[1].*", true}, + {"$.a[*]", "k", "$.a[*].k", true}, + {"$.*[2]", "bb", "$.*[2].bb", true}, + {"$**.a[3]", "cc", "$**.a[3].cc", true}, + } + for _, tt := range tests { + pe, err := ParseJSONPathExpr(tt.exprString) + c.Assert(err, IsNil) + + pe = pe.pushBackOneKeyLeg(tt.key) + c.Assert(pe.String(), Equals, tt.expected) + c.Assert(pe.ContainsAnyAsterisk(), Equals, tt.containsAnyAsterisk) + } +}