diff --git a/util/types/json/functions.go b/util/types/json/functions.go new file mode 100644 index 0000000000000..0f341fd991b9d --- /dev/null +++ b/util/types/json/functions.go @@ -0,0 +1,167 @@ +// Copyright 2017 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package json + +import ( + "bytes" + "fmt" + "unicode/utf8" + + "github.com/juju/errors" +) + +// Type returns type of JSON as string. +func (j JSON) Type() string { + switch j.typeCode { + case typeCodeObject: + return "OBJECT" + case typeCodeArray: + return "ARRAY" + case typeCodeLiteral: + switch byte(j.i64) { + case jsonLiteralNil: + return "NULL" + default: + return "BOOLEAN" + } + case typeCodeInt64: + return "INTEGER" + case typeCodeFloat64: + return "DOUBLE" + case typeCodeString: + return "STRING" + default: + msg := fmt.Sprintf(unknownTypeCodeErrorMsg, j.typeCode) + panic(msg) + } +} + +// Extract receives several path expressions as arguments, matches them in j, and returns: +// ret: target JSON matched any path expressions. maybe autowrapped as an array. +// found: true if any path expressions matched. +func (j JSON) Extract(pathExprList []PathExpression) (ret JSON, found bool) { + elemList := make([]JSON, 0, len(pathExprList)) + for _, pathExpr := range pathExprList { + elemList = append(elemList, extract(j, pathExpr)...) + } + if len(elemList) == 0 { + found = false + } else if len(pathExprList) == 1 && len(elemList) == 1 { + // If pathExpr contains asterisks, len(elemList) won't be 1 + // even if len(pathExprList) equals to 1. + found = true + ret = elemList[0] + } else { + found = true + ret.typeCode = typeCodeArray + ret.array = append(ret.array, elemList...) + } + return +} + +// Unquote is for JSON_UNQUOTE. +func (j JSON) Unquote() (string, error) { + switch j.typeCode { + case typeCodeString: + return unquoteString(j.str) + default: + return j.String(), nil + } +} + +// unquoteString recognizes the escape sequences shown in: +// https://dev.mysql.com/doc/refman/5.7/en/json-modification-functions.html#json-unquote-character-escape-sequences +func unquoteString(s string) (string, error) { + ret := new(bytes.Buffer) + for i := 0; i < len(s); i++ { + if s[i] == '\\' { + i++ + if i == len(s) { + return "", errors.New("Missing a closing quotation mark in string") + } + switch s[i] { + case '"': + ret.WriteByte('"') + case 'b': + ret.WriteByte('\b') + case 'f': + ret.WriteByte('\f') + case 'n': + ret.WriteByte('\n') + case 'r': + ret.WriteByte('\r') + case 't': + ret.WriteByte('\t') + case '\\': + ret.WriteByte('\\') + case 'u': + if i+4 >= len(s) { + return "", errors.New("Invalid unicode") + } + unicode, size := utf8.DecodeRuneInString(s[i-1 : i+5]) + utf8Buf := make([]byte, size) + utf8.EncodeRune(utf8Buf, unicode) + ret.Write(utf8Buf) + i += 4 + default: + ret.WriteByte(s[i]) + } + } else { + ret.WriteByte(s[i]) + } + } + return ret.String(), nil +} + +// extract is used by Extract. +// NOTE: the return value will share something with j. +func extract(j JSON, pathExpr PathExpression) (ret []JSON) { + if len(pathExpr.legs) == 0 { + return []JSON{j} + } + currentLeg, subPathExpr := pathExpr.popOneLeg() + if currentLeg.typ == pathLegIndex && j.typeCode == typeCodeArray { + if currentLeg.arrayIndex == arrayIndexAsterisk { + for _, child := range j.array { + ret = append(ret, extract(child, subPathExpr)...) + } + } else if currentLeg.arrayIndex < len(j.array) { + childRet := extract(j.array[currentLeg.arrayIndex], subPathExpr) + ret = append(ret, childRet...) + } + } else if currentLeg.typ == pathLegKey && j.typeCode == typeCodeObject { + if len(currentLeg.dotKey) == 1 && currentLeg.dotKey[0] == '*' { + var sortedKeys = getSortedKeys(j.object) // iterate over sorted keys. + for _, child := range sortedKeys { + ret = append(ret, extract(j.object[child], subPathExpr)...) + } + } else if child, ok := j.object[currentLeg.dotKey]; ok { + childRet := extract(child, subPathExpr) + ret = append(ret, childRet...) + } + } else if currentLeg.typ == pathLegDoubleAsterisk { + ret = append(ret, extract(j, subPathExpr)...) + if j.typeCode == typeCodeArray { + for _, child := range j.array { + ret = append(ret, extract(child, pathExpr)...) + } + } else if j.typeCode == typeCodeObject { + var sortedKeys = getSortedKeys(j.object) + for _, child := range sortedKeys { + ret = append(ret, extract(j.object[child], pathExpr)...) + } + } + } + return +} diff --git a/util/types/json/functions_test.go b/util/types/json/functions_test.go new file mode 100644 index 0000000000000..3e79ac43dc205 --- /dev/null +++ b/util/types/json/functions_test.go @@ -0,0 +1,105 @@ +// Copyright 2017 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package json + +import ( + "bytes" + + . "github.com/pingcap/check" +) + +func (s *testJSONSuite) TestJSONType(c *C) { + var tests = []struct { + In string + Out string + }{ + {`{"a": "b"}`, "OBJECT"}, + {`["a", "b"]`, "ARRAY"}, + {`3`, "INTEGER"}, + {`3.0`, "DOUBLE"}, + {`null`, "NULL"}, + {`true`, "BOOLEAN"}, + } + for _, tt := range tests { + j := mustParseFromString(tt.In) + c.Assert(j.Type(), Equals, tt.Out) + } +} + +func (s *testJSONSuite) TestJSONExtract(c *C) { + j1 := mustParseFromString(`{"a": [1, "2", {"aa": "bb"}, 4.0, {"aa": "cc"}], "b": true, "c": ["d"], "\"hello\"": "world"}`) + j2 := mustParseFromString(`[{"a": 1, "b": true}, 3, 3.5, "hello, world", null, true]`) + + var tests = []struct { + j JSON + pathExprStrings []string + expected JSON + found bool + err error + }{ + // test extract with only one path expression. + {j1, []string{"$.a"}, j1.object["a"], true, nil}, + {j2, []string{"$.a"}, CreateJSON(nil), false, nil}, + {j1, []string{"$[0]"}, CreateJSON(nil), false, nil}, + {j2, []string{"$[0]"}, j2.array[0], true, nil}, + {j1, []string{"$.a[2].aa"}, CreateJSON("bb"), true, nil}, + {j1, []string{"$.a[*].aa"}, mustParseFromString(`["bb", "cc"]`), true, nil}, + {j1, []string{"$.*[0]"}, mustParseFromString(`[1, "d"]`), true, nil}, + {j1, []string{`$.a[*]."aa"`}, mustParseFromString(`["bb", "cc"]`), true, nil}, + {j1, []string{`$."\"hello\""`}, mustParseFromString(`"world"`), true, nil}, + {j1, []string{`$**[0]`}, mustParseFromString(`[1, "d"]`), true, nil}, + + // test extract with multi path expressions. + {j1, []string{"$.a", "$[0]"}, mustParseFromString(`[[1, "2", {"aa": "bb"}, 4.0, {"aa": "cc"}]]`), true, nil}, + {j2, []string{"$.a", "$[0]"}, mustParseFromString(`[{"a": 1, "b": true}]`), true, nil}, + } + + for _, tt := range tests { + var pathExprList = make([]PathExpression, 0) + for _, peStr := range tt.pathExprStrings { + pe, err := ParseJSONPathExpr(peStr) + c.Assert(err, IsNil) + pathExprList = append(pathExprList, pe) + } + + expected, found := tt.j.Extract(pathExprList) + c.Assert(found, Equals, tt.found) + if found { + b1 := Serialize(expected) + b2 := Serialize(tt.expected) + c.Assert(bytes.Compare(b1, b2), Equals, 0) + } + } +} + +func (s *testJSONSuite) TestJSONUnquote(c *C) { + var tests = []struct { + j string + unquoted string + }{ + {j: `3`, unquoted: "3"}, + {j: `"3"`, unquoted: "3"}, + {j: `"hello, \"escaped quotes\" world"`, unquoted: "hello, \"escaped quotes\" world"}, + {j: "\"\\u4f60\"", unquoted: "你"}, + {j: `true`, unquoted: "true"}, + {j: `null`, unquoted: "null"}, + {j: `{"a": [1, 2]}`, unquoted: `{"a":[1,2]}`}, + } + for _, tt := range tests { + j := mustParseFromString(tt.j) + unquoted, err := j.Unquote() + c.Assert(err, IsNil) + c.Assert(unquoted, Equals, tt.unquoted) + } +} diff --git a/util/types/json/json.go b/util/types/json/json.go index 2be91731575f6..6dfec1675c975 100644 --- a/util/types/json/json.go +++ b/util/types/json/json.go @@ -123,32 +123,6 @@ func (j JSON) String() string { return strings.TrimSpace(hack.String(bytes)) } -// Type returns type of JSON as string. -func (j JSON) Type() string { - switch j.typeCode { - case typeCodeObject: - return "OBJECT" - case typeCodeArray: - return "ARRAY" - case typeCodeLiteral: - switch byte(j.i64) { - case jsonLiteralNil: - return "NULL" - default: - return "BOOLEAN" - } - case typeCodeInt64: - return "INTEGER" - case typeCodeFloat64: - return "DOUBLE" - case typeCodeString: - return "STRING" - default: - msg := fmt.Sprintf(unknownTypeCodeErrorMsg, j.typeCode) - panic(msg) - } -} - var ( // ErrInvalidJSONText means invalid JSON text. ErrInvalidJSONText = terror.ClassJSON.New(mysql.ErrInvalidJSONText, mysql.MySQLErrName[mysql.ErrInvalidJSONText]) diff --git a/util/types/json/json_test.go b/util/types/json/json_test.go index c80aef7816b62..9c4b972938d08 100644 --- a/util/types/json/json_test.go +++ b/util/types/json/json_test.go @@ -14,6 +14,7 @@ package json import ( + "fmt" "testing" . "github.com/pingcap/check" @@ -27,19 +28,30 @@ func TestT(t *testing.T) { TestingT(t) } -func (s *testJSONSuite) TestJSONSerde(c *C) { +// mustParseFromString parse a JSON from a string. +// Panic if string is not a valid JSON. +func mustParseFromString(s string) JSON { + j, err := ParseFromString(s) + if err != nil { + msg := fmt.Sprintf("ParseFromString(%s) fail", s) + panic(msg) + } + return j +} + +func (s *testJSONSuite) TestParseFromString(c *C) { + jstr1 := `{"a": [1, "2", {"aa": "bb"}, 4, null], "b": true, "c": null}` + jstr2 := mustParseFromString(jstr1).String() + c.Assert(jstr2, Equals, `{"a":[1,"2",{"aa":"bb"},4,null],"b":true,"c":null}`) +} + +func (s *testJSONSuite) TestSerializeAndDeserialize(c *C) { var jsonNilValue = CreateJSON(nil) var jsonBoolValue = CreateJSON(true) var jsonDoubleValue = CreateJSON(3.24) var jsonStringValue = CreateJSON("hello, 世界") - - var jstr1 = `{"aaaaaaaaaaa": [1, "2", {"aa": "bb"}, 4.0], "bbbbbbbbbb": true, "ccccccccc": "d"}` - j1, err := ParseFromString(jstr1) - c.Assert(err, IsNil) - - var jstr2 = `[{"a": 1, "b": true}, 3, 3.5, "hello, world", null, true]` - j2, err := ParseFromString(jstr2) - c.Assert(err, IsNil) + j1 := mustParseFromString(`{"aaaaaaaaaaa": [1, "2", {"aa": "bb"}, 4.0], "bbbbbbbbbb": true, "ccccccccc": "d"}`) + j2 := mustParseFromString(`[{"a": 1, "b": true}, 3, 3.5, "hello, world", null, true]`) var testcses = []struct { In JSON @@ -64,65 +76,19 @@ func (s *testJSONSuite) TestJSONSerde(c *C) { } } -func (s *testJSONSuite) TestParseFromString(c *C) { - var jstr1 = `{"a": [1, "2", {"aa": "bb"}, 4, null], "b": true, "c": null}` - - j1, err := ParseFromString(jstr1) - c.Assert(err, IsNil) - - var jstr2 = j1.String() - c.Assert(jstr2, Equals, `{"a":[1,"2",{"aa":"bb"},4,null],"b":true,"c":null}`) -} - -func (s *testJSONSuite) TestJSONType(c *C) { - j1, err := ParseFromString(`{"a": "b"}`) - c.Assert(err, IsNil) - - j2, err := ParseFromString(`["a", "b"]`) - c.Assert(err, IsNil) - - j3, err := ParseFromString(`3`) - c.Assert(err, IsNil) - - j4, err := ParseFromString(`3.0`) - c.Assert(err, IsNil) - - j5, err := ParseFromString(`null`) - c.Assert(err, IsNil) - - j6, err := ParseFromString(`true`) - c.Assert(err, IsNil) - - var jList = []struct { - In JSON - Out string - }{ - {j1, "OBJECT"}, - {j2, "ARRAY"}, - {j3, "INTEGER"}, - {j4, "DOUBLE"}, - {j5, "NULL"}, - {j6, "BOOLEAN"}, - } - - for _, j := range jList { - c.Assert(j.In.Type(), Equals, j.Out) - } -} - func (s *testJSONSuite) TestCompareJSON(c *C) { - jNull, _ := ParseFromString(`null`) - jBoolTrue, _ := ParseFromString(`true`) - jBoolFalse, _ := ParseFromString(`false`) - jIntegerLarge, _ := ParseFromString(`5`) - jIntegerSmall, _ := ParseFromString(`3`) - jStringLarge, _ := ParseFromString(`"hello, world"`) - jStringSmall, _ := ParseFromString(`"hello"`) - jArrayLarge, _ := ParseFromString(`["a", "c"]`) - jArraySmall, _ := ParseFromString(`["a", "b"]`) - jObject, _ := ParseFromString(`{"a": "b"}`) - - var caseList = []struct { + jNull := mustParseFromString(`null`) + jBoolTrue := mustParseFromString(`true`) + jBoolFalse := mustParseFromString(`false`) + jIntegerLarge := mustParseFromString(`5`) + jIntegerSmall := mustParseFromString(`3`) + jStringLarge := mustParseFromString(`"hello, world"`) + jStringSmall := mustParseFromString(`"hello"`) + jArrayLarge := mustParseFromString(`["a", "c"]`) + jArraySmall := mustParseFromString(`["a", "b"]`) + jObject := mustParseFromString(`{"a": "b"}`) + + var tests = []struct { left JSON right JSON }{ @@ -136,9 +102,8 @@ func (s *testJSONSuite) TestCompareJSON(c *C) { {jArrayLarge, jBoolFalse}, {jBoolFalse, jBoolTrue}, } - - for _, cmpCase := range caseList { - cmp, err := CompareJSON(cmpCase.left, cmpCase.right) + for _, tt := range tests { + cmp, err := CompareJSON(tt.left, tt.right) c.Assert(err, IsNil) c.Assert(cmp < 0, IsTrue) } diff --git a/util/types/json/path_expr.go b/util/types/json/path_expr.go new file mode 100644 index 0000000000000..a087855d73a18 --- /dev/null +++ b/util/types/json/path_expr.go @@ -0,0 +1,198 @@ +// Copyright 2017 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package json + +import ( + "regexp" + "strconv" + "strings" + + "github.com/juju/errors" +) + +/* + From MySQL 5.7, JSON path expression grammar: + pathExpression ::= scope (pathLeg)* + scope ::= [ columnReference ] '$' + columnReference ::= // omit... + pathLeg ::= member | arrayLocation | '**' + member ::= '.' (keyName | '*') + arrayLocation ::= '[' (non-negative-integer | '*') ']' + keyName ::= ECMAScript-identifier | ECMAScript-string-literal + + And some implementation limits in MySQL 5.7: + 1) columnReference in scope must be empty now; + 2) double asterisk(**) could not be last leg; + + Examples: + select json_extract('{"a": "b", "c": [1, "2"]}', '$.a') -> "b" + select json_extract('{"a": "b", "c": [1, "2"]}', '$.c') -> [1, "2"] + select json_extract('{"a": "b", "c": [1, "2"]}', '$.a', '$.c') -> ["b", [1, "2"]] + select json_extract('{"a": "b", "c": [1, "2"]}', '$.c[0]') -> 1 + select json_extract('{"a": "b", "c": [1, "2"]}', '$.c[2]') -> NULL + select json_extract('{"a": "b", "c": [1, "2"]}', '$.c[*]') -> [1, "2"] + select json_extract('{"a": "b", "c": [1, "2"]}', '$.*') -> ["b", [1, "2"]] +*/ + +// [a-zA-Z_][a-zA-Z0-9_]* matches any identifier; +// "[^"\\]*(\\.[^"\\]*)*" matches any string literal which can carry escaped quotes; +var jsonPathExprLegRe = regexp.MustCompile(`(\.\s*([a-zA-Z_][a-zA-Z0-9_]*|\*|"[^"\\]*(\\.[^"\\]*)*")|(\[\s*([0-9]+|\*)\s*\])|\*\*)`) + +type pathLegType byte + +const ( + // pathLegKey indicates the path leg with '.key'. + pathLegKey pathLegType = 0x01 + // pathLegIndex indicates the path leg with form '[number]'. + pathLegIndex pathLegType = 0x02 + // pathLegDoubleAsterisk indicates the path leg with form '**'. + pathLegDoubleAsterisk pathLegType = 0x03 +) + +// pathLeg is only used by PathExpression. +type pathLeg struct { + typ pathLegType + arrayIndex int // if typ is pathLegIndex, the value should be parsed into here. + dotKey string // if typ is pathLegKey, the key should be parsed into here. +} + +// arrayIndexAsterisk is for parsing `*` into a number. +// we need this number represent "all". +const arrayIndexAsterisk int = -1 + +// pathExpressionFlag holds attributes of PathExpression +type pathExpressionFlag byte + +const ( + pathExpressionContainsAsterisk pathExpressionFlag = 0x01 + pathExpressionContainsDoubleAsterisk pathExpressionFlag = 0x02 +) + +// containsAnyAsterisk returns true if pef contains any asterisk. +func (pef pathExpressionFlag) containsAnyAsterisk() bool { + pef &= pathExpressionContainsAsterisk + pef &= pathExpressionContainsDoubleAsterisk + return byte(pef) != 0 +} + +// PathExpression is for JSON path expression. +type PathExpression struct { + legs []pathLeg + flags pathExpressionFlag +} + +// popOneLeg returns a pathLeg, and a child PathExpression without that leg. +func (pe PathExpression) popOneLeg() (pathLeg, PathExpression) { + newPe := PathExpression{ + legs: pe.legs[1:], + flags: 0, + } + for _, leg := range newPe.legs { + if leg.typ == pathLegIndex && leg.arrayIndex == -1 { + newPe.flags |= pathExpressionContainsAsterisk + } else if leg.typ == pathLegKey && leg.dotKey == "*" { + newPe.flags |= pathExpressionContainsAsterisk + } else if leg.typ == pathLegDoubleAsterisk { + newPe.flags |= pathExpressionContainsDoubleAsterisk + } + } + return pe.legs[0], newPe +} + +// ParseJSONPathExpr parses a JSON path expression. Returns a PathExpression +// object which can be used in JSON_EXTRACT, JSON_SET and so on. +func ParseJSONPathExpr(pathExpr string) (pe PathExpression, err error) { + // Find the position of first '$'. If any no-blank characters in + // pathExpr[0: dollarIndex), return an ErrInvalidJSONPath error. + dollarIndex := strings.Index(pathExpr, "$") + if dollarIndex < 0 { + err = ErrInvalidJSONPath.GenByArgs(pathExpr) + return + } + for i := 0; i < dollarIndex; i++ { + if !isBlank(rune(pathExpr[i])) { + err = ErrInvalidJSONPath.GenByArgs(pathExpr) + return + } + } + + pathExprSuffix := pathExpr[dollarIndex+1:] + indices := jsonPathExprLegRe.FindAllStringIndex(pathExprSuffix, -1) + + pe.legs = make([]pathLeg, 0, len(indices)) + pe.flags = pathExpressionFlag(0) + + lastEnd := 0 + for _, indice := range indices { + start, end := indice[0], indice[1] + + // Check all characters between two legs are blank. + for i := lastEnd; i < start; i++ { + if !isBlank(rune(pathExprSuffix[i])) { + err = ErrInvalidJSONPath.GenByArgs(pathExpr) + return + } + } + lastEnd = end + + if pathExprSuffix[start] == '[' { + // The leg is an index of a JSON array. + var leg = strings.TrimFunc(pathExprSuffix[start+1:end], isBlank) + var indexStr = strings.TrimFunc(leg[0:len(leg)-1], isBlank) + var index int + if len(indexStr) == 1 && indexStr[0] == '*' { + pe.flags |= pathExpressionContainsAsterisk + index = arrayIndexAsterisk + } else { + if index, err = strconv.Atoi(indexStr); err != nil { + err = errors.Trace(err) + return + } + } + pe.legs = append(pe.legs, pathLeg{typ: pathLegIndex, arrayIndex: index}) + } else if pathExprSuffix[start] == '.' { + // The leg is a key of a JSON object. + var key = strings.TrimFunc(pathExprSuffix[start+1:end], isBlank) + if len(key) == 1 && key[0] == '*' { + pe.flags |= pathExpressionContainsAsterisk + } else if key[0] == '"' { + // We need unquote the origin string. + if key, err = unquoteString(key[1 : len(key)-1]); err != nil { + err = ErrInvalidJSONPath.GenByArgs(pathExpr) + return + } + } + pe.legs = append(pe.legs, pathLeg{typ: pathLegKey, dotKey: key}) + } else { + // The leg is '**'. + pe.flags |= pathExpressionContainsDoubleAsterisk + pe.legs = append(pe.legs, pathLeg{typ: pathLegDoubleAsterisk}) + } + } + if len(pe.legs) > 0 { + // The last leg of a path expression cannot be '**'. + if pe.legs[len(pe.legs)-1].typ == pathLegDoubleAsterisk { + err = ErrInvalidJSONPath.GenByArgs(pathExpr) + return + } + } + return +} + +func isBlank(c rune) bool { + if c == '\n' || c == '\r' || c == '\t' || c == ' ' { + return true + } + return false +} diff --git a/util/types/json/path_expr_test.go b/util/types/json/path_expr_test.go new file mode 100644 index 0000000000000..1b910e9fa6a81 --- /dev/null +++ b/util/types/json/path_expr_test.go @@ -0,0 +1,45 @@ +// Copyright 2017 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package json + +import ( + . "github.com/pingcap/check" +) + +func (s *testJSONSuite) TestValidatePathExpr(c *C) { + var tests = []struct { + exprString string + success bool + legs int + }{ + {` $ `, true, 0}, + {" $ . key1 [ 3 ]\t[*].*.key3", true, 5}, + {" $ . key1 [ 3 ]**[*].*.key3", true, 6}, + {`$."key1 string"[ 3 ][*].*.key3`, true, 5}, + {`$."hello \"escaped quotes\" world\\n"[3][*].*.key3`, true, 5}, + + {`$.\"escaped quotes\"[3][*].*.key3`, false, 5}, + {`$.hello \"escaped quotes\" world[3][*].*.key3`, false, 5}, + } + + for _, tt := range tests { + pe, err := ParseJSONPathExpr(tt.exprString) + if tt.success { + c.Assert(err, IsNil) + c.Assert(len(pe.legs), Equals, tt.legs) + } else { + c.Assert(err, NotNil) + } + } +}