diff --git a/errors.toml b/errors.toml index 0c5f5765d1269..baf8ee9bbb6d0 100644 --- a/errors.toml +++ b/errors.toml @@ -1696,6 +1696,21 @@ error = ''' Invalid data type for JSON data in argument %d to function %s; a JSON string or JSON type is required. ''' +["expression:3752"] +error = ''' +Value is out of range for expression index '%s' at row %d +''' + +["expression:3903"] +error = ''' +Invalid JSON value for CAST for expression index '%s' +''' + +["expression:3907"] +error = ''' +Data too long for expression index '%s' +''' + ["expression:8128"] error = ''' Invalid TABLESAMPLE: %s diff --git a/executor/insert_common.go b/executor/insert_common.go index d321f02a2f4fd..8440242f1dad5 100644 --- a/executor/insert_common.go +++ b/executor/insert_common.go @@ -388,7 +388,7 @@ func (e *InsertValues) evalRow(ctx context.Context, list []expression.Expression e.evalBuffer.SetDatum(offset, val1) } // Row may lack of generated column, autoIncrement column, empty column here. - return e.fillRow(ctx, row, hasValue) + return e.fillRow(ctx, row, hasValue, rowIdx) } var emptyRow chunk.Row @@ -422,7 +422,7 @@ func (e *InsertValues) fastEvalRow(ctx context.Context, list []expression.Expres offset := e.insertColumns[i].Offset row[offset], hasValue[offset] = val1, true } - return e.fillRow(ctx, row, hasValue) + return e.fillRow(ctx, row, hasValue, rowIdx) } // setValueForRefColumn set some default values for the row to eval the row value with other columns, @@ -562,7 +562,7 @@ func (e *InsertValues) getRow(ctx context.Context, vals []types.Datum) ([]types. hasValue[offset] = true } - return e.fillRow(ctx, row, hasValue) + return e.fillRow(ctx, row, hasValue, 0) } // getColDefaultValue gets the column default value. @@ -647,7 +647,7 @@ func (e *InsertValues) fillColValue(ctx context.Context, datum types.Datum, idx // `insert|replace values` can guarantee consecutive autoID in a batch. // Other statements like `insert select from` don't guarantee consecutive autoID. // https://dev.mysql.com/doc/refman/8.0/en/innodb-auto-increment-handling.html -func (e *InsertValues) fillRow(ctx context.Context, row []types.Datum, hasValue []bool) ([]types.Datum, error) { +func (e *InsertValues) fillRow(ctx context.Context, row []types.Datum, hasValue []bool, rowIdx int) ([]types.Datum, error) { gCols := make([]*table.Column, 0) tCols := e.Table.Cols() if e.hasExtraHandle { @@ -693,6 +693,9 @@ func (e *InsertValues) fillRow(ctx context.Context, row []types.Datum, hasValue for i, gCol := range gCols { colIdx := gCol.ColumnInfo.Offset val, err := e.GenExprs[i].Eval(chunk.MutRowFromDatums(row).ToRow()) + if err != nil && gCol.FieldType.IsArray() { + return nil, completeError(tbl, gCol.Offset, rowIdx, err) + } if e.ctx.GetSessionVars().StmtCtx.HandleTruncate(err) != nil { return nil, err } @@ -708,6 +711,29 @@ func (e *InsertValues) fillRow(ctx context.Context, row []types.Datum, hasValue return row, nil } +func completeError(tbl *model.TableInfo, offset int, rowIdx int, err error) error { + name := "expression_index" + for _, idx := range tbl.Indices { + for _, column := range idx.Columns { + if column.Offset == offset { + name = idx.Name.O + break + } + } + } + + if expression.ErrInvalidJSONForFuncIndex.Equal(err) { + return expression.ErrInvalidJSONForFuncIndex.GenWithStackByArgs(name) + } + if types.ErrOverflow.Equal(err) { + return expression.ErrDataOutOfRangeFuncIndex.GenWithStackByArgs(name, rowIdx+1) + } + if types.ErrDataTooLong.Equal(err) { + return expression.ErrFuncIndexDataIsTooLong.GenWithStackByArgs(name) + } + return err +} + // isAutoNull can help judge whether a datum is AutoIncrement Null quickly. // This used to help lazyFillAutoIncrement to find consecutive N datum backwards for batch autoID alloc. func (e *InsertValues) isAutoNull(ctx context.Context, d types.Datum, col *table.Column) bool { diff --git a/expression/builtin_cast.go b/expression/builtin_cast.go index e6257c4dd058c..545abd497a2da 100644 --- a/expression/builtin_cast.go +++ b/expression/builtin_cast.go @@ -36,6 +36,7 @@ import ( "github.com/pingcap/tidb/parser/mysql" "github.com/pingcap/tidb/parser/terror" "github.com/pingcap/tidb/sessionctx" + "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" @@ -420,7 +421,7 @@ func (c *castAsArrayFunctionClass) verifyArgs(args []Expression) error { } if args[0].GetType().EvalType() != types.ETJson { - return types.ErrInvalidJSONData.GenWithStackByArgs("1", "cast_as_array") + return ErrInvalidTypeForJSON.GenWithStackByArgs(1, "cast_as_array") } return nil @@ -467,9 +468,80 @@ func (b *castJSONAsArrayFunctionSig) evalJSON(row chunk.Row) (res types.BinaryJS return types.BinaryJSON{}, false, ErrNotSupportedYet.GenWithStackByArgs("CAST-ing Non-JSON Array type to array") } - // TODO: impl the cast(... as ... array) function + arrayVals := make([]any, 0, len(b.args)) + ft := b.tp.ArrayType() + f := convertJSON2Tp(ft.EvalType()) + if f == nil { + return types.BinaryJSON{}, false, ErrNotSupportedYet.GenWithStackByArgs("CAS-ing JSON to the target type") + } + sc := b.ctx.GetSessionVars().StmtCtx + originalOverflowAsWarning := sc.OverflowAsWarning + originIgnoreTruncate := sc.IgnoreTruncate + originTruncateAsWarning := sc.TruncateAsWarning + sc.OverflowAsWarning = false + sc.IgnoreTruncate = false + sc.TruncateAsWarning = false + defer func() { + sc.OverflowAsWarning = originalOverflowAsWarning + sc.IgnoreTruncate = originIgnoreTruncate + sc.TruncateAsWarning = originTruncateAsWarning + }() + for i := 0; i < val.GetElemCount(); i++ { + item, err := f(sc, val.ArrayGetElem(i), ft) + if err != nil { + return types.BinaryJSON{}, false, err + } + arrayVals = append(arrayVals, item) + } + return types.CreateBinaryJSON(arrayVals), false, nil +} - return types.BinaryJSON{}, false, nil +func convertJSON2Tp(eval types.EvalType) func(*stmtctx.StatementContext, types.BinaryJSON, *types.FieldType) (any, error) { + switch eval { + case types.ETString: + return func(sc *stmtctx.StatementContext, item types.BinaryJSON, tp *types.FieldType) (any, error) { + if item.TypeCode != types.JSONTypeCodeString { + return nil, ErrInvalidJSONForFuncIndex + } + return types.ProduceStrWithSpecifiedTp(string(item.GetString()), tp, sc, false) + } + case types.ETInt: + return func(sc *stmtctx.StatementContext, item types.BinaryJSON, tp *types.FieldType) (any, error) { + if item.TypeCode != types.JSONTypeCodeInt64 && item.TypeCode != types.JSONTypeCodeUint64 { + return nil, ErrInvalidJSONForFuncIndex + } + return types.ConvertJSONToInt(sc, item, mysql.HasUnsignedFlag(tp.GetFlag()), tp.GetType()) + } + case types.ETReal, types.ETDecimal: + return func(sc *stmtctx.StatementContext, item types.BinaryJSON, tp *types.FieldType) (any, error) { + if item.TypeCode != types.JSONTypeCodeInt64 && item.TypeCode != types.JSONTypeCodeUint64 && item.TypeCode != types.JSONTypeCodeFloat64 { + return nil, ErrInvalidJSONForFuncIndex + } + return types.ConvertJSONToFloat(sc, item) + } + case types.ETDatetime: + return func(sc *stmtctx.StatementContext, item types.BinaryJSON, tp *types.FieldType) (any, error) { + if (tp.GetType() == mysql.TypeDatetime && item.TypeCode != types.JSONTypeCodeDatetime) || (tp.GetType() == mysql.TypeDate && item.TypeCode != types.JSONTypeCodeDate) { + return nil, ErrInvalidJSONForFuncIndex + } + res := item.GetTime() + res.SetType(tp.GetType()) + if tp.GetType() == mysql.TypeDate { + // Truncate hh:mm:ss part if the type is Date. + res.SetCoreTime(types.FromDate(res.Year(), res.Month(), res.Day(), 0, 0, 0, 0)) + } + return res, nil + } + case types.ETDuration: + return func(sc *stmtctx.StatementContext, item types.BinaryJSON, tp *types.FieldType) (any, error) { + if item.TypeCode != types.JSONTypeCodeDuration { + return nil, ErrInvalidJSONForFuncIndex + } + return item.GetDuration(), nil + } + default: + return nil + } } type castAsJSONFunctionClass struct { diff --git a/expression/builtin_cast_test.go b/expression/builtin_cast_test.go index e670aad1e35ae..8208e0a0af0d6 100644 --- a/expression/builtin_cast_test.go +++ b/expression/builtin_cast_test.go @@ -1619,3 +1619,73 @@ func TestCastBinaryStringAsJSONSig(t *testing.T) { require.Equal(t, tt.resultStr, res.String()) } } + +func TestCastArrayFunc(t *testing.T) { + ctx := createContext(t) + tbl := []struct { + input interface{} + expected interface{} + tp *types.FieldType + success bool + buildFuncSuccess bool + }{ + { + []interface{}{int64(-1), int64(2), int64(3)}, + []interface{}{int64(-1), int64(2), int64(3)}, + types.NewFieldTypeBuilder().SetType(mysql.TypeLonglong).SetCharset(charset.CharsetBin).SetCollate(charset.CollationBin).SetArray(true).BuildP(), + true, + true, + }, + { + []interface{}{int64(-1), int64(2), int64(3)}, + nil, + types.NewFieldTypeBuilder().SetType(mysql.TypeString).SetCharset(charset.CharsetUTF8MB4).SetCollate(charset.CollationUTF8MB4).SetArray(true).BuildP(), + false, + true, + }, + { + []interface{}{"1"}, + nil, + types.NewFieldTypeBuilder().SetType(mysql.TypeLonglong).SetCharset(charset.CharsetBin).SetCollate(charset.CharsetBin).SetArray(true).BuildP(), + false, + true, + }, + { + []interface{}{"1", "2"}, + nil, + types.NewFieldTypeBuilder().SetType(mysql.TypeDouble).SetCharset(charset.CharsetBin).SetCollate(charset.CharsetBin).SetArray(true).BuildP(), + false, + true, + }, + { + []interface{}{int64(-1), 2.1, int64(3)}, + []interface{}{int64(-1), 2.1, int64(3)}, + types.NewFieldTypeBuilder().SetType(mysql.TypeDouble).SetCharset(charset.CharsetBin).SetCollate(charset.CharsetBin).SetArray(true).BuildP(), + true, + true, + }, + } + for _, tt := range tbl { + f, err := BuildCastFunctionWithCheck(ctx, datumsToConstants(types.MakeDatums(types.CreateBinaryJSON(tt.input)))[0], tt.tp) + if tt.buildFuncSuccess { + require.NoError(t, err, tt.input) + } else { + require.Error(t, err, tt.input) + continue + } + + val, isNull, err := f.EvalJSON(ctx, chunk.Row{}) + if tt.success { + require.NoError(t, err, tt.input) + if tt.expected == nil { + require.True(t, isNull, tt.input) + } else { + j1 := types.CreateBinaryJSON(tt.expected) + cmp := types.CompareBinaryJSON(j1, val) + require.Equal(t, 0, cmp, tt.input) + } + } else { + require.Error(t, err, tt.input) + } + } +} diff --git a/expression/errors.go b/expression/errors.go index c56737ec2fae3..62d5b89d547b9 100644 --- a/expression/errors.go +++ b/expression/errors.go @@ -38,6 +38,9 @@ var ( ErrInternal = dbterror.ClassOptimizer.NewStd(mysql.ErrInternal) ErrNoDB = dbterror.ClassOptimizer.NewStd(mysql.ErrNoDB) ErrNotSupportedYet = dbterror.ClassExpression.NewStd(mysql.ErrNotSupportedYet) + ErrInvalidJSONForFuncIndex = dbterror.ClassExpression.NewStd(mysql.ErrInvalidJSONValueForFuncIndex) + ErrDataOutOfRangeFuncIndex = dbterror.ClassExpression.NewStd(mysql.ErrDataOutOfRangeFunctionalIndex) + ErrFuncIndexDataIsTooLong = dbterror.ClassExpression.NewStd(mysql.ErrFunctionalIndexDataIsTooLong) // All the un-exported errors are defined here: errFunctionNotExists = dbterror.ClassExpression.NewStd(mysql.ErrSpDoesNotExist) diff --git a/expression/multi_valued_index_test.go b/expression/multi_valued_index_test.go index 788dadddab02c..33557726a1073 100644 --- a/expression/multi_valued_index_test.go +++ b/expression/multi_valued_index_test.go @@ -15,6 +15,7 @@ package expression_test import ( + "fmt" "testing" "github.com/pingcap/tidb/errno" @@ -36,7 +37,7 @@ func TestMultiValuedIndexDDL(t *testing.T) { tk.MustExec("drop table t") tk.MustGetErrCode("CREATE TABLE t(x INT, KEY k ((1 AND CAST(JSON_ARRAY(x) AS UNSIGNED ARRAY))));", errno.ErrNotSupportedYet) tk.MustGetErrCode("CREATE TABLE t1 (f1 json, key mvi((cast(cast(f1 as unsigned array) as unsigned array))));", errno.ErrNotSupportedYet) - tk.MustGetErrCode("CREATE TABLE t1 (f1 json, key mvi((cast(f1->>'$[*]' as unsigned array))));", errno.ErrInvalidJSONData) + tk.MustGetErrCode("CREATE TABLE t1 (f1 json, key mvi((cast(f1->>'$[*]' as unsigned array))));", errno.ErrInvalidTypeForJSON) tk.MustGetErrCode("CREATE TABLE t1 (f1 json, key mvi((cast(f1->'$[*]' as year array))));", errno.ErrNotSupportedYet) tk.MustGetErrCode("CREATE TABLE t1 (f1 json, key mvi((cast(f1->'$[*]' as json array))));", errno.ErrNotSupportedYet) tk.MustGetErrCode("CREATE TABLE t1 (f1 json, key mvi((cast(f1->'$[*]' as char(10) charset gbk array))));", errno.ErrNotSupportedYet) @@ -56,3 +57,171 @@ func TestMultiValuedIndexDDL(t *testing.T) { tk.MustExec("drop table t") tk.MustExec("create table t(a json, b int, index idx3(b, (cast(a as signed array))));") } + +func TestMultiValuedIndexDML(t *testing.T) { + store := testkit.CreateMockStore(t) + + tk := testkit.NewTestKit(t, store) + tk.MustExec("USE test;") + + mode := []string{`''`, `default`} + + for _, m := range mode { + tk.MustExec(fmt.Sprintf("set @@sql_mode=%s", m)) + + tk.MustExec(`drop table if exists t;`) + tk.MustExec(`create table t(a json, index idx((cast(a as unsigned array))));`) + tk.MustExec(`insert into t values ('[1,2,3]');`) + tk.MustGetErrCode(`insert into t values ('[-1]');`, errno.ErrDataOutOfRangeFunctionalIndex) + tk.MustGetErrCode(`insert into t values ('["1"]');`, errno.ErrInvalidJSONValueForFuncIndex) + tk.MustGetErrCode(`insert into t values ('["a"]');`, errno.ErrInvalidJSONValueForFuncIndex) + tk.MustGetErrCode(`insert into t values ('["汉字"]');`, errno.ErrInvalidJSONValueForFuncIndex) + tk.MustGetErrCode(`insert into t values ('[1.2]');`, errno.ErrInvalidJSONValueForFuncIndex) + tk.MustGetErrCode(`insert into t values ('[1.0]');`, errno.ErrInvalidJSONValueForFuncIndex) + tk.MustGetErrCode(`insert into t values (json_array(cast("11:00:00" as time)));`, errno.ErrInvalidJSONValueForFuncIndex) + tk.MustGetErrCode(`insert into t values (json_array(cast("2022-02-02" as date)));`, errno.ErrInvalidJSONValueForFuncIndex) + tk.MustGetErrCode(`insert into t values (json_array(cast("2022-02-02 11:00:00" as datetime)));`, errno.ErrInvalidJSONValueForFuncIndex) + tk.MustGetErrCode(`insert into t values (json_array(cast('{"a":1}' as json)));`, errno.ErrInvalidJSONValueForFuncIndex) + + tk.MustExec(`drop table if exists t;`) + tk.MustExec(`create table t(a json, index idx((cast(a as signed array))));`) + tk.MustExec(`insert into t values ('[1,2,3]');`) + tk.MustExec(`insert into t values ('[-1]');`) + tk.MustGetErrCode(`insert into t values ('["1"]');`, errno.ErrInvalidJSONValueForFuncIndex) + tk.MustGetErrCode(`insert into t values ('["a"]');`, errno.ErrInvalidJSONValueForFuncIndex) + tk.MustGetErrCode(`insert into t values ('["汉字"]');`, errno.ErrInvalidJSONValueForFuncIndex) + tk.MustGetErrCode(`insert into t values ('[1.2]');`, errno.ErrInvalidJSONValueForFuncIndex) + tk.MustGetErrCode(`insert into t values ('[1.0]');`, errno.ErrInvalidJSONValueForFuncIndex) + tk.MustGetErrCode(`insert into t values (json_array(cast("11:00:00" as time)));`, errno.ErrInvalidJSONValueForFuncIndex) + tk.MustGetErrCode(`insert into t values (json_array(cast("2022-02-02" as date)));`, errno.ErrInvalidJSONValueForFuncIndex) + tk.MustGetErrCode(`insert into t values (json_array(cast("2022-02-02 11:00:00" as datetime)));`, errno.ErrInvalidJSONValueForFuncIndex) + tk.MustGetErrCode(`insert into t values (json_array(cast('{"a":1}' as json)));`, errno.ErrInvalidJSONValueForFuncIndex) + + tk.MustExec(`drop table if exists t;`) + tk.MustExec(`create table t(a json, index idx((cast(a as double array))));`) + tk.MustExec(`insert into t values ('[1,2,3]');`) + tk.MustExec(`insert into t values ('[-1]');`) + tk.MustGetErrCode(`insert into t values ('["1"]');`, errno.ErrInvalidJSONValueForFuncIndex) + tk.MustGetErrCode(`insert into t values ('["a"]');`, errno.ErrInvalidJSONValueForFuncIndex) + tk.MustGetErrCode(`insert into t values ('["汉字"]');`, errno.ErrInvalidJSONValueForFuncIndex) + tk.MustExec(`insert into t values ('[1.2]');`) + tk.MustExec(`insert into t values ('[1.0]');`) + tk.MustGetErrCode(`insert into t values (json_array(cast("11:00:00" as time)));`, errno.ErrInvalidJSONValueForFuncIndex) + tk.MustGetErrCode(`insert into t values (json_array(cast("2022-02-02" as date)));`, errno.ErrInvalidJSONValueForFuncIndex) + tk.MustGetErrCode(`insert into t values (json_array(cast("2022-02-02 11:00:00" as datetime)));`, errno.ErrInvalidJSONValueForFuncIndex) + tk.MustGetErrCode(`insert into t values (json_array(cast('{"a":1}' as json)));`, errno.ErrInvalidJSONValueForFuncIndex) + + tk.MustExec(`drop table if exists t;`) + tk.MustExec(`create table t(a json, index idx((cast(a as decimal(10, 2) array))));`) + tk.MustExec(`insert into t values ('[1,2,3]');`) + tk.MustExec(`insert into t values ('[-1]');`) + tk.MustGetErrCode(`insert into t values ('["1"]');`, errno.ErrInvalidJSONValueForFuncIndex) + tk.MustGetErrCode(`insert into t values ('["a"]');`, errno.ErrInvalidJSONValueForFuncIndex) + tk.MustGetErrCode(`insert into t values ('["汉字"]');`, errno.ErrInvalidJSONValueForFuncIndex) + tk.MustExec(`insert into t values ('[1.2]');`) + tk.MustExec(`insert into t values ('[1.0]');`) + tk.MustExec(`insert into t values ('[1.1102]');`) + tk.MustGetErrCode(`insert into t values (json_array(cast("11:00:00" as time)));`, errno.ErrInvalidJSONValueForFuncIndex) + tk.MustGetErrCode(`insert into t values (json_array(cast("2022-02-02" as date)));`, errno.ErrInvalidJSONValueForFuncIndex) + tk.MustGetErrCode(`insert into t values (json_array(cast("2022-02-02 11:00:00" as datetime)));`, errno.ErrInvalidJSONValueForFuncIndex) + tk.MustGetErrCode(`insert into t values (json_array(cast('{"a":1}' as json)));`, errno.ErrInvalidJSONValueForFuncIndex) + + tk.MustExec(`drop table if exists t;`) + tk.MustExec(`create table t(a json, index idx((cast(a as char(1) array))));`) + tk.MustGetErrCode(`insert into t values ('[1,2,3]');`, errno.ErrInvalidJSONValueForFuncIndex) + tk.MustGetErrCode(`insert into t values ('[-1]');`, errno.ErrInvalidJSONValueForFuncIndex) + tk.MustExec(`insert into t values ('["1"]');`) + tk.MustExec(`insert into t values ('["a"]');`) + tk.MustGetErrCode(`insert into t values ('["汉字"]');`, errno.ErrFunctionalIndexDataIsTooLong) + tk.MustGetErrCode(`insert into t values ('[1.2]');`, errno.ErrInvalidJSONValueForFuncIndex) + tk.MustGetErrCode(`insert into t values ('[1.0]');`, errno.ErrInvalidJSONValueForFuncIndex) + tk.MustGetErrCode(`insert into t values (json_array(cast("11:00:00" as time)));`, errno.ErrInvalidJSONValueForFuncIndex) + tk.MustGetErrCode(`insert into t values (json_array(cast("2022-02-02" as date)));`, errno.ErrInvalidJSONValueForFuncIndex) + tk.MustGetErrCode(`insert into t values (json_array(cast("2022-02-02 11:00:00" as datetime)));`, errno.ErrInvalidJSONValueForFuncIndex) + tk.MustGetErrCode(`insert into t values (json_array(cast('{"a":1}' as json)));`, errno.ErrInvalidJSONValueForFuncIndex) + + tk.MustExec(`drop table if exists t;`) + tk.MustExec(`create table t(a json, index idx((cast(a as char(2) array))));`) + tk.MustGetErrCode(`insert into t values ('[1,2,3]');`, errno.ErrInvalidJSONValueForFuncIndex) + tk.MustGetErrCode(`insert into t values ('[-1]');`, errno.ErrInvalidJSONValueForFuncIndex) + tk.MustExec(`insert into t values ('["1"]');`) + tk.MustExec(`insert into t values ('["a"]');`) + tk.MustExec(`insert into t values ('["汉字"]');`) + tk.MustGetErrCode(`insert into t values ('[1.2]');`, errno.ErrInvalidJSONValueForFuncIndex) + tk.MustGetErrCode(`insert into t values ('[1.0]');`, errno.ErrInvalidJSONValueForFuncIndex) + tk.MustGetErrCode(`insert into t values (json_array(cast("11:00:00" as time)));`, errno.ErrInvalidJSONValueForFuncIndex) + tk.MustGetErrCode(`insert into t values (json_array(cast("2022-02-02" as date)));`, errno.ErrInvalidJSONValueForFuncIndex) + tk.MustGetErrCode(`insert into t values (json_array(cast("2022-02-02 11:00:00" as datetime)));`, errno.ErrInvalidJSONValueForFuncIndex) + tk.MustGetErrCode(`insert into t values (json_array(cast('{"a":1}' as json)));`, errno.ErrInvalidJSONValueForFuncIndex) + + tk.MustExec(`drop table if exists t;`) + tk.MustExec(`create table t(a json, index idx((cast(a as binary(1) array))));`) + tk.MustGetErrCode(`insert into t values ('[1,2,3]');`, errno.ErrInvalidJSONValueForFuncIndex) + tk.MustGetErrCode(`insert into t values ('[-1]');`, errno.ErrInvalidJSONValueForFuncIndex) + tk.MustExec(`insert into t values ('["1"]');`) + tk.MustExec(`insert into t values ('["a"]');`) + tk.MustGetErrCode(`insert into t values ('["汉字"]');`, errno.ErrFunctionalIndexDataIsTooLong) + tk.MustGetErrCode(`insert into t values ('[1.2]');`, errno.ErrInvalidJSONValueForFuncIndex) + tk.MustGetErrCode(`insert into t values ('[1.0]');`, errno.ErrInvalidJSONValueForFuncIndex) + tk.MustGetErrCode(`insert into t values (json_array(cast("11:00:00" as time)));`, errno.ErrInvalidJSONValueForFuncIndex) + tk.MustGetErrCode(`insert into t values (json_array(cast("2022-02-02" as date)));`, errno.ErrInvalidJSONValueForFuncIndex) + tk.MustGetErrCode(`insert into t values (json_array(cast("2022-02-02 11:00:00" as datetime)));`, errno.ErrInvalidJSONValueForFuncIndex) + tk.MustGetErrCode(`insert into t values (json_array(cast('{"a":1}' as json)));`, errno.ErrInvalidJSONValueForFuncIndex) + + tk.MustExec(`drop table if exists t;`) + tk.MustExec(`create table t(a json, index idx((cast(a as binary(2) array))));`) + tk.MustGetErrCode(`insert into t values ('[1,2,3]');`, errno.ErrInvalidJSONValueForFuncIndex) + tk.MustGetErrCode(`insert into t values ('[-1]');`, errno.ErrInvalidJSONValueForFuncIndex) + tk.MustExec(`insert into t values ('["1"]');`) + tk.MustExec(`insert into t values ('["a"]');`) + tk.MustGetErrCode(`insert into t values ('["汉字"]');`, errno.ErrFunctionalIndexDataIsTooLong) + tk.MustGetErrCode(`insert into t values ('[1.2]');`, errno.ErrInvalidJSONValueForFuncIndex) + tk.MustGetErrCode(`insert into t values ('[1.0]');`, errno.ErrInvalidJSONValueForFuncIndex) + tk.MustGetErrCode(`insert into t values (json_array(cast("11:00:00" as time)));`, errno.ErrInvalidJSONValueForFuncIndex) + tk.MustGetErrCode(`insert into t values (json_array(cast("2022-02-02" as date)));`, errno.ErrInvalidJSONValueForFuncIndex) + tk.MustGetErrCode(`insert into t values (json_array(cast("2022-02-02 11:00:00" as datetime)));`, errno.ErrInvalidJSONValueForFuncIndex) + tk.MustGetErrCode(`insert into t values (json_array(cast('{"a":1}' as json)));`, errno.ErrInvalidJSONValueForFuncIndex) + + tk.MustExec(`drop table if exists t;`) + tk.MustExec(`create table t(a json, index idx((cast(a as date array))));`) + tk.MustGetErrCode(`insert into t values ('[1,2,3]');`, errno.ErrInvalidJSONValueForFuncIndex) + tk.MustGetErrCode(`insert into t values ('[-1]');`, errno.ErrInvalidJSONValueForFuncIndex) + tk.MustGetErrCode(`insert into t values ('["1"]');`, errno.ErrInvalidJSONValueForFuncIndex) + tk.MustGetErrCode(`insert into t values ('["a"]');`, errno.ErrInvalidJSONValueForFuncIndex) + tk.MustGetErrCode(`insert into t values ('["汉字"]');`, errno.ErrInvalidJSONValueForFuncIndex) + tk.MustGetErrCode(`insert into t values ('[1.2]');`, errno.ErrInvalidJSONValueForFuncIndex) + tk.MustGetErrCode(`insert into t values ('[1.0]');`, errno.ErrInvalidJSONValueForFuncIndex) + tk.MustGetErrCode(`insert into t values (json_array(cast("11:00:00" as time)));`, errno.ErrInvalidJSONValueForFuncIndex) + tk.MustExec(`insert into t values (json_array(cast("2022-02-02" as date)));`) + tk.MustGetErrCode(`insert into t values (json_array(cast("2022-02-02 11:00:00" as datetime)));`, errno.ErrInvalidJSONValueForFuncIndex) + tk.MustGetErrCode(`insert into t values (json_array(cast('{"a":1}' as json)));`, errno.ErrInvalidJSONValueForFuncIndex) + + tk.MustExec(`drop table if exists t;`) + tk.MustExec(`create table t(a json, index idx((cast(a as time array))));`) + tk.MustGetErrCode(`insert into t values ('[1,2,3]');`, errno.ErrInvalidJSONValueForFuncIndex) + tk.MustGetErrCode(`insert into t values ('[-1]');`, errno.ErrInvalidJSONValueForFuncIndex) + tk.MustGetErrCode(`insert into t values ('["1"]');`, errno.ErrInvalidJSONValueForFuncIndex) + tk.MustGetErrCode(`insert into t values ('["a"]');`, errno.ErrInvalidJSONValueForFuncIndex) + tk.MustGetErrCode(`insert into t values ('["汉字"]');`, errno.ErrInvalidJSONValueForFuncIndex) + tk.MustGetErrCode(`insert into t values ('[1.2]');`, errno.ErrInvalidJSONValueForFuncIndex) + tk.MustGetErrCode(`insert into t values ('[1.0]');`, errno.ErrInvalidJSONValueForFuncIndex) + tk.MustExec(`insert into t values (json_array(cast("11:00:00" as time)));`) + tk.MustGetErrCode(`insert into t values (json_array(cast("2022-02-02" as date)));`, errno.ErrInvalidJSONValueForFuncIndex) + tk.MustGetErrCode(`insert into t values (json_array(cast("2022-02-02 11:00:00" as datetime)));`, errno.ErrInvalidJSONValueForFuncIndex) + tk.MustGetErrCode(`insert into t values (json_array(cast('{"a":1}' as json)));`, errno.ErrInvalidJSONValueForFuncIndex) + + tk.MustExec(`drop table if exists t;`) + tk.MustExec(`create table t(a json, index idx((cast(a as datetime array))));`) + tk.MustGetErrCode(`insert into t values ('[1,2,3]');`, errno.ErrInvalidJSONValueForFuncIndex) + tk.MustGetErrCode(`insert into t values ('[-1]');`, errno.ErrInvalidJSONValueForFuncIndex) + tk.MustGetErrCode(`insert into t values ('["1"]');`, errno.ErrInvalidJSONValueForFuncIndex) + tk.MustGetErrCode(`insert into t values ('["a"]');`, errno.ErrInvalidJSONValueForFuncIndex) + tk.MustGetErrCode(`insert into t values ('["汉字"]');`, errno.ErrInvalidJSONValueForFuncIndex) + tk.MustGetErrCode(`insert into t values ('[1.2]');`, errno.ErrInvalidJSONValueForFuncIndex) + tk.MustGetErrCode(`insert into t values ('[1.0]');`, errno.ErrInvalidJSONValueForFuncIndex) + tk.MustGetErrCode(`insert into t values (json_array(cast("11:00:00" as time)));`, errno.ErrInvalidJSONValueForFuncIndex) + tk.MustGetErrCode(`insert into t values (json_array(cast("2022-02-02" as date)));`, errno.ErrInvalidJSONValueForFuncIndex) + tk.MustExec(`insert into t values (json_array(cast("2022-02-02 11:00:00" as datetime)));`) + tk.MustGetErrCode(`insert into t values (json_array(cast('{"a":1}' as json)));`, errno.ErrInvalidJSONValueForFuncIndex) + } +} diff --git a/planner/core/planbuilder.go b/planner/core/planbuilder.go index 238e2a687d758..d2c19c9b181a8 100644 --- a/planner/core/planbuilder.go +++ b/planner/core/planbuilder.go @@ -3610,7 +3610,10 @@ func (b *PlanBuilder) resolveGeneratedColumns(ctx context.Context, columns []*ta } colExpr := mockPlan.Schema().Columns[idx] + originalVal := b.allowBuildCastArray + b.allowBuildCastArray = true expr, _, err := b.rewrite(ctx, column.GeneratedExpr, mockPlan, nil, true) + b.allowBuildCastArray = originalVal if err != nil { return igc, err }