From a6096e5aedc8842bfd96a66cf71098a0f94d6efd Mon Sep 17 00:00:00 2001 From: Ti Chi Robot Date: Fri, 12 Jul 2024 14:52:35 +0800 Subject: [PATCH] expression: Fix errors for JSON functions (#53800) (#54561) close pingcap/tidb#53799 --- errors.toml | 22 +++++++- pkg/errno/errcode.go | 4 ++ pkg/errno/errname.go | 6 +- pkg/expression/builtin_json.go | 33 ++++++----- pkg/expression/builtin_json_test.go | 2 +- pkg/expression/builtin_json_vec.go | 10 ++-- pkg/parser/mysql/errcode.go | 4 ++ pkg/parser/mysql/errname.go | 6 +- pkg/parser/mysql/state.go | 6 ++ pkg/types/errors.go | 4 ++ pkg/types/json_binary_functions.go | 11 ++-- pkg/types/json_constants.go | 4 ++ .../integrationtest/r/expression/json.result | 56 ++++++++++++++++--- tests/integrationtest/t/expression/json.test | 43 ++++++++++++++ 14 files changed, 169 insertions(+), 42 deletions(-) diff --git a/errors.toml b/errors.toml index d27481aae5cd1..d657d652037fe 100644 --- a/errors.toml +++ b/errors.toml @@ -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. @@ -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"] @@ -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 @@ -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 diff --git a/pkg/errno/errcode.go b/pkg/errno/errcode.go index 3f2bde189f854..76cb3427176a0 100644 --- a/pkg/errno/errcode.go +++ b/pkg/errno/errcode.go @@ -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 @@ -927,6 +930,7 @@ const ( ErrCheckConstraintDupName = 3822 ErrCheckConstraintClauseUsingFKReferActionColumn = 3823 ErrDependentByFunctionalIndex = 3837 + ErrInvalidJSONType = 3853 ErrCannotConvertString = 3854 ErrDependentByPartitionFunctional = 3855 ErrInvalidJSONValueForFuncIndex = 3903 diff --git a/pkg/errno/errname.go b/pkg/errno/errname.go index 569caa32ba04d..2764a44e07d5f 100644 --- a/pkg/errno/errname.go +++ b/pkg/errno/errname.go @@ -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), @@ -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), @@ -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), diff --git a/pkg/expression/builtin_json.go b/pkg/expression/builtin_json.go index 58b2b39f9dbe8..06465008b4d09 100644 --- a/pkg/expression/builtin_json.go +++ b/pkg/expression/builtin_json.go @@ -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 } @@ -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 @@ -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) @@ -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 } @@ -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) @@ -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 } @@ -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 } @@ -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 } @@ -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 @@ -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 @@ -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 @@ -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 @@ -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 } @@ -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 } diff --git a/pkg/expression/builtin_json_test.go b/pkg/expression/builtin_json_test.go index 993323d6376f5..2ce50f2d1460b 100644 --- a/pkg/expression/builtin_json_test.go +++ b/pkg/expression/builtin_json_test.go @@ -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)) } } diff --git a/pkg/expression/builtin_json_vec.go b/pkg/expression/builtin_json_vec.go index f4f2aab04c929..846e0fcfe1b9e 100644 --- a/pkg/expression/builtin_json_vec.go +++ b/pkg/expression/builtin_json_vec.go @@ -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" @@ -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) } @@ -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) { @@ -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 @@ -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) diff --git a/pkg/parser/mysql/errcode.go b/pkg/parser/mysql/errcode.go index 3a217f93a93ec..f7ecb9fb12ad0 100644 --- a/pkg/parser/mysql/errcode.go +++ b/pkg/parser/mysql/errcode.go @@ -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 @@ -944,6 +947,7 @@ const ( ErrFKIncompatibleColumns = 3780 ErrFunctionalIndexRowValueIsNotAllowed = 3800 ErrDependentByFunctionalIndex = 3837 + ErrInvalidJSONType = 3853 ErrInvalidJsonValueForFuncIndex = 3903 //nolint: revive ErrJsonValueOutOfRangeForFuncIndex = 3904 //nolint: revive ErrFunctionalIndexDataIsTooLong = 3907 diff --git a/pkg/parser/mysql/errname.go b/pkg/parser/mysql/errname.go index 6c140adc5f05e..29ede7122f359 100644 --- a/pkg/parser/mysql/errname.go +++ b/pkg/parser/mysql/errname.go @@ -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), @@ -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), @@ -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), diff --git a/pkg/parser/mysql/state.go b/pkg/parser/mysql/state.go index 88fcd8a09b396..ecfb9e2011446 100644 --- a/pkg/parser/mysql/state.go +++ b/pkg/parser/mysql/state.go @@ -251,10 +251,16 @@ var MySQLState = map[uint16]string{ ErrGeneratedColumnNonPrior: "HY000", ErrDependentByGeneratedColumn: "HY000", ErrInvalidJSONText: "22032", + ErrInvalidJSONTextInParam: "22032", ErrInvalidJSONPath: "42000", ErrInvalidJSONData: "22032", ErrInvalidJSONPathWildcard: "42000", ErrJSONUsedAsKey: "42000", + ErrJSONVacuousPath: "42000", + ErrJSONBadOneOrAllArg: "42000", ErrJSONDocumentNULLKey: "22032", ErrInvalidJSONPathArrayCell: "42000", + ErrInvalidTypeForJSON: "22032", + ErrInvalidJsonValueForFuncIndex: "22018", + ErrInvalidJSONType: "22032", } diff --git a/pkg/types/errors.go b/pkg/types/errors.go index 8c7dffcf73b8b..08f0fe6bfb977 100644 --- a/pkg/types/errors.go +++ b/pkg/types/errors.go @@ -94,4 +94,8 @@ var ( ErrPartitionColumnStatsMissing = dbterror.ClassTypes.NewStd(mysql.ErrPartitionColumnStatsMissing) // ErrIncorrectDatetimeValue is returned when the input value is in wrong format for datetime. ErrIncorrectDatetimeValue = dbterror.ClassTypes.NewStd(mysql.ErrIncorrectDatetimeValue) + // ErrJSONBadOneOrAllArg is returned when the one_or_all argument isn't 'one' or 'all'. + ErrJSONBadOneOrAllArg = dbterror.ClassTypes.NewStd(mysql.ErrJSONBadOneOrAllArg) + // ErrJSONVacuousPath is returned for path expressions that are not allowed in that context. + ErrJSONVacuousPath = dbterror.ClassTypes.NewStd(mysql.ErrJSONVacuousPath) ) diff --git a/pkg/types/json_binary_functions.go b/pkg/types/json_binary_functions.go index 0dbf067c40a4a..4d3e95604b016 100644 --- a/pkg/types/json_binary_functions.go +++ b/pkg/types/json_binary_functions.go @@ -402,8 +402,7 @@ func (bj BinaryJSON) Modify(pathExprList []JSONPathExpression, values []BinaryJS } for _, pathExpr := range pathExprList { if pathExpr.flags.containsAnyAsterisk() || pathExpr.flags.containsAnyRange() { - // TODO: should return 3149(42000) - return retj, errors.New("Invalid path expression") + return retj, ErrInvalidJSONPathMultipleSelection } } for i := 0; i < len(pathExprList); i++ { @@ -480,12 +479,10 @@ func (bj BinaryJSON) ArrayInsert(pathExpr JSONPathExpression, value BinaryJSON) func (bj BinaryJSON) Remove(pathExprList []JSONPathExpression) (BinaryJSON, error) { for _, pathExpr := range pathExprList { if len(pathExpr.legs) == 0 { - // TODO: should return 3153(42000) - return bj, errors.New("Invalid path expression") + return bj, ErrJSONVacuousPath } if pathExpr.flags.containsAnyAsterisk() || pathExpr.flags.containsAnyRange() { - // TODO: should return 3149(42000) - return bj, errors.New("Invalid path expression") + return bj, ErrInvalidJSONPathMultipleSelection } modifer := &binaryModifier{bj: bj} bj = modifer.remove(pathExpr) @@ -1188,7 +1185,7 @@ func (bj BinaryJSON) GetElemDepth() int { // [https://dev.mysql.com/doc/refman/5.7/en/json-search-functions.html#function_json-search] func (bj BinaryJSON) Search(containType string, search string, escape byte, pathExpres []JSONPathExpression) (res BinaryJSON, isNull bool, err error) { if containType != JSONContainsPathOne && containType != JSONContainsPathAll { - return res, true, ErrInvalidJSONPath + return res, true, ErrJSONBadOneOrAllArg.GenWithStackByArgs("json_search") } patChars, patTypes := stringutil.CompilePattern(search, escape) diff --git a/pkg/types/json_constants.go b/pkg/types/json_constants.go index 9c463815526f6..9fb2a2e98ebbc 100644 --- a/pkg/types/json_constants.go +++ b/pkg/types/json_constants.go @@ -220,6 +220,10 @@ const ( var ( // ErrInvalidJSONText means invalid JSON text. ErrInvalidJSONText = dbterror.ClassJSON.NewStd(mysql.ErrInvalidJSONText) + // ErrInvalidJSONType means invalid JSON type. + ErrInvalidJSONType = dbterror.ClassJSON.NewStd(mysql.ErrInvalidJSONType) + // ErrInvalidJSONTextInParam means invalid JSON text in arguments. + ErrInvalidJSONTextInParam = dbterror.ClassJSON.NewStd(mysql.ErrInvalidJSONTextInParam) // ErrInvalidJSONPath means invalid JSON path. ErrInvalidJSONPath = dbterror.ClassJSON.NewStd(mysql.ErrInvalidJSONPath) // ErrInvalidJSONCharset means invalid JSON charset. diff --git a/tests/integrationtest/r/expression/json.result b/tests/integrationtest/r/expression/json.result index d2ad35eb6ee30..45a899cb62725 100644 --- a/tests/integrationtest/r/expression/json.result +++ b/tests/integrationtest/r/expression/json.result @@ -180,19 +180,19 @@ select json_valid('"2019-8-19"'); json_valid('"2019-8-19"') 1 select json_merge(1, 2); -Error 3146 (HY000): Invalid data type for JSON data in argument 0 to function json_merge; a JSON string or JSON type is required. +Error 3146 (22032): Invalid data type for JSON data in argument 1 to function json_merge; a JSON string or JSON type is required. select json_merge_preserve(1, 2); -Error 3146 (HY000): Invalid data type for JSON data in argument 0 to function json_merge_preserve; a JSON string or JSON type is required. +Error 3146 (22032): Invalid data type for JSON data in argument 1 to function json_merge_preserve; a JSON string or JSON type is required. select json_merge_patch(1, 2); -Error 3146 (HY000): Invalid data type for JSON data in argument 0 to function json_merge_patch; a JSON string or JSON type is required. +Error 3146 (22032): Invalid data type for JSON data in argument 1 to function json_merge_patch; a JSON string or JSON type is required. select JSON_CONTAINS_PATH(1, 'one', '$.a'); -Error 3146 (HY000): Invalid data type for JSON data in argument 0 to function json_contains_path; a JSON string or JSON type is required. +Error 3146 (22032): Invalid data type for JSON data in argument 1 to function json_contains_path; a JSON string or JSON type is required. select json_search(1, 'one', '$.a'); -Error 3146 (HY000): Invalid data type for JSON data in argument 0 to function json_search; a JSON string or JSON type is required. +Error 3146 (22032): Invalid data type for JSON data in argument 1 to function json_search; a JSON string or JSON type is required. select json_keys(1, '$.a'); -Error 3146 (HY000): Invalid data type for JSON data in argument 0 to function json_keys; a JSON string or JSON type is required. +Error 3146 (22032): Invalid data type for JSON data in argument 1 to function json_keys; a JSON string or JSON type is required. select JSON_extract(1, '$.a'); -Error 3146 (HY000): Invalid data type for JSON data in argument 0 to function json_extract; a JSON string or JSON type is required. +Error 3146 (22032): Invalid data type for JSON data in argument 1 to function json_extract; a JSON string or JSON type is required. DROP TABLE IF EXISTS table_json; CREATE TABLE table_json(a json, b VARCHAR(255)); INSERT INTO table_json values('{"\\"hello\\"": "world", "a": [1, "2", {"aa": "bb"}, 4.0, {"aa": "cc"}], "b": true, "c": ["d"]}', '{"\\"hello\\"": "world", "a": [1, "2", {"aa": "bb"}, 4.0, {"aa": "cc"}], "b": true, "c": ["d"]}'); @@ -385,7 +385,7 @@ select json_contains('{"a": 1}', '1', "$.c"), json_contains('{"a": [1, 2]}', '1' json_contains('{"a": 1}', '1', "$.c") json_contains('{"a": [1, 2]}', '1', "$.a[2]") json_contains('{"a": [1, {"a": 1}]}', '1', "$.a[1].b") NULL NULL NULL select json_contains('1','1','$.*'); -Error 3149 (42000): In this situation, path expressions may not contain the * and ** tokens or range selection. +Error 3149 (42000): In this situation, path expressions may not contain the * and ** tokens or an array range. select json_contains_path(NULL, 'one', "$.c"), json_contains_path(NULL, 'all', "$.c"), @@ -623,3 +623,43 @@ select 1 from t where cast(BINARY vc as json) = '1'; 1 select 1 from t where cast(BINARY c as json) = '1'; 1 +SELECT JSON_CONTAINS(123, ''); +Error 3146 (22032): Invalid data type for JSON data in argument 1 to function json_contains; a JSON string or JSON type is required. +SELECT JSON_CONTAINS('{}', 123); +Error 3146 (22032): Invalid data type for JSON data in argument 2 to function json_contains; a JSON string or JSON type is required. +SELECT JSON_OVERLAPS(123, ''); +Error 3146 (22032): Invalid data type for JSON data in argument 1 to function json_overlaps; a JSON string or JSON type is required. +SELECT JSON_OVERLAPS('{}', 123); +Error 3146 (22032): Invalid data type for JSON data in argument 2 to function json_overlaps; a JSON string or JSON type is required. +SELECT 'abc' MEMBER OF(123); +Error 3146 (22032): Invalid data type for JSON data in argument 2 to function member of; a JSON string or JSON type is required. +SELECT JSON_SEARCH('{}',3,4); +Error 3154 (42000): The oneOrAll argument to json_search may take these values: 'one' or 'all'. +SELECT JSON_ARRAY_APPEND('[]','abc','def'); +Error 3143 (42000): Invalid JSON path expression. The error is around character position 1. +SELECT JSON_ARRAY_INSERT('{}','abc','def'); +Error 3143 (42000): Invalid JSON path expression. The error is around character position 1. +SELECT JSON_REMOVE('{}','$'); +Error 3153 (42000): The path expression '$' is not allowed in this context. +SELECT JSON_REMOVE('{}','$.*'); +Error 3149 (42000): In this situation, path expressions may not contain the * and ** tokens or an array range. +SELECT JSON_SET('{}','$.*',''); +Error 3149 (42000): In this situation, path expressions may not contain the * and ** tokens or an array range. +SELECT JSON_CONTAINS_PATH('{}', 'on', '$.a'); +Error 3154 (42000): The oneOrAll argument to json_contains_path may take these values: 'one' or 'all'. +SELECT JSON_SEARCH('{}', 'on', '$.a'); +Error 3154 (42000): The oneOrAll argument to json_search may take these values: 'one' or 'all'. +SELECT JSON_OBJECT(NULL,'abc'); +Error 3158 (22032): JSON documents may not contain NULL member names. +SELECT JSON_SCHEMA_VALID(1, '{}'); +Error 1305 (42000): FUNCTION expression__json.json_schema_valid does not exist +SELECT JSON_SCHEMA_VALID('{}', 1); +Error 1305 (42000): FUNCTION expression__json.json_schema_valid does not exist +SELECT JSON_SCHEMA_VALID('','{}'); +Error 1305 (42000): FUNCTION expression__json.json_schema_valid does not exist +SELECT JSON_SCHEMA_VALID('{}',''); +Error 1305 (42000): FUNCTION expression__json.json_schema_valid does not exist +SELECT JSON_SCHEMA_VALID('', NULL); +Error 1305 (42000): FUNCTION expression__json.json_schema_valid does not exist +SELECT JSON_SCHEMA_VALID('1','{}'); +Error 1305 (42000): FUNCTION expression__json.json_schema_valid does not exist diff --git a/tests/integrationtest/t/expression/json.test b/tests/integrationtest/t/expression/json.test index a7b613438623c..d80d95764abfd 100644 --- a/tests/integrationtest/t/expression/json.test +++ b/tests/integrationtest/t/expression/json.test @@ -371,3 +371,46 @@ select 1 from t where cast(vc as json) = '1'; select 1 from t where cast(c as json) = '1'; select 1 from t where cast(BINARY vc as json) = '1'; select 1 from t where cast(BINARY c as json) = '1'; + +# Test argment errors +# https://github.com/pingcap/tidb/issues/53799 +-- error 3146 +SELECT JSON_CONTAINS(123, ''); +-- error 3146 +SELECT JSON_CONTAINS('{}', 123); +-- error 3146 +SELECT JSON_OVERLAPS(123, ''); +-- error 3146 +SELECT JSON_OVERLAPS('{}', 123); +-- error 3146 +SELECT 'abc' MEMBER OF(123); +-- error 3154 +SELECT JSON_SEARCH('{}',3,4); +-- error 3143 +SELECT JSON_ARRAY_APPEND('[]','abc','def'); +-- error 3143 +SELECT JSON_ARRAY_INSERT('{}','abc','def'); +-- error 3153 +SELECT JSON_REMOVE('{}','$'); +-- error 3149 +SELECT JSON_REMOVE('{}','$.*'); +-- error 3149 +SELECT JSON_SET('{}','$.*',''); +-- error 3154 +SELECT JSON_CONTAINS_PATH('{}', 'on', '$.a'); +-- error 3154 +SELECT JSON_SEARCH('{}', 'on', '$.a'); +-- error 3158 +SELECT JSON_OBJECT(NULL,'abc'); +-- error 3146 +SELECT JSON_SCHEMA_VALID(1, '{}'); +-- error 3146 +SELECT JSON_SCHEMA_VALID('{}', 1); +-- error 3141 +SELECT JSON_SCHEMA_VALID('','{}'); +-- error 3141 +SELECT JSON_SCHEMA_VALID('{}',''); +-- error 3141 +SELECT JSON_SCHEMA_VALID('', NULL); +-- error 3853 +SELECT JSON_SCHEMA_VALID('1','{}');