From cf69205919f8e6a8b13d22f935bcbea75f4bd145 Mon Sep 17 00:00:00 2001 From: xiongjiwei Date: Tue, 23 Aug 2022 11:14:20 +0800 Subject: [PATCH] expression: refactor json path syntax and make it compatible with mysql (#37074) ref pingcap/tidb#37133 --- errno/errname.go | 2 +- errors.toml | 2 +- types/json/path_expr.go | 283 ++++++++++++++++++++++++----------- types/json/path_expr_test.go | 17 ++- 4 files changed, 209 insertions(+), 95 deletions(-) diff --git a/errno/errname.go b/errno/errname.go index 9b27b44c09da7..4a72160a4681e 100644 --- a/errno/errname.go +++ b/errno/errname.go @@ -845,7 +845,7 @@ var MySQLErrName = map[uint16]*mysql.ErrMessage{ ErrAggregateInOrderNotSelect: mysql.Message("Expression #%d of ORDER BY clause is not in SELECT list, contains aggregate function; this is incompatible with %s", nil), ErrInvalidJSONData: mysql.Message("Invalid JSON data provided to function %s: %s", nil), ErrInvalidJSONText: mysql.Message("Invalid JSON text: %-.192s", []int{0}), - ErrInvalidJSONPath: mysql.Message("Invalid JSON path expression %s.", nil), + ErrInvalidJSONPath: mysql.Message("Invalid JSON path expression. The error is around character position %d.", []int{0}), ErrInvalidJSONCharset: mysql.Message("Cannot create a JSON value from a string with CHARACTER SET '%s'.", nil), ErrInvalidTypeForJSON: mysql.Message("Invalid data type for JSON data in argument %d to function %s; a JSON string or JSON type is required.", nil), ErrInvalidJSONPathWildcard: mysql.Message("In this situation, path expressions may not contain the * and ** tokens.", nil), diff --git a/errors.toml b/errors.toml index c320aff67413d..8dc8f32cb5d50 100644 --- a/errors.toml +++ b/errors.toml @@ -1588,7 +1588,7 @@ Invalid JSON text: %-.192s ["json:3143"] error = ''' -Invalid JSON path expression %s. +Invalid JSON path expression. The error is around character position %d. ''' ["json:3144"] diff --git a/types/json/path_expr.go b/types/json/path_expr.go index cee5f56c33c4b..46773ed2e9b16 100644 --- a/types/json/path_expr.go +++ b/types/json/path_expr.go @@ -15,13 +15,13 @@ package json import ( + "encoding/json" "math" - "regexp" "strconv" "strings" "sync" + "unicode" - "github.com/pingcap/errors" "github.com/pingcap/tidb/util/hack" "github.com/pingcap/tidb/util/kvcache" ) @@ -50,10 +50,6 @@ import ( select json_extract('{"a": "b", "c": [1, "2"]}', '$.*') -> ["b", [1, "2"]] */ -// ([\$]*[a-zA-Z_][a-zA-Z0-9_]*)+ matches any identifier, may start with $ and appear multiple times; -// "[^"\\]*(\\.[^"\\]*)*" 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 ( @@ -128,7 +124,7 @@ func (pe PathExpression) popOneLeg() (pathLeg, PathExpression) { return pe.legs[0], newPe } -// popOneLastLeg returns the a parent PathExpression and the last pathLeg +// popOneLastLeg returns the parent PathExpression and the last pathLeg func (pe PathExpression) popOneLastLeg() (PathExpression, pathLeg) { lastLegIdx := len(pe.legs) - 1 lastLeg := pe.legs[lastLegIdx] @@ -165,111 +161,218 @@ func (pe PathExpression) ContainsAnyAsterisk() bool { return pe.flags.containsAnyAsterisk() } -// 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) { - peCache.mu.Lock() - val, ok := peCache.cache.Get(pathExpressionKey(pathExpr)) - if ok { - peCache.mu.Unlock() - return val.(PathExpression), nil - } - peCache.mu.Unlock() +type stream struct { + pathExpr string + pos int +} - defer func() { - if err == nil { - peCache.mu.Lock() - peCache.cache.Put(pathExpressionKey(pathExpr), kvcache.Value(pe)) - peCache.mu.Unlock() +func (s *stream) skipWhiteSpace() { + for ; s.pos < len(s.pathExpr); s.pos++ { + if !unicode.IsSpace(rune(s.pathExpr[s.pos])) { + break } - }() - - // 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.GenWithStackByArgs(pathExpr) - return } - for i := 0; i < dollarIndex; i++ { - if !isBlank(rune(pathExpr[i])) { - err = ErrInvalidJSONPath.GenWithStackByArgs(pathExpr) - return +} + +func (s *stream) read() byte { + b := s.pathExpr[s.pos] + s.pos++ + return b +} + +func (s *stream) peek() byte { + return s.pathExpr[s.pos] +} + +func (s *stream) skip(i int) { + s.pos += i +} + +func (s *stream) exhausted() bool { + return s.pos >= len(s.pathExpr) +} + +func (s *stream) readWhile(f func(byte) bool) (str string, metEnd bool) { + start := s.pos + for ; !s.exhausted(); s.skip(1) { + if !f(s.peek()) { + return s.pathExpr[start:s.pos], false } } + return s.pathExpr[start:s.pos], true +} - pathExprSuffix := strings.TrimFunc(pathExpr[dollarIndex+1:], isBlank) - indices := jsonPathExprLegRe.FindAllStringIndex(pathExprSuffix, -1) - if len(indices) == 0 && len(pathExprSuffix) != 0 { - err = ErrInvalidJSONPath.GenWithStackByArgs(pathExpr) - return +func parseJSONPathExpr(pathExpr string) (pe PathExpression, err error) { + s := &stream{pathExpr: pathExpr, pos: 0} + s.skipWhiteSpace() + if s.exhausted() || s.read() != '$' { + return PathExpression{}, ErrInvalidJSONPath.GenWithStackByArgs(1) } + s.skipWhiteSpace() - pe.legs = make([]pathLeg, 0, len(indices)) + pe.legs = make([]pathLeg, 0, 16) pe.flags = pathExpressionFlag(0) - lastEnd := 0 - for _, indice := range indices { - start, end := indice[0], indice[1] + var ok bool + + for !s.exhausted() { + switch s.peek() { + case '.': + ok = parseMember(s, &pe) + case '[': + ok = parseArray(s, &pe) + case '*': + ok = parseWildcard(s, &pe) + default: + ok = false + } - // Check all characters between two legs are blank. - for i := lastEnd; i < start; i++ { - if !isBlank(rune(pathExprSuffix[i])) { - err = ErrInvalidJSONPath.GenWithStackByArgs(pathExpr) - return - } + if !ok { + return PathExpression{}, ErrInvalidJSONPath.GenWithStackByArgs(s.pos) } - 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.GenWithStackByArgs(pathExpr) - return + + s.skipWhiteSpace() + } + + if len(pe.legs) > 0 && pe.legs[len(pe.legs)-1].typ == pathLegDoubleAsterisk { + return PathExpression{}, ErrInvalidJSONPath.GenWithStackByArgs(s.pos) + } + + return +} + +func parseWildcard(s *stream, p *PathExpression) bool { + s.skip(1) + if s.exhausted() || s.read() != '*' { + return false + } + if s.exhausted() || s.peek() == '*' { + return false + } + + p.flags |= pathExpressionContainsDoubleAsterisk + p.legs = append(p.legs, pathLeg{typ: pathLegDoubleAsterisk}) + return true +} + +func parseArray(s *stream, p *PathExpression) bool { + s.skip(1) + s.skipWhiteSpace() + if s.exhausted() { + return false + } + + if s.peek() == '*' { + s.skip(1) + p.flags |= pathExpressionContainsAsterisk + p.legs = append(p.legs, pathLeg{typ: pathLegIndex, arrayIndex: arrayIndexAsterisk}) + } else { + // FIXME: only support an integer index for now. Need to support [last], [1 to 2]... in the future. + str, meetEnd := s.readWhile(func(b byte) bool { + return b >= '0' && b <= '9' + }) + if meetEnd { + return false + } + index, err := strconv.Atoi(str) + if err != nil || index > math.MaxUint32 { + return false + } + p.legs = append(p.legs, pathLeg{typ: pathLegIndex, arrayIndex: index}) + } + + s.skipWhiteSpace() + if s.exhausted() || s.read() != ']' { + return false + } + + return true +} + +func parseMember(s *stream, p *PathExpression) bool { + var err error + s.skip(1) + s.skipWhiteSpace() + if s.exhausted() { + return false + } + + if s.peek() == '*' { + s.skip(1) + p.flags |= pathExpressionContainsAsterisk + p.legs = append(p.legs, pathLeg{typ: pathLegKey, dotKey: "*"}) + } else { + var dotKey string + var wasQuoted bool + if s.peek() == '"' { + s.skip(1) + str, meetEnd := s.readWhile(func(b byte) bool { + if b == '\\' { + s.skip(1) + return true } + return b != '"' + }) + if meetEnd { + return false } - pe.legs = append(pe.legs, pathLeg{typ: pathLegKey, dotKey: key}) + s.skip(1) + dotKey = str + wasQuoted = true } else { - // The leg is '**'. - pe.flags |= pathExpressionContainsDoubleAsterisk - pe.legs = append(pe.legs, pathLeg{typ: pathLegDoubleAsterisk}) + dotKey, _ = s.readWhile(func(b byte) bool { + return !(unicode.IsSpace(rune(b)) || b == '.' || b == '[' || b == '*') + }) + } + dotKey = "\"" + dotKey + "\"" + + if !json.Valid(hack.Slice(dotKey)) { + return false } + dotKey, err = unquoteString(dotKey[1 : len(dotKey)-1]) + if err != nil || (!wasQuoted && !isEcmascriptIdentifier(dotKey)) { + return false + } + + p.legs = append(p.legs, pathLeg{typ: pathLegKey, dotKey: dotKey}) + } + return true +} + +func isEcmascriptIdentifier(s string) bool { + if s == "" { + return false } - 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.GenWithStackByArgs(pathExpr) - return + + for i := 0; i < len(s); i++ { + if (i != 0 && s[i] >= '0' && s[i] <= '9') || + (s[i] >= 'a' && s[i] <= 'z') || (s[i] >= 'A' && s[i] <= 'Z') || + s[i] == '_' || s[i] == '$' || s[i] >= 0x80 { + continue } + return false } - return + return true } -func isBlank(c rune) bool { - if c == '\n' || c == '\r' || c == '\t' || c == ' ' { - return true +// 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) (PathExpression, error) { + peCache.mu.Lock() + val, ok := peCache.cache.Get(pathExpressionKey(pathExpr)) + if ok { + peCache.mu.Unlock() + return val.(PathExpression), nil + } + peCache.mu.Unlock() + + pathExpression, err := parseJSONPathExpr(pathExpr) + if err == nil { + peCache.mu.Lock() + peCache.cache.Put(pathExpressionKey(pathExpr), kvcache.Value(pathExpression)) + peCache.mu.Unlock() } - return false + return pathExpression, err } func (pe PathExpression) String() string { diff --git a/types/json/path_expr_test.go b/types/json/path_expr_test.go index 0d229848da01f..c0aa716aa9145 100644 --- a/types/json/path_expr_test.go +++ b/types/json/path_expr_test.go @@ -25,10 +25,10 @@ func TestContainsAnyAsterisk(t *testing.T) { expression string containsAsterisks bool }{ - {"$.a[b]", false}, + {"$.a[1]", false}, {"$.a[*]", true}, - {"$.*[b]", true}, - {"$**.a[b]", true}, + {"$.*[1]", true}, + {"$**.a[1]", true}, } for _, test := range tests { @@ -58,6 +58,17 @@ func TestValidatePathExpr(t *testing.T) { {`$.hello \"escaped quotes\" world[3][*].*.key3`, false, 0}, {`$NoValidLegsHere`, false, 0}, {`$ No Valid Legs Here .a.b.c`, false, 0}, + {`$.a[b]`, false, 0}, + {`$.*[b]`, false, 0}, + {`$**.a[b]`, false, 0}, + {`$.b[ 1 ].`, false, 0}, + {`$.performance.txn-entry-size-limit`, false, 0}, + {`$."performance".txn-entry-size-limit`, false, 0}, + {`$."performance."txn-entry-size-limit`, false, 0}, + {`$."performance."txn-entry-size-limit"`, false, 0}, + {`$[`, false, 0}, + {`$a.***[3]`, false, 0}, + {`$1a`, false, 0}, } for _, test := range tests {