Skip to content

Commit

Permalink
json: support array range selection and from last selection (pingcap#…
Browse files Browse the repository at this point in the history
  • Loading branch information
YangKeao authored Nov 1, 2022
1 parent a1de01d commit 97add26
Show file tree
Hide file tree
Showing 12 changed files with 386 additions and 99 deletions.
2 changes: 1 addition & 1 deletion errno/errcode.go
Original file line number Diff line number Diff line change
Expand Up @@ -848,7 +848,7 @@ const (
ErrInvalidJSONPath = 3143
ErrInvalidJSONCharset = 3144
ErrInvalidTypeForJSON = 3146
ErrInvalidJSONPathWildcard = 3149
ErrInvalidJSONPathMultipleSelection = 3149
ErrInvalidJSONContainsPathType = 3150
ErrJSONUsedAsKey = 3152
ErrJSONDocumentTooDeep = 3157
Expand Down
2 changes: 1 addition & 1 deletion errno/errname.go
Original file line number Diff line number Diff line change
Expand Up @@ -850,7 +850,7 @@ var MySQLErrName = map[uint16]*mysql.ErrMessage{
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),
ErrInvalidJSONPathMultipleSelection: mysql.Message("In this situation, path expressions may not contain the * and ** tokens or range selection.", nil),
ErrInvalidJSONContainsPathType: mysql.Message("The second argument can only be either 'one' or 'all'.", nil),
ErrJSONUsedAsKey: mysql.Message("JSON column '%-.192s' cannot be used in key specification.", nil),
ErrJSONDocumentTooDeep: mysql.Message("The JSON document exceeds the maximum depth.", nil),
Expand Down
2 changes: 1 addition & 1 deletion errors.toml
Original file line number Diff line number Diff line change
Expand Up @@ -1663,7 +1663,7 @@ Cannot create a JSON value from a string with CHARACTER SET '%s'.

["json:3149"]
error = '''
In this situation, path expressions may not contain the * and ** tokens.
In this situation, path expressions may not contain the * and ** tokens or range selection.
'''

["json:3150"]
Expand Down
20 changes: 10 additions & 10 deletions expression/builtin_json.go
Original file line number Diff line number Diff line change
Expand Up @@ -804,8 +804,8 @@ func (b *builtinJSONContainsSig) evalInt(row chunk.Row) (res int64, isNull bool,
if err != nil {
return res, true, err
}
if pathExpr.ContainsAnyAsterisk() {
return res, true, types.ErrInvalidJSONPathWildcard
if pathExpr.CouldMatchMultipleValues() {
return res, true, types.ErrInvalidJSONPathMultipleSelection
}
var exists bool
obj, exists = obj.Extract([]types.JSONPathExpression{pathExpr})
Expand Down Expand Up @@ -990,8 +990,8 @@ func (b *builtinJSONArrayAppendSig) appendJSONArray(res types.BinaryJSON, p stri
if err != nil {
return res, true, types.ErrInvalidJSONPath.GenWithStackByArgs(p)
}
if pathExpr.ContainsAnyAsterisk() {
return res, true, types.ErrInvalidJSONPathWildcard.GenWithStackByArgs(p)
if pathExpr.CouldMatchMultipleValues() {
return res, true, types.ErrInvalidJSONPathMultipleSelection
}

obj, exists := res.Extract([]types.JSONPathExpression{pathExpr})
Expand Down Expand Up @@ -1071,8 +1071,8 @@ func (b *builtinJSONArrayInsertSig) evalJSON(row chunk.Row) (res types.BinaryJSO
if err != nil {
return res, true, types.ErrInvalidJSONPath.GenWithStackByArgs(s)
}
if pathExpr.ContainsAnyAsterisk() {
return res, true, types.ErrInvalidJSONPathWildcard.GenWithStackByArgs(s)
if pathExpr.CouldMatchMultipleValues() {
return res, true, types.ErrInvalidJSONPathMultipleSelection
}

value, isnull, err := b.args[i+1].EvalJSON(b.ctx, row)
Expand Down Expand Up @@ -1592,8 +1592,8 @@ func (b *builtinJSONKeys2ArgsSig) evalJSON(row chunk.Row) (res types.BinaryJSON,
if err != nil {
return res, true, err
}
if pathExpr.ContainsAnyAsterisk() {
return res, true, types.ErrInvalidJSONPathWildcard
if pathExpr.CouldMatchMultipleValues() {
return res, true, types.ErrInvalidJSONPathMultipleSelection
}

res, exists := res.Extract([]types.JSONPathExpression{pathExpr})
Expand Down Expand Up @@ -1661,8 +1661,8 @@ func (b *builtinJSONLengthSig) evalInt(row chunk.Row) (res int64, isNull bool, e
if err != nil {
return res, true, err
}
if pathExpr.ContainsAnyAsterisk() {
return res, true, types.ErrInvalidJSONPathWildcard
if pathExpr.CouldMatchMultipleValues() {
return res, true, types.ErrInvalidJSONPathMultipleSelection
}

var exists bool
Expand Down
10 changes: 5 additions & 5 deletions expression/builtin_json_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -423,9 +423,9 @@ func TestJSONContains(t *testing.T) {
{[]interface{}{`[{"a":1,"b":2}]`, `{"a":1}`}, 1, nil},
{[]interface{}{`[{"a":{"a":1},"b":2}]`, `{"a":1}`}, 0, nil},
// Tests path expression contains any asterisk
{[]interface{}{`{"a": [1, 2, {"aa": "xx"}]}`, `1`, "$.*"}, nil, types.ErrInvalidJSONPathWildcard},
{[]interface{}{`{"a": [1, 2, {"aa": "xx"}]}`, `1`, "$[*]"}, nil, types.ErrInvalidJSONPathWildcard},
{[]interface{}{`{"a": [1, 2, {"aa": "xx"}]}`, `1`, "$**.a"}, nil, types.ErrInvalidJSONPathWildcard},
{[]interface{}{`{"a": [1, 2, {"aa": "xx"}]}`, `1`, "$.*"}, nil, types.ErrInvalidJSONPathMultipleSelection},
{[]interface{}{`{"a": [1, 2, {"aa": "xx"}]}`, `1`, "$[*]"}, nil, types.ErrInvalidJSONPathMultipleSelection},
{[]interface{}{`{"a": [1, 2, {"aa": "xx"}]}`, `1`, "$**.a"}, nil, types.ErrInvalidJSONPathMultipleSelection},
// Tests path expression does not identify a section of the target document
{[]interface{}{`{"a": [1, 2, {"aa": "xx"}]}`, `1`, "$.c"}, nil, nil},
{[]interface{}{`{"a": [1, 2, {"aa": "xx"}]}`, `1`, "$.a[3]"}, nil, nil},
Expand Down Expand Up @@ -758,7 +758,7 @@ func TestJSONArrayAppend(t *testing.T) {
{[]interface{}{`{"a": 1, "b": [2, 3], "c": 4}`, nil, nil}, nil, nil},
{[]interface{}{`{"a": 1, "b": [2, 3], "c": 4}`, `asdf`, nil}, nil, types.ErrInvalidJSONPath},
{[]interface{}{`{"a": 1, "b": [2, 3], "c": 4}`, 42, nil}, nil, types.ErrInvalidJSONPath},
{[]interface{}{`{"a": 1, "b": [2, 3], "c": 4}`, `$.*`, nil}, nil, types.ErrInvalidJSONPathWildcard},
{[]interface{}{`{"a": 1, "b": [2, 3], "c": 4}`, `$.*`, nil}, nil, types.ErrInvalidJSONPathMultipleSelection},
// Following tests come from MySQL doc.
{[]interface{}{`["a", ["b", "c"], "d"]`, `$[1]`, 1}, `["a", ["b", "c", 1], "d"]`, nil},
{[]interface{}{`["a", ["b", "c"], "d"]`, `$[0]`, 2}, `[["a", 2], ["b", "c"], "d"]`, nil},
Expand Down Expand Up @@ -910,7 +910,7 @@ func TestJSONArrayInsert(t *testing.T) {
{[]interface{}{`{"a": 1, "b": [2, 3], "c": 4}`, nil, nil}, nil, true, nil},
{[]interface{}{`{"a": 1, "b": [2, 3], "c": 4}`, `asdf`, nil}, nil, false, types.ErrInvalidJSONPath},
{[]interface{}{`{"a": 1, "b": [2, 3], "c": 4}`, 42, nil}, nil, false, types.ErrInvalidJSONPath},
{[]interface{}{`{"a": 1, "b": [2, 3], "c": 4}`, `$.*`, nil}, nil, false, types.ErrInvalidJSONPathWildcard},
{[]interface{}{`{"a": 1, "b": [2, 3], "c": 4}`, `$.*`, nil}, nil, false, types.ErrInvalidJSONPathMultipleSelection},
{[]interface{}{`{"a": 1, "b": [2, 3], "c": 4}`, `$.b[0]`, nil, `$.a`, nil}, nil, false, types.ErrInvalidJSONPathArrayCell},
{[]interface{}{`{"a": 1, "b": [2, 3], "c": 4}`, `$.a`, nil}, nil, false, types.ErrInvalidJSONPathArrayCell},
// Following tests come from MySQL doc.
Expand Down
16 changes: 8 additions & 8 deletions expression/builtin_json_vec.go
Original file line number Diff line number Diff line change
Expand Up @@ -326,8 +326,8 @@ func (b *builtinJSONContainsSig) vecEvalInt(input *chunk.Chunk, result *chunk.Co
if err != nil {
return err
}
if pathExpr.ContainsAnyAsterisk() {
return types.ErrInvalidJSONPathWildcard
if pathExpr.CouldMatchMultipleValues() {
return types.ErrInvalidJSONPathMultipleSelection
}

obj, exists := objCol.GetJSON(i).Extract([]types.JSONPathExpression{pathExpr})
Expand Down Expand Up @@ -634,8 +634,8 @@ func (b *builtinJSONArrayInsertSig) vecEvalJSON(input *chunk.Chunk, result *chun
if err != nil {
return types.ErrInvalidJSONPath.GenWithStackByArgs(pathBufs[j].GetString(i))
}
if pathExpr.ContainsAnyAsterisk() {
return types.ErrInvalidJSONPathWildcard.GenWithStackByArgs(pathBufs[j].GetString(i))
if pathExpr.CouldMatchMultipleValues() {
return types.ErrInvalidJSONPathMultipleSelection
}
if valueBufs[j].IsNull(i) {
value = types.CreateBinaryJSON(nil)
Expand Down Expand Up @@ -690,8 +690,8 @@ func (b *builtinJSONKeys2ArgsSig) vecEvalJSON(input *chunk.Chunk, result *chunk.
if err != nil {
return err
}
if pathExpr.ContainsAnyAsterisk() {
return types.ErrInvalidJSONPathWildcard
if pathExpr.CouldMatchMultipleValues() {
return types.ErrInvalidJSONPathMultipleSelection
}

jsonItem := jsonBuf.GetJSON(i)
Expand Down Expand Up @@ -759,8 +759,8 @@ func (b *builtinJSONLengthSig) vecEvalInt(input *chunk.Chunk, result *chunk.Colu
if err != nil {
return err
}
if pathExpr.ContainsAnyAsterisk() {
return types.ErrInvalidJSONPathWildcard
if pathExpr.CouldMatchMultipleValues() {
return types.ErrInvalidJSONPathMultipleSelection
}

obj, exists := jsonItem.Extract([]types.JSONPathExpression{pathExpr})
Expand Down
21 changes: 21 additions & 0 deletions expression/integration_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -7767,3 +7767,24 @@ func TestJSONStorageFree(t *testing.T) {
err := tk.ExecToErr(`select json_storage_free('{"c":["a","b"]`)
require.Error(t, err, "[json:3140]Invalid JSON text: The document root must not be followed by other values.")
}

func TestJSONExtractFromLast(t *testing.T) {
store := testkit.CreateMockStore(t)
tk := testkit.NewTestKit(t, store)
tk.MustExec("use test")
tk.MustQuery(`select json_extract('[{"a": [1,2,3,4]}]', '$[0] . a[last]')`).Check(testkit.Rows("4"))
tk.MustQuery(`select json_extract('[{"a": [1,2,3,4]}]', '$[0] . a [last - 1]')`).Check(testkit.Rows("3"))
tk.MustQuery(`select json_extract('[{"a": [1,2,3,4]}]', '$[0].a [last - 100]')`).Check(testkit.Rows("<nil>"))
}

func TestJSONExtractRange(t *testing.T) {
store := testkit.CreateMockStore(t)
tk := testkit.NewTestKit(t, store)
tk.MustExec("use test")
tk.MustQuery(`select json_extract('[{"a": [1,2,3,4]}]', '$[0].a[1 to last]')`).Check(testkit.Rows("[2, 3, 4]"))
tk.MustQuery(`select json_extract('[{"a": [1,2,3,4]}]', '$[0].a[1 to last - 1]')`).Check(testkit.Rows("[2, 3]"))
tk.MustQuery(`select json_extract('[{"a": [1,2,3,4]}]', '$[0].a[1 to last - 100]')`).Check(testkit.Rows("<nil>"))
tk.MustQuery(`select json_extract('[{"a": [1,2,3,4]}]', '$[0].a[1 to 100]')`).Check(testkit.Rows("[2, 3, 4]"))
tk.MustQuery(`select json_extract('[{"a": [1,2,3,4]}]', '$[0].a[0 to last]')`).Check(testkit.Rows("[1, 2, 3, 4]"))
tk.MustQuery(`select json_extract('[{"a": [1,2,3,4]}]', '$[0].a[0 to 2]')`).Check(testkit.Rows("[1, 2, 3]"))
}
97 changes: 70 additions & 27 deletions types/json_binary_functions.go
Original file line number Diff line number Diff line change
Expand Up @@ -244,7 +244,7 @@ func (bj BinaryJSON) Extract(pathExprList []JSONPathExpression) (ret BinaryJSON,
found = true
ret = buf[0]
// Fix https://github.com/pingcap/tidb/issues/30352
if pathExprList[0].ContainsAnyAsterisk() {
if pathExprList[0].CouldMatchMultipleValues() {
ret = buildBinaryJSONArray(buf)
}
} else {
Expand All @@ -270,20 +270,32 @@ func (bj BinaryJSON) extractTo(buf []BinaryJSON, pathExpr JSONPathExpression, du
return append(buf, bj)
}
currentLeg, subPathExpr := pathExpr.popOneLeg()
if currentLeg.typ == jsonPathLegIndex {
if currentLeg.typ == jsonPathLegArraySelection {
if bj.TypeCode != JSONTypeCodeArray {
if currentLeg.arrayIndex <= 0 && currentLeg.arrayIndex != arrayIndexAsterisk {
buf = bj.extractTo(buf, subPathExpr, dup, one)
// If the current object is not an array, still append them if the selection includes
// 0. But for asterisk, it still returns NULL.
//
// don't call `getIndexRange` or `getIndexFromStart`, they will panic if the argument
// is not array.
switch selection := currentLeg.arraySelection.(type) {
case jsonPathArraySelectionIndex:
if selection.index == 0 {
buf = bj.extractTo(buf, subPathExpr, dup, one)
}
case jsonPathArraySelectionRange:
// for [0 to Non-negative Number] and [0 to last], it extracts itself
if selection.start == 0 && selection.end >= -1 {
buf = bj.extractTo(buf, subPathExpr, dup, one)
}
}
return buf
}
elemCount := bj.GetElemCount()
if currentLeg.arrayIndex == arrayIndexAsterisk {
for i := 0; i < elemCount && !jsonFinished(buf, one); i++ {

start, end := currentLeg.arraySelection.getIndexRange(bj)
if start >= 0 && start <= end {
for i := start; i <= end; i++ {
buf = bj.arrayGetElem(i).extractTo(buf, subPathExpr, dup, one)
}
} else if currentLeg.arrayIndex < elemCount {
buf = bj.arrayGetElem(currentLeg.arrayIndex).extractTo(buf, subPathExpr, dup, one)
}
} else if currentLeg.typ == jsonPathLegKey && bj.TypeCode == JSONTypeCodeObject {
elemCount := bj.GetElemCount()
Expand Down Expand Up @@ -389,7 +401,7 @@ func (bj BinaryJSON) Modify(pathExprList []JSONPathExpression, values []BinaryJS
return retj, errors.New("Incorrect parameter count")
}
for _, pathExpr := range pathExprList {
if pathExpr.flags.containsAnyAsterisk() {
if pathExpr.flags.containsAnyAsterisk() || pathExpr.flags.containsAnyRange() {
// TODO: should return 3149(42000)
return retj, errors.New("Invalid path expression")
}
Expand Down Expand Up @@ -424,7 +436,7 @@ func (bj BinaryJSON) ArrayInsert(pathExpr JSONPathExpression, value BinaryJSON)
return bj, ErrInvalidJSONPathArrayCell
}
parentPath, lastLeg := pathExpr.popOneLastLeg()
if lastLeg.typ != jsonPathLegIndex {
if lastLeg.typ != jsonPathLegArraySelection {
return bj, ErrInvalidJSONPathArrayCell
}
// Find the target array
Expand All @@ -433,7 +445,13 @@ func (bj BinaryJSON) ArrayInsert(pathExpr JSONPathExpression, value BinaryJSON)
return bj, nil
}

idx := lastLeg.arrayIndex
idx := 0
switch selection := lastLeg.arraySelection.(type) {
case jsonPathArraySelectionIndex:
idx = selection.index.getIndexFromStart(obj)
default:
return bj, ErrInvalidJSONPathArrayCell
}
count := obj.GetElemCount()
if idx >= count {
idx = count
Expand Down Expand Up @@ -465,7 +483,7 @@ func (bj BinaryJSON) Remove(pathExprList []JSONPathExpression) (BinaryJSON, erro
// TODO: should return 3153(42000)
return bj, errors.New("Invalid path expression")
}
if pathExpr.flags.containsAnyAsterisk() {
if pathExpr.flags.containsAnyAsterisk() || pathExpr.flags.containsAnyRange() {
// TODO: should return 3149(42000)
return bj, errors.New("Invalid path expression")
}
Expand Down Expand Up @@ -529,7 +547,7 @@ func (bm *binaryModifier) doInsert(path JSONPathExpression, newBj BinaryJSON) {
return
}
parentBj := result[0]
if lastLeg.typ == jsonPathLegIndex {
if lastLeg.typ == jsonPathLegArraySelection {
bm.modifyPtr = &parentBj.Value[0]
if parentBj.TypeCode != JSONTypeCodeArray {
bm.modifyValue = buildBinaryJSONArray([]BinaryJSON{parentBj, newBj})
Expand Down Expand Up @@ -589,15 +607,21 @@ func (bm *binaryModifier) doRemove(path JSONPathExpression) {
return
}
parentBj := result[0]
if lastLeg.typ == jsonPathLegIndex {
if lastLeg.typ == jsonPathLegArraySelection {
if parentBj.TypeCode != JSONTypeCodeArray {
return
}
selectionIndex, ok := lastLeg.arraySelection.(jsonPathArraySelectionIndex)
if !ok {
return
}
idx := selectionIndex.index.getIndexFromStart(parentBj)

bm.modifyPtr = &parentBj.Value[0]
elemCount := parentBj.GetElemCount()
elems := make([]BinaryJSON, 0, elemCount-1)
for i := 0; i < elemCount; i++ {
if i != lastLeg.arrayIndex {
if i != idx {
elems = append(elems, parentBj.arrayGetElem(i))
}
}
Expand Down Expand Up @@ -1175,23 +1199,42 @@ func (bj BinaryJSON) extractToCallback(pathExpr JSONPathExpression, callbackFn e
}

currentLeg, subPathExpr := pathExpr.popOneLeg()
if currentLeg.typ == jsonPathLegIndex && bj.TypeCode == JSONTypeCodeArray {
if currentLeg.typ == jsonPathLegArraySelection && bj.TypeCode == JSONTypeCodeArray {
elemCount := bj.GetElemCount()
if currentLeg.arrayIndex == arrayIndexAsterisk {
switch selection := currentLeg.arraySelection.(type) {
case jsonPathArraySelectionAsterisk:
for i := 0; i < elemCount; i++ {
// buf = bj.arrayGetElem(i).extractTo(buf, subPathExpr)
path := fullpath.pushBackOneIndexLeg(i)
path := fullpath.pushBackOneArraySelectionLeg(jsonPathArraySelectionIndex{jsonPathArrayIndexFromStart(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
case jsonPathArraySelectionIndex:
idx := selection.index.getIndexFromStart(bj)
if idx < elemCount && idx >= 0 {
// buf = bj.arrayGetElem(currentLeg.arraySelection).extractTo(buf, subPathExpr)
path := fullpath.pushBackOneArraySelectionLeg(currentLeg.arraySelection)
stop, err = bj.arrayGetElem(idx).extractToCallback(subPathExpr, callbackFn, path)
if stop || err != nil {
return
}
}
case jsonPathArraySelectionRange:
start := selection.start.getIndexFromStart(bj)
end := selection.end.getIndexFromStart(bj)
if end >= elemCount {
end = elemCount - 1
}
if start <= end && start >= 0 {
for i := start; i <= end; i++ {
path := fullpath.pushBackOneArraySelectionLeg(jsonPathArraySelectionIndex{jsonPathArrayIndexFromStart(i)})
stop, err = bj.arrayGetElem(i).extractToCallback(subPathExpr, callbackFn, path)
if stop || err != nil {
return
}
}
}
}
} else if currentLeg.typ == jsonPathLegKey && bj.TypeCode == JSONTypeCodeObject {
Expand Down Expand Up @@ -1227,7 +1270,7 @@ func (bj BinaryJSON) extractToCallback(pathExpr JSONPathExpression, callbackFn e
elemCount := bj.GetElemCount()
for i := 0; i < elemCount; i++ {
// buf = bj.arrayGetElem(i).extractTo(buf, pathExpr)
path := fullpath.pushBackOneIndexLeg(i)
path := fullpath.pushBackOneArraySelectionLeg(jsonPathArraySelectionIndex{jsonPathArrayIndexFromStart(i)})
stop, err = bj.arrayGetElem(i).extractToCallback(pathExpr, callbackFn, path)
if stop || err != nil {
return
Expand Down Expand Up @@ -1271,7 +1314,7 @@ func (bj BinaryJSON) Walk(walkFn BinaryJSONWalkFunc, pathExprList ...JSONPathExp
if bj.TypeCode == JSONTypeCodeArray {
elemCount := bj.GetElemCount()
for i := 0; i < elemCount; i++ {
path := fullpath.pushBackOneIndexLeg(i)
path := fullpath.pushBackOneArraySelectionLeg(jsonPathArraySelectionIndex{jsonPathArrayIndexFromStart(i)})
stop, err = doWalk(path, bj.arrayGetElem(i))
if stop || err != nil {
return
Expand Down
9 changes: 9 additions & 0 deletions types/json_binary_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -77,6 +77,15 @@ func TestBinaryJSONExtract(t *testing.T) {
{bj5, []string{`$.properties.$type.$a.$b`}, mustParseBinaryFromString(t, `"TiDB"`), true, nil},
{bj5, []string{`$.properties.$type.$a.*[0]`}, mustParseBinaryFromString(t, `["TiDB"]`), true, nil},
{bj11, []string{"$.metadata.comment"}, mustParseBinaryFromString(t, `"1234"`), true, nil},
{bj9, []string{"$[0]"}, mustParseBinaryFromString(t, `[0, 1] `), true, nil},
{bj9, []string{"$[last][last]"}, mustParseBinaryFromString(t, `[5,6]`), true, nil},
{bj9, []string{"$[last-1][last]"}, mustParseBinaryFromString(t, `3`), true, nil},
{bj9, []string{"$[last-1][last-1]"}, mustParseBinaryFromString(t, `2`), true, nil},
{bj9, []string{"$[1 to 2]"}, mustParseBinaryFromString(t, `[[2,3],[4,[5,6]]]`), true, nil},
{bj9, []string{"$[1 to 2][1 to 2]"}, mustParseBinaryFromString(t, `[3,[5,6]]`), true, nil},
{bj9, []string{"$[1 to last][1 to last]"}, mustParseBinaryFromString(t, `[3,[5,6]]`), true, nil},
{bj9, []string{"$[1 to last][1 to last - 1]"}, bj9, false, nil},
{bj9, []string{"$[1 to last][0 to last - 1]"}, mustParseBinaryFromString(t, `[2,4]`), true, nil},

// test extract with multi path expressions.
{bj1, []string{"$.a", "$[5]"}, mustParseBinaryFromString(t, `[[1, "2", {"aa": "bb"}, 4.0, {"aa": "cc"}]]`), true, nil},
Expand Down
Loading

0 comments on commit 97add26

Please sign in to comment.