Skip to content

Commit

Permalink
expression: Fix errors for JSON functions (#53800) (#54561)
Browse files Browse the repository at this point in the history
close #53799
  • Loading branch information
ti-chi-bot committed Jul 12, 2024
1 parent 20d69a7 commit a6096e5
Show file tree
Hide file tree
Showing 14 changed files with 169 additions and 42 deletions.
22 changes: 21 additions & 1 deletion errors.toml
Original file line number Diff line number Diff line change
Expand Up @@ -1996,6 +1996,11 @@ error = '''
Invalid JSON text: %-.192s
'''

["json:3141"]
error = '''
Invalid JSON text in argument %d to function %s: "%s" at position %d.
'''

["json:3143"]
error = '''
Invalid JSON path expression. The error is around character position %d.
Expand All @@ -2008,7 +2013,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 or range selection.
In this situation, path expressions may not contain the * and ** tokens or an array range.
'''

["json:3150"]
Expand All @@ -2031,6 +2036,11 @@ error = '''
A path expression is not a path to a cell in an array.
'''

["json:3853"]
error = '''
Invalid JSON type in argument %d to function %s; an %s is required.
'''

["json:8067"]
error = '''
JSON_OBJECTAGG: unsupported second argument type %v
Expand Down Expand Up @@ -3221,6 +3231,16 @@ error = '''
Invalid size for column '%s'.
'''

["types:3153"]
error = '''
The path expression '$' is not allowed in this context.
'''

["types:3154"]
error = '''
The oneOrAll argument to %s may take these values: 'one' or 'all'.
'''

["types:8029"]
error = '''
Bad Number
Expand Down
4 changes: 4 additions & 0 deletions pkg/errno/errcode.go
Original file line number Diff line number Diff line change
Expand Up @@ -846,12 +846,15 @@ const (
ErrWarnConflictingHint = 3126
ErrUnresolvedHintName = 3128
ErrInvalidJSONText = 3140
ErrInvalidJSONTextInParam = 3141
ErrInvalidJSONPath = 3143
ErrInvalidJSONCharset = 3144
ErrInvalidTypeForJSON = 3146
ErrInvalidJSONPathMultipleSelection = 3149
ErrInvalidJSONContainsPathType = 3150
ErrJSONUsedAsKey = 3152
ErrJSONVacuousPath = 3153
ErrJSONBadOneOrAllArg = 3154
ErrJSONDocumentTooDeep = 3157
ErrJSONDocumentNULLKey = 3158
ErrSecureTransportRequired = 3159
Expand Down Expand Up @@ -927,6 +930,7 @@ const (
ErrCheckConstraintDupName = 3822
ErrCheckConstraintClauseUsingFKReferActionColumn = 3823
ErrDependentByFunctionalIndex = 3837
ErrInvalidJSONType = 3853
ErrCannotConvertString = 3854
ErrDependentByPartitionFunctional = 3855
ErrInvalidJSONValueForFuncIndex = 3903
Expand Down
6 changes: 5 additions & 1 deletion pkg/errno/errname.go
Original file line number Diff line number Diff line change
Expand Up @@ -95,6 +95,8 @@ var MySQLErrName = map[uint16]*mysql.ErrMessage{
ErrTooLongKey: mysql.Message("Specified key was too long (%d bytes); max key length is %d bytes", nil),
ErrKeyColumnDoesNotExits: mysql.Message("Key column '%-.192s' doesn't exist in table", nil),
ErrBlobUsedAsKey: mysql.Message("BLOB column '%-.192s' can't be used in key specification with the used table type", nil),
ErrJSONVacuousPath: mysql.Message("The path expression '$' is not allowed in this context.", nil),
ErrJSONBadOneOrAllArg: mysql.Message("The oneOrAll argument to %s may take these values: 'one' or 'all'.", nil),
ErrTooBigFieldlength: mysql.Message("Column length too big for column '%-.192s' (max = %d); use BLOB or TEXT instead", nil),
ErrWrongAutoKey: mysql.Message("Incorrect table definition; there can be only one auto column and it must be defined as a key", nil),
ErrReady: mysql.Message("%s: ready for connections.\nVersion: '%s' socket: '%s' port: %d", nil),
Expand Down Expand Up @@ -849,10 +851,11 @@ 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}),
ErrInvalidJSONTextInParam: mysql.Message("Invalid JSON text in argument %d to function %s: \"%s\" at position %d.", 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),
ErrInvalidJSONPathMultipleSelection: mysql.Message("In this situation, path expressions may not contain the * and ** tokens or range selection.", nil),
ErrInvalidJSONPathMultipleSelection: mysql.Message("In this situation, path expressions may not contain the * and ** tokens or an array range.", 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 Expand Up @@ -923,6 +926,7 @@ var MySQLErrName = map[uint16]*mysql.ErrMessage{
ErrDependentByFunctionalIndex: mysql.Message("Column '%s' has an expression index dependency and cannot be dropped or renamed", nil),
ErrDependentByPartitionFunctional: mysql.Message("Column '%s' has a partitioning function dependency and cannot be dropped or renamed", nil),
ErrCannotConvertString: mysql.Message("Cannot convert string '%.64s' from %s to %s", nil),
ErrInvalidJSONType: mysql.Message("Invalid JSON type in argument %d to function %s; an %s is required.", nil),
ErrInvalidJSONValueForFuncIndex: mysql.Message("Invalid JSON value for CAST for expression index '%s'", nil),
ErrJSONValueOutOfRangeForFuncIndex: mysql.Message("Out of range JSON value for CAST for expression index '%s'", nil),
ErrFunctionalIndexDataIsTooLong: mysql.Message("Data too long for expression index '%s'", nil),
Expand Down
33 changes: 16 additions & 17 deletions pkg/expression/builtin_json.go
Original file line number Diff line number Diff line change
Expand Up @@ -147,7 +147,7 @@ func (c *jsonExtractFunctionClass) verifyArgs(args []Expression) error {
return err
}
if evalType := args[0].GetType().EvalType(); evalType != types.ETString && evalType != types.ETJson {
return ErrInvalidTypeForJSON.GenWithStackByArgs(0, "json_extract")
return ErrInvalidTypeForJSON.GenWithStackByArgs(1, "json_extract")
}
return nil
}
Expand Down Expand Up @@ -447,7 +447,7 @@ func (c *jsonMergeFunctionClass) verifyArgs(args []Expression) error {
}
for i, arg := range args {
if evalType := arg.GetType().EvalType(); evalType != types.ETString && evalType != types.ETJson {
return ErrInvalidTypeForJSON.GenWithStackByArgs(i, "json_merge")
return ErrInvalidTypeForJSON.GenWithStackByArgs(i+1, "json_merge")
}
}
return nil
Expand Down Expand Up @@ -554,8 +554,7 @@ func (b *builtinJSONObjectSig) evalJSON(row chunk.Row) (res types.BinaryJSON, is
return res, true, err
}
if isNull {
err = errors.New("JSON documents may not contain NULL member names")
return res, true, err
return res, true, types.ErrJSONDocumentNULLKey
}
} else {
value, isNull, err = arg.EvalJSON(b.ctx, row)
Expand Down Expand Up @@ -647,7 +646,7 @@ func (c *jsonContainsPathFunctionClass) verifyArgs(args []Expression) error {
return err
}
if evalType := args[0].GetType().EvalType(); evalType != types.ETString && evalType != types.ETJson {
return ErrInvalidTypeForJSON.GenWithStackByArgs(0, "json_contains_path")
return ErrInvalidTypeForJSON.GenWithStackByArgs(1, "json_contains_path")
}
return nil
}
Expand Down Expand Up @@ -680,7 +679,7 @@ func (b *builtinJSONContainsPathSig) evalInt(row chunk.Row) (res int64, isNull b
}
containType = strings.ToLower(containType)
if containType != types.JSONContainsPathAll && containType != types.JSONContainsPathOne {
return res, true, types.ErrInvalidJSONContainsPathType
return res, true, types.ErrJSONBadOneOrAllArg.GenWithStackByArgs("json_contains_path")
}
var pathExpr types.JSONPathExpression
contains := int64(1)
Expand Down Expand Up @@ -763,7 +762,7 @@ func (c *jsonMemberOfFunctionClass) verifyArgs(args []Expression) error {
return err
}
if evalType := args[1].GetType().EvalType(); evalType != types.ETJson && evalType != types.ETString {
return types.ErrInvalidJSONData.GenWithStackByArgs(2, "member of")
return ErrInvalidTypeForJSON.GenWithStackByArgs(2, "member of")
}
return nil
}
Expand Down Expand Up @@ -826,10 +825,10 @@ func (c *jsonContainsFunctionClass) verifyArgs(args []Expression) error {
return err
}
if evalType := args[0].GetType().EvalType(); evalType != types.ETJson && evalType != types.ETString {
return types.ErrInvalidJSONData.GenWithStackByArgs(1, "json_contains")
return ErrInvalidTypeForJSON.GenWithStackByArgs(1, "json_contains")
}
if evalType := args[1].GetType().EvalType(); evalType != types.ETJson && evalType != types.ETString {
return types.ErrInvalidJSONData.GenWithStackByArgs(2, "json_contains")
return ErrInvalidTypeForJSON.GenWithStackByArgs(2, "json_contains")
}
return nil
}
Expand Down Expand Up @@ -906,10 +905,10 @@ func (c *jsonOverlapsFunctionClass) verifyArgs(args []Expression) error {
return err
}
if evalType := args[0].GetType().EvalType(); evalType != types.ETJson && evalType != types.ETString {
return types.ErrInvalidJSONData.GenWithStackByArgs(1, "json_overlaps")
return ErrInvalidTypeForJSON.GenWithStackByArgs(1, "json_overlaps")
}
if evalType := args[1].GetType().EvalType(); evalType != types.ETJson && evalType != types.ETString {
return types.ErrInvalidJSONData.GenWithStackByArgs(2, "json_overlaps")
return ErrInvalidTypeForJSON.GenWithStackByArgs(2, "json_overlaps")
}
return nil
}
Expand Down Expand Up @@ -1111,7 +1110,7 @@ func (b *builtinJSONArrayAppendSig) appendJSONArray(res types.BinaryJSON, p stri
// We should do the following checks to get correct values in res.Extract
pathExpr, err := types.ParseJSONPathExpr(p)
if err != nil {
return res, true, types.ErrInvalidJSONPath.GenWithStackByArgs(p)
return res, true, err
}
if pathExpr.CouldMatchMultipleValues() {
return res, true, types.ErrInvalidJSONPathMultipleSelection
Expand Down Expand Up @@ -1192,7 +1191,7 @@ func (b *builtinJSONArrayInsertSig) evalJSON(row chunk.Row) (res types.BinaryJSO

pathExpr, err := types.ParseJSONPathExpr(s)
if err != nil {
return res, true, types.ErrInvalidJSONPath.GenWithStackByArgs(s)
return res, true, err
}
if pathExpr.CouldMatchMultipleValues() {
return res, true, types.ErrInvalidJSONPathMultipleSelection
Expand Down Expand Up @@ -1225,7 +1224,7 @@ func (c *jsonMergePatchFunctionClass) verifyArgs(args []Expression) error {
}
for i, arg := range args {
if evalType := arg.GetType().EvalType(); evalType != types.ETString && evalType != types.ETJson {
return ErrInvalidTypeForJSON.GenWithStackByArgs(i, "json_merge_patch")
return ErrInvalidTypeForJSON.GenWithStackByArgs(i+1, "json_merge_patch")
}
}
return nil
Expand Down Expand Up @@ -1294,7 +1293,7 @@ func (c *jsonMergePreserveFunctionClass) verifyArgs(args []Expression) error {
}
for i, arg := range args {
if evalType := arg.GetType().EvalType(); evalType != types.ETString && evalType != types.ETJson {
return ErrInvalidTypeForJSON.GenWithStackByArgs(i, "json_merge_preserve")
return ErrInvalidTypeForJSON.GenWithStackByArgs(i+1, "json_merge_preserve")
}
}
return nil
Expand Down Expand Up @@ -1431,7 +1430,7 @@ func (c *jsonSearchFunctionClass) verifyArgs(args []Expression) error {
return err
}
if evalType := args[0].GetType().EvalType(); evalType != types.ETString && evalType != types.ETJson {
return ErrInvalidTypeForJSON.GenWithStackByArgs(0, "json_search")
return ErrInvalidTypeForJSON.GenWithStackByArgs(1, "json_search")
}
return nil
}
Expand Down Expand Up @@ -1640,7 +1639,7 @@ func (c *jsonKeysFunctionClass) verifyArgs(args []Expression) error {
return err
}
if evalType := args[0].GetType().EvalType(); evalType != types.ETString && evalType != types.ETJson {
return ErrInvalidTypeForJSON.GenWithStackByArgs(0, "json_keys")
return ErrInvalidTypeForJSON.GenWithStackByArgs(1, "json_keys")
}
return nil
}
Expand Down
2 changes: 1 addition & 1 deletion pkg/expression/builtin_json_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -503,7 +503,7 @@ func TestJSONContains(t *testing.T) {
}
for _, cs := range cases {
_, err := fc.getFunction(ctx, datumsToConstants(types.MakeDatums(cs.arg1, cs.arg2)))
require.True(t, types.ErrInvalidJSONData.Equal(err))
require.True(t, ErrInvalidTypeForJSON.Equal(err))
}
}

Expand Down
10 changes: 4 additions & 6 deletions pkg/expression/builtin_json_vec.go
Original file line number Diff line number Diff line change
Expand Up @@ -20,7 +20,6 @@ import (
"strconv"
"strings"

"github.com/pingcap/errors"
"github.com/pingcap/tidb/pkg/parser/ast"
"github.com/pingcap/tidb/pkg/sessionctx"
"github.com/pingcap/tidb/pkg/types"
Expand Down Expand Up @@ -570,7 +569,7 @@ func (b *builtinJSONSearchSig) vecEvalJSON(input *chunk.Chunk, result *chunk.Col
}
pathExpr, err := types.ParseJSONPathExpr(pathBufs[j].GetString(i))
if err != nil {
return types.ErrInvalidJSONPath.GenWithStackByArgs(pathBufs[j].GetString(i))
return err
}
pathExprs = append(pathExprs, pathExpr)
}
Expand Down Expand Up @@ -651,8 +650,7 @@ func (b *builtinJSONObjectSig) vecEvalJSON(input *chunk.Chunk, result *chunk.Col
var value types.BinaryJSON
for j := 0; j < nr; j++ {
if keyCol.IsNull(j) {
err := errors.New("JSON documents may not contain NULL member names")
return err
return types.ErrJSONDocumentNULLKey
}
key = keyCol.GetString(j)
if valueCol.IsNull(j) {
Expand Down Expand Up @@ -730,7 +728,7 @@ func (b *builtinJSONArrayInsertSig) vecEvalJSON(input *chunk.Chunk, result *chun
}
pathExpr, err = types.ParseJSONPathExpr(pathBufs[j].GetString(i))
if err != nil {
return types.ErrInvalidJSONPath.GenWithStackByArgs(pathBufs[j].GetString(i))
return err
}
if pathExpr.CouldMatchMultipleValues() {
return types.ErrInvalidJSONPathMultipleSelection
Expand Down Expand Up @@ -1161,7 +1159,7 @@ func (b *builtinJSONContainsPathSig) vecEvalInt(input *chunk.Chunk, result *chun
}
containType := strings.ToLower(typeBuf.GetString(i))
if containType != types.JSONContainsPathAll && containType != types.JSONContainsPathOne {
return types.ErrInvalidJSONContainsPathType
return types.ErrJSONBadOneOrAllArg.GenWithStackByArgs("json_contains_path")
}
obj := jsonBuf.GetJSON(i)
contains := int64(1)
Expand Down
4 changes: 4 additions & 0 deletions pkg/parser/mysql/errcode.go
Original file line number Diff line number Diff line change
Expand Up @@ -896,11 +896,14 @@ const (
ErrDependentByGeneratedColumn = 3108
ErrGeneratedColumnRefAutoInc = 3109
ErrInvalidJSONText = 3140
ErrInvalidJSONTextInParam = 3141
ErrInvalidJSONPath = 3143
ErrInvalidTypeForJSON = 3146
ErrInvalidJSONPathWildcard = 3149
ErrInvalidJSONContainsPathType = 3150
ErrJSONUsedAsKey = 3152
ErrJSONVacuousPath = 3153
ErrJSONBadOneOrAllArg = 3154
ErrJSONDocumentNULLKey = 3158
ErrBadUser = 3162
ErrUserAlreadyExists = 3163
Expand Down Expand Up @@ -944,6 +947,7 @@ const (
ErrFKIncompatibleColumns = 3780
ErrFunctionalIndexRowValueIsNotAllowed = 3800
ErrDependentByFunctionalIndex = 3837
ErrInvalidJSONType = 3853
ErrInvalidJsonValueForFuncIndex = 3903 //nolint: revive
ErrJsonValueOutOfRangeForFuncIndex = 3904 //nolint: revive
ErrFunctionalIndexDataIsTooLong = 3907
Expand Down
6 changes: 5 additions & 1 deletion pkg/parser/mysql/errname.go
Original file line number Diff line number Diff line change
Expand Up @@ -100,6 +100,8 @@ var MySQLErrName = map[uint16]*ErrMessage{
ErrTooLongKey: Message("Specified key was too long (%d bytes); max key length is %d bytes", nil),
ErrKeyColumnDoesNotExits: Message("Key column '%-.192s' doesn't exist in table", nil),
ErrBlobUsedAsKey: Message("BLOB column '%-.192s' can't be used in key specification with the used table type", nil),
ErrJSONVacuousPath: Message("The path expression '$' is not allowed in this context.", nil),
ErrJSONBadOneOrAllArg: Message("The oneOrAll argument to %s may take these values: 'one' or 'all'.", nil),
ErrTooBigFieldlength: Message("Column length too big for column '%-.192s' (max = %d); use BLOB or TEXT instead", nil),
ErrWrongAutoKey: Message("Incorrect table definition; there can be only one auto column and it must be defined as a key", nil),
ErrReady: Message("%s: ready for connections.\nVersion: '%s' socket: '%s' port: %d", nil),
Expand Down Expand Up @@ -903,9 +905,10 @@ var MySQLErrName = map[uint16]*ErrMessage{
ErrIncorrectType: Message("Incorrect type for argument %s in function %s.", nil),
ErrInvalidJSONData: Message("Invalid JSON data provided to function %s: %s", nil),
ErrInvalidJSONText: Message("Invalid JSON text: %-.192s", nil),
ErrInvalidJSONTextInParam: Message("Invalid JSON text in argument %d to function %s: \"%s\" at position %d.", nil),
ErrInvalidJSONPath: Message("Invalid JSON path expression %s.", nil),
ErrInvalidTypeForJSON: Message("Invalid data type for JSON data in argument %d to function %s; a JSON string or JSON type is required.", nil),
ErrInvalidJSONPathWildcard: Message("In this situation, path expressions may not contain the * and ** tokens.", nil),
ErrInvalidJSONPathWildcard: Message("In this situation, path expressions may not contain the * and ** tokens or an array range.", nil),
ErrInvalidJSONContainsPathType: Message("The second argument can only be either 'one' or 'all'.", nil),
ErrJSONUsedAsKey: Message("JSON column '%-.192s' cannot be used in key specification.", nil),
ErrJSONDocumentNULLKey: Message("JSON documents may not contain NULL member names.", nil),
Expand Down Expand Up @@ -952,6 +955,7 @@ var MySQLErrName = map[uint16]*ErrMessage{
ErrFKIncompatibleColumns: Message("Referencing column '%s' and referenced column '%s' in foreign key constraint '%s' are incompatible.", nil),
ErrFunctionalIndexRowValueIsNotAllowed: Message("Expression of functional index '%s' cannot refer to a row value", nil),
ErrDependentByFunctionalIndex: Message("Column '%s' has a functional index dependency and cannot be dropped or renamed", nil),
ErrInvalidJSONType: Message("Invalid JSON type in argument %d to function %s; an %s is required.", nil),
ErrInvalidJsonValueForFuncIndex: Message("Invalid JSON value for CAST for functional index '%s'", nil),
ErrJsonValueOutOfRangeForFuncIndex: Message("Out of range JSON value for CAST for functional index '%s'", nil),
ErrFunctionalIndexDataIsTooLong: Message("Data too long for functional index '%s'", nil),
Expand Down
Loading

0 comments on commit a6096e5

Please sign in to comment.