From 28c74a479be42d7f44451defb159fd0e8a09af04 Mon Sep 17 00:00:00 2001 From: crazycs Date: Wed, 28 Feb 2024 13:46:59 +0800 Subject: [PATCH 1/2] This is an automated cherry-pick of #51309 Signed-off-by: ti-chi-bot --- pkg/ddl/db_integration_test.go | 5 +- pkg/errctx/context.go | 250 ++++++++++++++++++++ pkg/executor/insert_common.go | 7 + pkg/executor/test/executor/executor_test.go | 134 +++++++++++ pkg/planner/core/planbuilder.go | 7 +- pkg/table/BUILD.bazel | 5 + pkg/table/column.go | 57 +++-- 7 files changed, 448 insertions(+), 17 deletions(-) create mode 100644 pkg/errctx/context.go diff --git a/pkg/ddl/db_integration_test.go b/pkg/ddl/db_integration_test.go index 5fcedeb4e2384..ddf3629275e5c 100644 --- a/pkg/ddl/db_integration_test.go +++ b/pkg/ddl/db_integration_test.go @@ -1172,7 +1172,10 @@ func TestAlterColumn(t *testing.T) { tk.MustExec("alter table test_alter_column alter column d set default null") tk.MustExec("alter table test_alter_column alter column a drop default") tk.MustGetErrCode("insert into test_alter_column set b = 'd', c = 'dd'", errno.ErrNoDefaultForField) - tk.MustQuery("select a from test_alter_column").Check(testkit.Rows("111", "222", "222", "123")) + tk.MustGetErrCode("insert into test_alter_column set a = DEFAULT, b = 'd', c = 'dd'", errno.ErrNoDefaultForField) + tk.MustGetErrCode("insert into test_alter_column values (DEFAULT, 'd', 'dd', DEFAULT)", errno.ErrNoDefaultForField) + tk.MustExec("insert into test_alter_column set a = NULL, b = 'd', c = 'dd'") + tk.MustQuery("select a from test_alter_column").Check(testkit.Rows("111", "222", "222", "123", "")) // for failing tests sql := "alter table db_not_exist.test_alter_column alter column b set default 'c'" diff --git a/pkg/errctx/context.go b/pkg/errctx/context.go new file mode 100644 index 0000000000000..0a1f48161b484 --- /dev/null +++ b/pkg/errctx/context.go @@ -0,0 +1,250 @@ +// Copyright 2023 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package errctx + +import ( + "github.com/pingcap/errors" + "github.com/pingcap/tidb/pkg/errno" + contextutil "github.com/pingcap/tidb/pkg/util/context" + "github.com/pingcap/tidb/pkg/util/intest" +) + +// Level defines the behavior for each error +type Level uint8 + +const ( + // LevelError means the error will be returned + LevelError Level = iota + // LevelWarn means it will be regarded as a warning + LevelWarn + // LevelIgnore means the error will be ignored + LevelIgnore +) + +// LevelMap indicates the map from `ErrGroup` to `Level` +type LevelMap [errGroupCount]Level + +// Context defines how to handle an error +type Context struct { + levelMap LevelMap + warnHandler contextutil.WarnHandler +} + +// LevelMap returns the `levelMap` of the context. +func (ctx *Context) LevelMap() LevelMap { + return ctx.levelMap +} + +// LevelForGroup returns the level for a specified group. +func (ctx *Context) LevelForGroup(errGroup ErrGroup) Level { + return ctx.levelMap[errGroup] +} + +// WithStrictErrGroupLevel makes the context to return the error directly for any kinds of errors. +func (ctx *Context) WithStrictErrGroupLevel() Context { + newCtx := Context{ + warnHandler: ctx.warnHandler, + } + + return newCtx +} + +// WithErrGroupLevel sets a `Level` for an `ErrGroup` +func (ctx *Context) WithErrGroupLevel(eg ErrGroup, l Level) Context { + newCtx := Context{ + levelMap: ctx.levelMap, + warnHandler: ctx.warnHandler, + } + newCtx.levelMap[eg] = l + + return newCtx +} + +// WithErrGroupLevels sets `levelMap` for an `ErrGroup` +func (ctx *Context) WithErrGroupLevels(levels LevelMap) Context { + return Context{ + levelMap: levels, + warnHandler: ctx.warnHandler, + } +} + +// AppendWarning appends the error to warning. If the inner `warnHandler` is nil, do nothing. +func (ctx *Context) AppendWarning(err error) { + intest.Assert(ctx.warnHandler != nil) + if w := ctx.warnHandler; w != nil { + // warnHandler should always not be nil, check fn != nil here to just make code safe. + w.AppendWarning(err) + } +} + +// HandleError handles the error according to the contextutil. See the comment of `HandleErrorWithAlias` for detailed logic. +// +// It also allows using `errors.ErrorGroup`, in this case, it'll handle each error in order, and return the first error +// it founds. +func (ctx *Context) HandleError(err error) error { + // The function of handling `errors.ErrorGroup` is placed in `HandleError` but not in `HandleErrorWithAlias`, because + // it's hard to give a proper error and warn alias for an error group. + if errs, ok := err.(errors.ErrorGroup); ok { + for _, singleErr := range errs.Errors() { + singleErr = ctx.HandleError(singleErr) + // If the one error is found, just return it. + // TODO: consider whether it's more appropriate to continue to handle other errors. For example, other errors + // may need to append warnings. The current behavior is same with TiDB original behavior before using + // `errctx` to handle multiple errors. + if singleErr != nil { + return singleErr + } + } + + return nil + } + + return ctx.HandleErrorWithAlias(err, err, err) +} + +// HandleErrorWithAlias handles the error according to the contextutil. +// 1. If the `internalErr` is not `"pingcap/errors".Error`, or the error code is not defined in the `errGroupMap`, or the error +// level is set to `LevelError`(0), the `err` will be returned directly. +// 2. If the error level is set to `LevelWarn`, the `warnErr` will be appended as a warning. +// 3. If the error level is set to `LevelIgnore`, this function will return a `nil`. +// +// In most cases, these three should be the same. If there are many different kinds of error internally, but they are expected +// to give the same error to users, the `err` can be different form `internalErr`. Also, if the warning is expected to be +// different from the initial error, you can also use the `warnErr` argument. +// +// TODO: is it good to give an error code for internal only errors? Or should we use another way to distinguish different +// group of errors? +// TODO: both `types.Context` and `errctx.Context` can handle truncate error now. Refractor them. +func (ctx *Context) HandleErrorWithAlias(internalErr error, err error, warnErr error) error { + if internalErr == nil { + return nil + } + + internalErr = errors.Cause(internalErr) + + e, ok := internalErr.(*errors.Error) + if !ok { + return err + } + + eg, ok := errGroupMap[e.Code()] + if !ok { + return err + } + + switch ctx.levelMap[eg] { + case LevelError: + return err + case LevelWarn: + ctx.AppendWarning(warnErr) + case LevelIgnore: + } + + return nil +} + +// NewContext creates an error context to handle the errors and warnings +func NewContext(handler contextutil.WarnHandler) Context { + return NewContextWithLevels(LevelMap{}, handler) +} + +// NewContextWithLevels creates an error context to handle the errors and warnings +func NewContextWithLevels(levels LevelMap, handler contextutil.WarnHandler) Context { + intest.Assert(handler != nil) + return Context{ + warnHandler: handler, + levelMap: levels, + } +} + +// StrictNoWarningContext returns all errors directly, and ignore all errors +var StrictNoWarningContext = NewContext(contextutil.IgnoreWarn) + +var errGroupMap = make(map[errors.ErrCode]ErrGroup) + +// ErrGroup groups the error according to the behavior of handling errors +type ErrGroup int + +const ( + // ErrGroupTruncate is the group of truncated errors + ErrGroupTruncate ErrGroup = iota + // ErrGroupDupKey is the group of duplicate key errors + ErrGroupDupKey + // ErrGroupBadNull is the group of bad null errors + ErrGroupBadNull + // ErrGroupDividedByZero is the group of divided by zero errors + ErrGroupDividedByZero + // ErrGroupAutoIncReadFailed is the group of auto increment read failed errors + ErrGroupAutoIncReadFailed + // ErrGroupNoMatchedPartition is the group of no partition is matched errors. + ErrGroupNoMatchedPartition + // errGroupCount is the count of all `ErrGroup`. Please leave it at the end of the list. + errGroupCount +) + +func init() { + group2Errors := map[ErrGroup][]errors.ErrCode{ + ErrGroupTruncate: { + errno.ErrTruncatedWrongValue, + errno.ErrDataTooLong, + errno.ErrTruncatedWrongValueForField, + errno.ErrWarnDataOutOfRange, + errno.ErrDataOutOfRange, + errno.ErrBadNumber, + errno.ErrWrongValueForType, + errno.ErrDatetimeFunctionOverflow, + errno.WarnDataTruncated, + errno.ErrIncorrectDatetimeValue, + }, + ErrGroupBadNull: { + errno.ErrBadNull, + errno.ErrWarnNullToNotnull, + errno.ErrNoDefaultForField, + }, + ErrGroupDividedByZero: { + errno.ErrDivisionByZero, + }, + ErrGroupAutoIncReadFailed: { + errno.ErrAutoincReadFailed, + }, + ErrGroupNoMatchedPartition: { + errno.ErrNoPartitionForGivenValue, + errno.ErrRowDoesNotMatchGivenPartitionSet, + }, + ErrGroupDupKey: { + errno.ErrDupEntry, + }, + } + + for group, codes := range group2Errors { + for _, errCode := range codes { + errGroupMap[errCode] = group + } + } +} + +// ResolveErrLevel resolves the error level according to the `ignore` and `warn` flags +// if ignore is true, it will return `LevelIgnore` to ignore the error, +// otherwise, it will return `LevelWarn` or `LevelError` according to the `warn` flag +// Only one of `ignore` and `warn` can be true. +func ResolveErrLevel(ignore bool, warn bool) Level { + if ignore { + return LevelIgnore + } + if warn { + return LevelWarn + } + return LevelError +} diff --git a/pkg/executor/insert_common.go b/pkg/executor/insert_common.go index e8d48aaf45709..d0af675786a31 100644 --- a/pkg/executor/insert_common.go +++ b/pkg/executor/insert_common.go @@ -580,7 +580,14 @@ func (e *InsertValues) getColDefaultValue(idx int, col *table.Column) (d types.D if col.DefaultIsExpr && col.DefaultExpr != nil { defaultVal, err = table.EvalColDefaultExpr(e.Ctx(), col.ToInfo(), col.DefaultExpr) } else { +<<<<<<< HEAD defaultVal, err = table.GetColDefaultValue(e.Ctx(), col.ToInfo()) +======= + if err := table.CheckNoDefaultValueForInsert(e.Ctx().GetSessionVars().StmtCtx, col.ToInfo()); err != nil { + return types.Datum{}, err + } + defaultVal, err = table.GetColDefaultValue(e.Ctx().GetExprCtx(), col.ToInfo()) +>>>>>>> e586960027b (table: fix issue of get default value from column when column doesn't have default value (#51309)) } if err != nil { return types.Datum{}, err diff --git a/pkg/executor/test/executor/executor_test.go b/pkg/executor/test/executor/executor_test.go index d16d02a5148cb..67c6180a12445 100644 --- a/pkg/executor/test/executor/executor_test.go +++ b/pkg/executor/test/executor/executor_test.go @@ -4352,3 +4352,137 @@ func TestIssues49377(t *testing.T) { "limit 1" + ");").Sort().Check(testkit.Rows("1 Furina 1", "1 Furina 1", "1 Furina 1", "2 Klee 1", "2 Klee 1", "3 Eula 1", "3 Eula 1")) } +<<<<<<< HEAD +======= + +func TestIssues40463(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + + tk.MustExec("use test;") + tk.MustExec("CREATE TABLE `4f380f26-9af6-4df8-959d-ad6296eff914` (`f7a9a4be-3728-449b-a5ea-df9b957eec67` enum('bkdv0','9rqy','lw','neud','ym','4nbv','9a7','bpkfo','xtfl','59','6vjj') NOT NULL DEFAULT 'neud', `43ca0135-1650-429b-8887-9eabcae2a234` set('8','5x47','xc','o31','lnz','gs5s','6yam','1','20ea','i','e') NOT NULL DEFAULT 'e', PRIMARY KEY (`f7a9a4be-3728-449b-a5ea-df9b957eec67`,`43ca0135-1650-429b-8887-9eabcae2a234`) /*T![clustered_index] CLUSTERED */) ENGINE=InnoDB DEFAULT CHARSET=ascii COLLATE=ascii_bin;") + tk.MustExec("INSERT INTO `4f380f26-9af6-4df8-959d-ad6296eff914` VALUES ('bkdv0','gs5s'),('lw','20ea'),('neud','8'),('ym','o31'),('4nbv','o31'),('xtfl','e');") + + tk.MustExec("CREATE TABLE `ba35a09f-76f4-40aa-9b48-13154a24bdd2` (`9b2a7138-14a3-4e8f-b29a-720392aad22c` set('zgn','if8yo','e','k7','bav','xj6','lkag','m5','as','ia','l3') DEFAULT 'zgn,if8yo,e,k7,ia,l3',`a60d6b5c-08bd-4a6d-b951-716162d004a5` set('6li6','05jlu','w','l','m','e9r','5q','d0ol','i6ajr','csf','d32') DEFAULT '6li6,05jlu,w,l,m,d0ol,i6ajr,csf,d32',`fb753d37-6252-4bd3-9bd1-0059640e7861` year(4) DEFAULT '2065', UNIQUE KEY `51816c39-27df-4bbe-a0e7-d6b6f54be2a2` (`fb753d37-6252-4bd3-9bd1-0059640e7861`), KEY `b0dfda0a-ffed-4c5b-9a72-4113bc1cbc8e` (`9b2a7138-14a3-4e8f-b29a-720392aad22c`,`fb753d37-6252-4bd3-9bd1-0059640e7861`)) ENGINE=InnoDB DEFAULT CHARSET=utf8 COLLATE=utf8_bin /*T! SHARD_ROW_ID_BITS=5 */;") + tk.MustExec("insert into `ba35a09f-76f4-40aa-9b48-13154a24bdd2` values ('if8yo', '6li6,05jlu,w,l,m,d0ol,i6ajr,csf,d32', 2065);") + + tk.MustExec("CREATE TABLE `07ccc74e-14c3-4685-bb41-c78a069b1a6d` (`8a93bdc5-2214-4f96-b5a7-1ba4c0d396ae` bigint(20) NOT NULL DEFAULT '-4604789462044748682',`30b19ecf-679f-4ca3-813f-d3c3b8f7da7e` date NOT NULL DEFAULT '5030-11-23',`1c52eaf2-1ebb-4486-9410-dfd00c7c835c` decimal(7,5) DEFAULT '-81.91307',`4b09dfdc-e688-41cb-9ffa-d03071a43077` float DEFAULT '1.7989023',PRIMARY KEY (`30b19ecf-679f-4ca3-813f-d3c3b8f7da7e`,`8a93bdc5-2214-4f96-b5a7-1ba4c0d396ae`) /*T![clustered_index] CLUSTERED */,KEY `ae7a7637-ca52-443b-8a3f-69694f730cc4` (`8a93bdc5-2214-4f96-b5a7-1ba4c0d396ae`),KEY `42640042-8a17-4145-9510-5bb419f83ed9` (`8a93bdc5-2214-4f96-b5a7-1ba4c0d396ae`),KEY `839f4f5a-83f3-449b-a7dd-c7d2974d351a` (`30b19ecf-679f-4ca3-813f-d3c3b8f7da7e`),KEY `c474cde1-6fe4-45df-9067-b4e479f84149` (`8a93bdc5-2214-4f96-b5a7-1ba4c0d396ae`),KEY `f834d0a9-709e-4ca8-925d-73f48322b70d` (`8a93bdc5-2214-4f96-b5a7-1ba4c0d396ae`)) ENGINE=InnoDB DEFAULT CHARSET=gbk COLLATE=gbk_chinese_ci;") + tk.MustExec("set sql_mode=``;") + tk.MustExec("INSERT INTO `07ccc74e-14c3-4685-bb41-c78a069b1a6d` VALUES (616295989348159438,'0000-00-00',1.00000,1.7989023),(2215857492573998768,'1970-02-02',0.00000,1.7989023),(2215857492573998768,'1983-05-13',0.00000,1.7989023),(-2840083604831267906,'1984-01-30',1.00000,1.7989023),(599388718360890339,'1986-09-09',1.00000,1.7989023),(3506764933630033073,'1987-11-22',1.00000,1.7989023),(3506764933630033073,'2002-02-26',1.00000,1.7989023),(3506764933630033073,'2003-05-14',1.00000,1.7989023),(3506764933630033073,'2007-05-16',1.00000,1.7989023),(3506764933630033073,'2017-02-20',1.00000,1.7989023),(3506764933630033073,'2017-08-06',1.00000,1.7989023),(2215857492573998768,'2019-02-18',1.00000,1.7989023),(3506764933630033073,'2020-08-11',1.00000,1.7989023),(3506764933630033073,'2028-06-07',1.00000,1.7989023),(3506764933630033073,'2036-08-16',1.00000,1.7989023);") + + tk.MustQuery("select /*+ use_index_merge( `4f380f26-9af6-4df8-959d-ad6296eff914` ) */ /*+ stream_agg() */ approx_percentile( `4f380f26-9af6-4df8-959d-ad6296eff914`.`f7a9a4be-3728-449b-a5ea-df9b957eec67` , 77 ) as r0 , `4f380f26-9af6-4df8-959d-ad6296eff914`.`f7a9a4be-3728-449b-a5ea-df9b957eec67` as r1 from `4f380f26-9af6-4df8-959d-ad6296eff914` where not( IsNull( `4f380f26-9af6-4df8-959d-ad6296eff914`.`f7a9a4be-3728-449b-a5ea-df9b957eec67` ) ) and not( `4f380f26-9af6-4df8-959d-ad6296eff914`.`f7a9a4be-3728-449b-a5ea-df9b957eec67` in ( select `8a93bdc5-2214-4f96-b5a7-1ba4c0d396ae` from `07ccc74e-14c3-4685-bb41-c78a069b1a6d` where `4f380f26-9af6-4df8-959d-ad6296eff914`.`f7a9a4be-3728-449b-a5ea-df9b957eec67` in ( select `a60d6b5c-08bd-4a6d-b951-716162d004a5` from `ba35a09f-76f4-40aa-9b48-13154a24bdd2` where not( `4f380f26-9af6-4df8-959d-ad6296eff914`.`f7a9a4be-3728-449b-a5ea-df9b957eec67` between 'bpkfo' and '59' ) and not( `4f380f26-9af6-4df8-959d-ad6296eff914`.`f7a9a4be-3728-449b-a5ea-df9b957eec67` in ( select `fb753d37-6252-4bd3-9bd1-0059640e7861` from `ba35a09f-76f4-40aa-9b48-13154a24bdd2` where IsNull( `4f380f26-9af6-4df8-959d-ad6296eff914`.`f7a9a4be-3728-449b-a5ea-df9b957eec67` ) or not( `4f380f26-9af6-4df8-959d-ad6296eff914`.`43ca0135-1650-429b-8887-9eabcae2a234` in ( select `8a93bdc5-2214-4f96-b5a7-1ba4c0d396ae` from `07ccc74e-14c3-4685-bb41-c78a069b1a6d` where IsNull( `4f380f26-9af6-4df8-959d-ad6296eff914`.`43ca0135-1650-429b-8887-9eabcae2a234` ) and not( `4f380f26-9af6-4df8-959d-ad6296eff914`.`f7a9a4be-3728-449b-a5ea-df9b957eec67` between 'neud' and 'bpkfo' ) ) ) ) ) ) ) ) group by `4f380f26-9af6-4df8-959d-ad6296eff914`.`f7a9a4be-3728-449b-a5ea-df9b957eec67`;") +} + +func TestIssue38756(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + + tk.MustExec("use test") + tk.MustExec("create table t (c1 int)") + tk.MustExec("insert into t values (1), (2), (3)") + tk.MustQuery("SELECT SQRT(1) FROM t").Check(testkit.Rows("1", "1", "1")) + tk.MustQuery("(SELECT DISTINCT SQRT(1) FROM t)").Check(testkit.Rows("1")) + tk.MustQuery("SELECT DISTINCT cast(1 as double) FROM t").Check(testkit.Rows("1")) +} + +func TestIssue50043(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + // Test simplified case by update. + tk.MustExec("use test") + tk.MustExec("create table t (c1 boolean ,c2 decimal ( 37 , 17 ), unique key idx1 (c1 ,c2),unique key idx2 ( c1 ));") + tk.MustExec("insert into t values (0,NULL);") + tk.MustExec("alter table t alter column c2 drop default;") + tk.MustExec("update t set c2 = 5 where c1 = 0;") + tk.MustQuery("select * from t order by c1,c2").Check(testkit.Rows("0 5.00000000000000000")) + + // Test simplified case by insert on duplicate key update. + tk.MustExec("drop table if exists t") + tk.MustExec("create table t (c1 boolean ,c2 decimal ( 37 , 17 ), unique key idx1 (c1 ,c2));") + tk.MustExec("alter table t alter column c2 drop default;") + tk.MustExec("alter table t add unique key idx4 ( c1 );") + tk.MustExec("insert into t values (0, NULL), (1, 1);") + tk.MustExec("insert into t values (0, 2) ,(1, 3) on duplicate key update c2 = 5;") + tk.MustQuery("show warnings").Check(testkit.Rows()) + tk.MustQuery("select * from t order by c1,c2").Check(testkit.Rows("0 5.00000000000000000", "1 5.00000000000000000")) + + // Test Issue 50043. + tk.MustExec("drop table if exists t") + tk.MustExec("create table t (c1 boolean ,c2 decimal ( 37 , 17 ), unique key idx1 (c1 ,c2));") + tk.MustExec("alter table t alter column c2 drop default;") + tk.MustExec("alter table t add unique key idx4 ( c1 );") + tk.MustExec("insert into t values (0, NULL), (1, 1);") + tk.MustExec("insert ignore into t values (0, 2) ,(1, 3) on duplicate key update c2 = 5, c1 = 0") + tk.MustQuery("select * from t order by c1,c2").Check(testkit.Rows("0 5.00000000000000000", "1 1.00000000000000000")) +} + +func TestIssue51324(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("create table t (id int key, a int, b enum('a', 'b'))") + tk.MustGetErrMsg("insert into t values ()", "[table:1364]Field 'id' doesn't have a default value") + tk.MustExec("insert into t set id = 1") + tk.MustExec("insert into t set id = 2, a = NULL, b = NULL") + tk.MustExec("insert into t set id = 3, a = DEFAULT, b = DEFAULT") + tk.MustQuery("select * from t order by id").Check(testkit.Rows("1 ", "2 ", "3 ")) + + tk.MustExec("alter table t alter column a drop default") + tk.MustExec("alter table t alter column b drop default") + tk.MustGetErrMsg("insert into t set id = 4;", "[table:1364]Field 'a' doesn't have a default value") + tk.MustExec("insert into t set id = 5, a = NULL, b = NULL;") + tk.MustGetErrMsg("insert into t set id = 6, a = DEFAULT, b = DEFAULT;", "[table:1364]Field 'a' doesn't have a default value") + tk.MustQuery("select * from t order by id").Check(testkit.Rows("1 ", "2 ", "3 ", "5 ")) + + tk.MustExec("insert ignore into t set id = 4;") + tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1364 Field 'a' doesn't have a default value")) + tk.MustExec("insert ignore into t set id = 6, a = DEFAULT, b = DEFAULT;") + tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1364 Field 'a' doesn't have a default value")) + tk.MustQuery("select * from t order by id").Check(testkit.Rows("1 ", "2 ", "3 ", "4 ", "5 ", "6 ")) + tk.MustExec("update t set id = id + 10") + tk.MustQuery("show warnings").Check(testkit.Rows()) + tk.MustQuery("select * from t order by id").Check(testkit.Rows("11 ", "12 ", "13 ", "14 ", "15 ", "16 ")) + + // Test not null case. + tk.MustExec("drop table t") + tk.MustExec("create table t (id int key, a int not null, b enum('a', 'b') not null)") + tk.MustGetErrMsg("insert into t values ()", "[table:1364]Field 'id' doesn't have a default value") + tk.MustGetErrMsg("insert into t set id = 1", "[table:1364]Field 'a' doesn't have a default value") + tk.MustGetErrMsg("insert into t set id = 2, a = NULL, b = NULL", "[table:1048]Column 'a' cannot be null") + tk.MustGetErrMsg("insert into t set id = 2, a = 2, b = NULL", "[table:1048]Column 'b' cannot be null") + tk.MustGetErrMsg("insert into t set id = 3, a = DEFAULT, b = DEFAULT", "[table:1364]Field 'a' doesn't have a default value") + tk.MustExec("alter table t alter column a drop default") + tk.MustExec("alter table t alter column b drop default") + tk.MustExec("insert ignore into t set id = 4;") + tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1364 Field 'a' doesn't have a default value")) + tk.MustExec("insert ignore into t set id = 5, a = NULL, b = NULL;") + tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1048 Column 'a' cannot be null", "Warning 1048 Column 'b' cannot be null")) + tk.MustExec("insert ignore into t set id = 6, a = 6, b = NULL;") + tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1048 Column 'b' cannot be null")) + tk.MustExec("insert ignore into t set id = 7, a = DEFAULT, b = DEFAULT;") + tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1364 Field 'a' doesn't have a default value")) + tk.MustQuery("select * from t order by id").Check(testkit.Rows("4 0 a", "5 0 ", "6 6 ", "7 0 a")) + + // Test add column with OriginDefaultValue case. + tk.MustExec("drop table t") + tk.MustExec("create table t (id int, unique key idx (id))") + tk.MustExec("insert into t values (1)") + tk.MustExec("alter table t add column a int default 1") + tk.MustExec("alter table t add column b int default null") + tk.MustExec("alter table t add column c int not null") + tk.MustExec("alter table t add column d int not null default 1") + tk.MustExec("insert ignore into t (id) values (2)") + tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1364 Field 'c' doesn't have a default value")) + tk.MustExec("insert ignore into t (id) values (1),(2) on duplicate key update id = id+10") + tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1364 Field 'c' doesn't have a default value")) + tk.MustExec("alter table t alter column a drop default") + tk.MustExec("alter table t alter column b drop default") + tk.MustExec("alter table t alter column c drop default") + tk.MustExec("alter table t alter column d drop default") + tk.MustExec("insert ignore into t (id) values (3)") + tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1364 Field 'a' doesn't have a default value", "Warning 1364 Field 'b' doesn't have a default value", "Warning 1364 Field 'c' doesn't have a default value", "Warning 1364 Field 'd' doesn't have a default value")) + tk.MustExec("insert ignore into t (id) values (11),(12),(3) on duplicate key update id = id+10") + tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1364 Field 'a' doesn't have a default value", "Warning 1364 Field 'b' doesn't have a default value", "Warning 1364 Field 'c' doesn't have a default value", "Warning 1364 Field 'd' doesn't have a default value")) + tk.MustQuery("select * from t order by id").Check(testkit.Rows("13 0 0", "21 1 0 1", "22 1 0 1")) +} +>>>>>>> e586960027b (table: fix issue of get default value from column when column doesn't have default value (#51309)) diff --git a/pkg/planner/core/planbuilder.go b/pkg/planner/core/planbuilder.go index 8d3e6926965c4..5fc63f50ad1a8 100644 --- a/pkg/planner/core/planbuilder.go +++ b/pkg/planner/core/planbuilder.go @@ -3855,7 +3855,7 @@ func genAuthErrForGrantStmt(sctx sessionctx.Context, dbName string) error { return ErrDBaccessDenied.FastGenByArgs(u, h, dbName) } -func (b *PlanBuilder) getDefaultValue(col *table.Column) (*expression.Constant, error) { +func (b *PlanBuilder) getDefaultValueForInsert(col *table.Column) (*expression.Constant, error) { var ( value types.Datum err error @@ -3863,6 +3863,9 @@ func (b *PlanBuilder) getDefaultValue(col *table.Column) (*expression.Constant, if col.DefaultIsExpr && col.DefaultExpr != nil { value, err = table.EvalColDefaultExpr(b.ctx, col.ToInfo(), col.DefaultExpr) } else { + if err := table.CheckNoDefaultValueForInsert(b.ctx.GetSessionVars().StmtCtx, col.ToInfo()); err != nil { + return nil, err + } value, err = table.GetColDefaultValue(b.ctx, col.ToInfo()) } if err != nil { @@ -4151,7 +4154,7 @@ func (b PlanBuilder) getInsertColExpr(ctx context.Context, insertPlan *Insert, m // See note in the end of the function. Only default for generated columns are OK. return nil, nil } - outExpr, err = b.getDefaultValue(refCol) + outExpr, err = b.getDefaultValueForInsert(refCol) case *driver.ValueExpr: outExpr = &expression.Constant{ Value: x.Datum, diff --git a/pkg/table/BUILD.bazel b/pkg/table/BUILD.bazel index 8f5df5622c166..0f0273add7154 100644 --- a/pkg/table/BUILD.bazel +++ b/pkg/table/BUILD.bazel @@ -23,6 +23,11 @@ go_library( "//pkg/parser/types", "//pkg/sessionctx", "//pkg/sessionctx/stmtctx", +<<<<<<< HEAD +======= + "//pkg/sessionctx/variable", + "//pkg/table/context", +>>>>>>> e586960027b (table: fix issue of get default value from column when column doesn't have default value (#51309)) "//pkg/types", "//pkg/util/chunk", "//pkg/util/dbterror", diff --git a/pkg/table/column.go b/pkg/table/column.go index b7a3c95de27f9..7ec609d05657a 100644 --- a/pkg/table/column.go +++ b/pkg/table/column.go @@ -34,6 +34,10 @@ import ( field_types "github.com/pingcap/tidb/pkg/parser/types" "github.com/pingcap/tidb/pkg/sessionctx" "github.com/pingcap/tidb/pkg/sessionctx/stmtctx" +<<<<<<< HEAD +======= + "github.com/pingcap/tidb/pkg/sessionctx/variable" +>>>>>>> e586960027b (table: fix issue of get default value from column when column doesn't have default value (#51309)) "github.com/pingcap/tidb/pkg/types" "github.com/pingcap/tidb/pkg/util/chunk" "github.com/pingcap/tidb/pkg/util/hack" @@ -529,6 +533,23 @@ func GetColOriginDefaultValueWithoutStrictSQLMode(ctx sessionctx.Context, col *m }) } +// CheckNoDefaultValueForInsert checks if the column has no default value before insert data. +// CheckNoDefaultValueForInsert extracts the check logic from getColDefaultValueFromNil, +// since getColDefaultValueFromNil function is public path and both read/write and other places use it. +// But CheckNoDefaultValueForInsert logic should only check before insert. +func CheckNoDefaultValueForInsert(sc *stmtctx.StatementContext, col *model.ColumnInfo) error { + if mysql.HasNoDefaultValueFlag(col.GetFlag()) && !col.DefaultIsExpr && col.GetDefaultValue() == nil && col.GetType() != mysql.TypeEnum { + ignoreErr := sc.ErrGroupLevel(errctx.ErrGroupBadNull) != errctx.LevelError + if !ignoreErr { + return ErrNoDefaultValue.GenWithStackByArgs(col.Name) + } + if !mysql.HasNotNullFlag(col.GetFlag()) { + sc.AppendWarning(ErrNoDefaultValue.FastGenByArgs(col.Name)) + } + } + return nil +} + // GetColDefaultValue gets default value of the column. func GetColDefaultValue(ctx sessionctx.Context, col *model.ColumnInfo) (types.Datum, error) { defaultValue := col.GetDefaultValue() @@ -616,23 +637,25 @@ func getColDefaultValue(ctx sessionctx.Context, col *model.ColumnInfo, defaultVa return value, nil } +<<<<<<< HEAD func getColDefaultValueFromNil(ctx sessionctx.Context, col *model.ColumnInfo, args *getColOriginDefaultValue) (types.Datum, error) { if !mysql.HasNotNullFlag(col.GetFlag()) && !mysql.HasNoDefaultValueFlag(col.GetFlag()) { +======= +func getColDefaultValueFromNil(ctx expression.BuildContext, col *model.ColumnInfo, args *getColOriginDefaultValue) (types.Datum, error) { + if !mysql.HasNotNullFlag(col.GetFlag()) { +>>>>>>> e586960027b (table: fix issue of get default value from column when column doesn't have default value (#51309)) return types.Datum{}, nil } if col.GetType() == mysql.TypeEnum { // For enum type, if no default value and not null is set, // the default value is the first element of the enum list - if mysql.HasNotNullFlag(col.GetFlag()) { - defEnum, err := types.ParseEnumValue(col.FieldType.GetElems(), 1) - if err != nil { - return types.Datum{}, err - } - return types.NewCollateMysqlEnumDatum(defEnum, col.GetCollate()), nil + defEnum, err := types.ParseEnumValue(col.FieldType.GetElems(), 1) + if err != nil { + return types.Datum{}, err } - return types.Datum{}, nil + return types.NewCollateMysqlEnumDatum(defEnum, col.GetCollate()), nil } - if mysql.HasAutoIncrementFlag(col.GetFlag()) && !mysql.HasNoDefaultValueFlag(col.GetFlag()) { + if mysql.HasAutoIncrementFlag(col.GetFlag()) { // Auto increment column doesn't have default value and we should not return error. return GetZeroValue(col), nil } @@ -646,15 +669,21 @@ func getColDefaultValueFromNil(ctx sessionctx.Context, col *model.ColumnInfo, ar } if !strictSQLMode { sc.AppendWarning(ErrNoDefaultValue.FastGenByArgs(col.Name)) - if mysql.HasNotNullFlag(col.GetFlag()) { - return GetZeroValue(col), nil - } - if mysql.HasNoDefaultValueFlag(col.GetFlag()) { - return types.Datum{}, nil - } + return GetZeroValue(col), nil } +<<<<<<< HEAD if sc.BadNullAsWarning { sc.AppendWarning(ErrColumnCantNull.FastGenByArgs(col.Name)) +======= + ec := sc.ErrCtx() + var err error + if mysql.HasNoDefaultValueFlag(col.GetFlag()) { + err = ErrNoDefaultValue.FastGenByArgs(col.Name) + } else { + err = ErrColumnCantNull.FastGenByArgs(col.Name) + } + if ec.HandleError(err) == nil { +>>>>>>> e586960027b (table: fix issue of get default value from column when column doesn't have default value (#51309)) return GetZeroValue(col), nil } return types.Datum{}, ErrNoDefaultValue.GenWithStackByArgs(col.Name) From f5b0c14777a0b79cbb852126650e0b7b47267e74 Mon Sep 17 00:00:00 2001 From: crazycs520 Date: Wed, 28 Feb 2024 15:02:51 +0800 Subject: [PATCH 2/2] resolve conflict Signed-off-by: crazycs520 --- pkg/errctx/context.go | 250 -------------------- pkg/executor/insert_common.go | 6 +- pkg/executor/test/executor/executor_test.go | 33 --- pkg/planner/core/expression_rewriter.go | 2 +- pkg/planner/core/planbuilder.go | 10 +- pkg/table/BUILD.bazel | 5 - pkg/table/column.go | 31 +-- 7 files changed, 16 insertions(+), 321 deletions(-) delete mode 100644 pkg/errctx/context.go diff --git a/pkg/errctx/context.go b/pkg/errctx/context.go deleted file mode 100644 index 0a1f48161b484..0000000000000 --- a/pkg/errctx/context.go +++ /dev/null @@ -1,250 +0,0 @@ -// Copyright 2023 PingCAP, Inc. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -package errctx - -import ( - "github.com/pingcap/errors" - "github.com/pingcap/tidb/pkg/errno" - contextutil "github.com/pingcap/tidb/pkg/util/context" - "github.com/pingcap/tidb/pkg/util/intest" -) - -// Level defines the behavior for each error -type Level uint8 - -const ( - // LevelError means the error will be returned - LevelError Level = iota - // LevelWarn means it will be regarded as a warning - LevelWarn - // LevelIgnore means the error will be ignored - LevelIgnore -) - -// LevelMap indicates the map from `ErrGroup` to `Level` -type LevelMap [errGroupCount]Level - -// Context defines how to handle an error -type Context struct { - levelMap LevelMap - warnHandler contextutil.WarnHandler -} - -// LevelMap returns the `levelMap` of the context. -func (ctx *Context) LevelMap() LevelMap { - return ctx.levelMap -} - -// LevelForGroup returns the level for a specified group. -func (ctx *Context) LevelForGroup(errGroup ErrGroup) Level { - return ctx.levelMap[errGroup] -} - -// WithStrictErrGroupLevel makes the context to return the error directly for any kinds of errors. -func (ctx *Context) WithStrictErrGroupLevel() Context { - newCtx := Context{ - warnHandler: ctx.warnHandler, - } - - return newCtx -} - -// WithErrGroupLevel sets a `Level` for an `ErrGroup` -func (ctx *Context) WithErrGroupLevel(eg ErrGroup, l Level) Context { - newCtx := Context{ - levelMap: ctx.levelMap, - warnHandler: ctx.warnHandler, - } - newCtx.levelMap[eg] = l - - return newCtx -} - -// WithErrGroupLevels sets `levelMap` for an `ErrGroup` -func (ctx *Context) WithErrGroupLevels(levels LevelMap) Context { - return Context{ - levelMap: levels, - warnHandler: ctx.warnHandler, - } -} - -// AppendWarning appends the error to warning. If the inner `warnHandler` is nil, do nothing. -func (ctx *Context) AppendWarning(err error) { - intest.Assert(ctx.warnHandler != nil) - if w := ctx.warnHandler; w != nil { - // warnHandler should always not be nil, check fn != nil here to just make code safe. - w.AppendWarning(err) - } -} - -// HandleError handles the error according to the contextutil. See the comment of `HandleErrorWithAlias` for detailed logic. -// -// It also allows using `errors.ErrorGroup`, in this case, it'll handle each error in order, and return the first error -// it founds. -func (ctx *Context) HandleError(err error) error { - // The function of handling `errors.ErrorGroup` is placed in `HandleError` but not in `HandleErrorWithAlias`, because - // it's hard to give a proper error and warn alias for an error group. - if errs, ok := err.(errors.ErrorGroup); ok { - for _, singleErr := range errs.Errors() { - singleErr = ctx.HandleError(singleErr) - // If the one error is found, just return it. - // TODO: consider whether it's more appropriate to continue to handle other errors. For example, other errors - // may need to append warnings. The current behavior is same with TiDB original behavior before using - // `errctx` to handle multiple errors. - if singleErr != nil { - return singleErr - } - } - - return nil - } - - return ctx.HandleErrorWithAlias(err, err, err) -} - -// HandleErrorWithAlias handles the error according to the contextutil. -// 1. If the `internalErr` is not `"pingcap/errors".Error`, or the error code is not defined in the `errGroupMap`, or the error -// level is set to `LevelError`(0), the `err` will be returned directly. -// 2. If the error level is set to `LevelWarn`, the `warnErr` will be appended as a warning. -// 3. If the error level is set to `LevelIgnore`, this function will return a `nil`. -// -// In most cases, these three should be the same. If there are many different kinds of error internally, but they are expected -// to give the same error to users, the `err` can be different form `internalErr`. Also, if the warning is expected to be -// different from the initial error, you can also use the `warnErr` argument. -// -// TODO: is it good to give an error code for internal only errors? Or should we use another way to distinguish different -// group of errors? -// TODO: both `types.Context` and `errctx.Context` can handle truncate error now. Refractor them. -func (ctx *Context) HandleErrorWithAlias(internalErr error, err error, warnErr error) error { - if internalErr == nil { - return nil - } - - internalErr = errors.Cause(internalErr) - - e, ok := internalErr.(*errors.Error) - if !ok { - return err - } - - eg, ok := errGroupMap[e.Code()] - if !ok { - return err - } - - switch ctx.levelMap[eg] { - case LevelError: - return err - case LevelWarn: - ctx.AppendWarning(warnErr) - case LevelIgnore: - } - - return nil -} - -// NewContext creates an error context to handle the errors and warnings -func NewContext(handler contextutil.WarnHandler) Context { - return NewContextWithLevels(LevelMap{}, handler) -} - -// NewContextWithLevels creates an error context to handle the errors and warnings -func NewContextWithLevels(levels LevelMap, handler contextutil.WarnHandler) Context { - intest.Assert(handler != nil) - return Context{ - warnHandler: handler, - levelMap: levels, - } -} - -// StrictNoWarningContext returns all errors directly, and ignore all errors -var StrictNoWarningContext = NewContext(contextutil.IgnoreWarn) - -var errGroupMap = make(map[errors.ErrCode]ErrGroup) - -// ErrGroup groups the error according to the behavior of handling errors -type ErrGroup int - -const ( - // ErrGroupTruncate is the group of truncated errors - ErrGroupTruncate ErrGroup = iota - // ErrGroupDupKey is the group of duplicate key errors - ErrGroupDupKey - // ErrGroupBadNull is the group of bad null errors - ErrGroupBadNull - // ErrGroupDividedByZero is the group of divided by zero errors - ErrGroupDividedByZero - // ErrGroupAutoIncReadFailed is the group of auto increment read failed errors - ErrGroupAutoIncReadFailed - // ErrGroupNoMatchedPartition is the group of no partition is matched errors. - ErrGroupNoMatchedPartition - // errGroupCount is the count of all `ErrGroup`. Please leave it at the end of the list. - errGroupCount -) - -func init() { - group2Errors := map[ErrGroup][]errors.ErrCode{ - ErrGroupTruncate: { - errno.ErrTruncatedWrongValue, - errno.ErrDataTooLong, - errno.ErrTruncatedWrongValueForField, - errno.ErrWarnDataOutOfRange, - errno.ErrDataOutOfRange, - errno.ErrBadNumber, - errno.ErrWrongValueForType, - errno.ErrDatetimeFunctionOverflow, - errno.WarnDataTruncated, - errno.ErrIncorrectDatetimeValue, - }, - ErrGroupBadNull: { - errno.ErrBadNull, - errno.ErrWarnNullToNotnull, - errno.ErrNoDefaultForField, - }, - ErrGroupDividedByZero: { - errno.ErrDivisionByZero, - }, - ErrGroupAutoIncReadFailed: { - errno.ErrAutoincReadFailed, - }, - ErrGroupNoMatchedPartition: { - errno.ErrNoPartitionForGivenValue, - errno.ErrRowDoesNotMatchGivenPartitionSet, - }, - ErrGroupDupKey: { - errno.ErrDupEntry, - }, - } - - for group, codes := range group2Errors { - for _, errCode := range codes { - errGroupMap[errCode] = group - } - } -} - -// ResolveErrLevel resolves the error level according to the `ignore` and `warn` flags -// if ignore is true, it will return `LevelIgnore` to ignore the error, -// otherwise, it will return `LevelWarn` or `LevelError` according to the `warn` flag -// Only one of `ignore` and `warn` can be true. -func ResolveErrLevel(ignore bool, warn bool) Level { - if ignore { - return LevelIgnore - } - if warn { - return LevelWarn - } - return LevelError -} diff --git a/pkg/executor/insert_common.go b/pkg/executor/insert_common.go index d0af675786a31..861ef511c0c0e 100644 --- a/pkg/executor/insert_common.go +++ b/pkg/executor/insert_common.go @@ -580,14 +580,10 @@ func (e *InsertValues) getColDefaultValue(idx int, col *table.Column) (d types.D if col.DefaultIsExpr && col.DefaultExpr != nil { defaultVal, err = table.EvalColDefaultExpr(e.Ctx(), col.ToInfo(), col.DefaultExpr) } else { -<<<<<<< HEAD - defaultVal, err = table.GetColDefaultValue(e.Ctx(), col.ToInfo()) -======= if err := table.CheckNoDefaultValueForInsert(e.Ctx().GetSessionVars().StmtCtx, col.ToInfo()); err != nil { return types.Datum{}, err } - defaultVal, err = table.GetColDefaultValue(e.Ctx().GetExprCtx(), col.ToInfo()) ->>>>>>> e586960027b (table: fix issue of get default value from column when column doesn't have default value (#51309)) + defaultVal, err = table.GetColDefaultValue(e.Ctx(), col.ToInfo()) } if err != nil { return types.Datum{}, err diff --git a/pkg/executor/test/executor/executor_test.go b/pkg/executor/test/executor/executor_test.go index 67c6180a12445..fb8455e05421d 100644 --- a/pkg/executor/test/executor/executor_test.go +++ b/pkg/executor/test/executor/executor_test.go @@ -4352,38 +4352,6 @@ func TestIssues49377(t *testing.T) { "limit 1" + ");").Sort().Check(testkit.Rows("1 Furina 1", "1 Furina 1", "1 Furina 1", "2 Klee 1", "2 Klee 1", "3 Eula 1", "3 Eula 1")) } -<<<<<<< HEAD -======= - -func TestIssues40463(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - - tk.MustExec("use test;") - tk.MustExec("CREATE TABLE `4f380f26-9af6-4df8-959d-ad6296eff914` (`f7a9a4be-3728-449b-a5ea-df9b957eec67` enum('bkdv0','9rqy','lw','neud','ym','4nbv','9a7','bpkfo','xtfl','59','6vjj') NOT NULL DEFAULT 'neud', `43ca0135-1650-429b-8887-9eabcae2a234` set('8','5x47','xc','o31','lnz','gs5s','6yam','1','20ea','i','e') NOT NULL DEFAULT 'e', PRIMARY KEY (`f7a9a4be-3728-449b-a5ea-df9b957eec67`,`43ca0135-1650-429b-8887-9eabcae2a234`) /*T![clustered_index] CLUSTERED */) ENGINE=InnoDB DEFAULT CHARSET=ascii COLLATE=ascii_bin;") - tk.MustExec("INSERT INTO `4f380f26-9af6-4df8-959d-ad6296eff914` VALUES ('bkdv0','gs5s'),('lw','20ea'),('neud','8'),('ym','o31'),('4nbv','o31'),('xtfl','e');") - - tk.MustExec("CREATE TABLE `ba35a09f-76f4-40aa-9b48-13154a24bdd2` (`9b2a7138-14a3-4e8f-b29a-720392aad22c` set('zgn','if8yo','e','k7','bav','xj6','lkag','m5','as','ia','l3') DEFAULT 'zgn,if8yo,e,k7,ia,l3',`a60d6b5c-08bd-4a6d-b951-716162d004a5` set('6li6','05jlu','w','l','m','e9r','5q','d0ol','i6ajr','csf','d32') DEFAULT '6li6,05jlu,w,l,m,d0ol,i6ajr,csf,d32',`fb753d37-6252-4bd3-9bd1-0059640e7861` year(4) DEFAULT '2065', UNIQUE KEY `51816c39-27df-4bbe-a0e7-d6b6f54be2a2` (`fb753d37-6252-4bd3-9bd1-0059640e7861`), KEY `b0dfda0a-ffed-4c5b-9a72-4113bc1cbc8e` (`9b2a7138-14a3-4e8f-b29a-720392aad22c`,`fb753d37-6252-4bd3-9bd1-0059640e7861`)) ENGINE=InnoDB DEFAULT CHARSET=utf8 COLLATE=utf8_bin /*T! SHARD_ROW_ID_BITS=5 */;") - tk.MustExec("insert into `ba35a09f-76f4-40aa-9b48-13154a24bdd2` values ('if8yo', '6li6,05jlu,w,l,m,d0ol,i6ajr,csf,d32', 2065);") - - tk.MustExec("CREATE TABLE `07ccc74e-14c3-4685-bb41-c78a069b1a6d` (`8a93bdc5-2214-4f96-b5a7-1ba4c0d396ae` bigint(20) NOT NULL DEFAULT '-4604789462044748682',`30b19ecf-679f-4ca3-813f-d3c3b8f7da7e` date NOT NULL DEFAULT '5030-11-23',`1c52eaf2-1ebb-4486-9410-dfd00c7c835c` decimal(7,5) DEFAULT '-81.91307',`4b09dfdc-e688-41cb-9ffa-d03071a43077` float DEFAULT '1.7989023',PRIMARY KEY (`30b19ecf-679f-4ca3-813f-d3c3b8f7da7e`,`8a93bdc5-2214-4f96-b5a7-1ba4c0d396ae`) /*T![clustered_index] CLUSTERED */,KEY `ae7a7637-ca52-443b-8a3f-69694f730cc4` (`8a93bdc5-2214-4f96-b5a7-1ba4c0d396ae`),KEY `42640042-8a17-4145-9510-5bb419f83ed9` (`8a93bdc5-2214-4f96-b5a7-1ba4c0d396ae`),KEY `839f4f5a-83f3-449b-a7dd-c7d2974d351a` (`30b19ecf-679f-4ca3-813f-d3c3b8f7da7e`),KEY `c474cde1-6fe4-45df-9067-b4e479f84149` (`8a93bdc5-2214-4f96-b5a7-1ba4c0d396ae`),KEY `f834d0a9-709e-4ca8-925d-73f48322b70d` (`8a93bdc5-2214-4f96-b5a7-1ba4c0d396ae`)) ENGINE=InnoDB DEFAULT CHARSET=gbk COLLATE=gbk_chinese_ci;") - tk.MustExec("set sql_mode=``;") - tk.MustExec("INSERT INTO `07ccc74e-14c3-4685-bb41-c78a069b1a6d` VALUES (616295989348159438,'0000-00-00',1.00000,1.7989023),(2215857492573998768,'1970-02-02',0.00000,1.7989023),(2215857492573998768,'1983-05-13',0.00000,1.7989023),(-2840083604831267906,'1984-01-30',1.00000,1.7989023),(599388718360890339,'1986-09-09',1.00000,1.7989023),(3506764933630033073,'1987-11-22',1.00000,1.7989023),(3506764933630033073,'2002-02-26',1.00000,1.7989023),(3506764933630033073,'2003-05-14',1.00000,1.7989023),(3506764933630033073,'2007-05-16',1.00000,1.7989023),(3506764933630033073,'2017-02-20',1.00000,1.7989023),(3506764933630033073,'2017-08-06',1.00000,1.7989023),(2215857492573998768,'2019-02-18',1.00000,1.7989023),(3506764933630033073,'2020-08-11',1.00000,1.7989023),(3506764933630033073,'2028-06-07',1.00000,1.7989023),(3506764933630033073,'2036-08-16',1.00000,1.7989023);") - - tk.MustQuery("select /*+ use_index_merge( `4f380f26-9af6-4df8-959d-ad6296eff914` ) */ /*+ stream_agg() */ approx_percentile( `4f380f26-9af6-4df8-959d-ad6296eff914`.`f7a9a4be-3728-449b-a5ea-df9b957eec67` , 77 ) as r0 , `4f380f26-9af6-4df8-959d-ad6296eff914`.`f7a9a4be-3728-449b-a5ea-df9b957eec67` as r1 from `4f380f26-9af6-4df8-959d-ad6296eff914` where not( IsNull( `4f380f26-9af6-4df8-959d-ad6296eff914`.`f7a9a4be-3728-449b-a5ea-df9b957eec67` ) ) and not( `4f380f26-9af6-4df8-959d-ad6296eff914`.`f7a9a4be-3728-449b-a5ea-df9b957eec67` in ( select `8a93bdc5-2214-4f96-b5a7-1ba4c0d396ae` from `07ccc74e-14c3-4685-bb41-c78a069b1a6d` where `4f380f26-9af6-4df8-959d-ad6296eff914`.`f7a9a4be-3728-449b-a5ea-df9b957eec67` in ( select `a60d6b5c-08bd-4a6d-b951-716162d004a5` from `ba35a09f-76f4-40aa-9b48-13154a24bdd2` where not( `4f380f26-9af6-4df8-959d-ad6296eff914`.`f7a9a4be-3728-449b-a5ea-df9b957eec67` between 'bpkfo' and '59' ) and not( `4f380f26-9af6-4df8-959d-ad6296eff914`.`f7a9a4be-3728-449b-a5ea-df9b957eec67` in ( select `fb753d37-6252-4bd3-9bd1-0059640e7861` from `ba35a09f-76f4-40aa-9b48-13154a24bdd2` where IsNull( `4f380f26-9af6-4df8-959d-ad6296eff914`.`f7a9a4be-3728-449b-a5ea-df9b957eec67` ) or not( `4f380f26-9af6-4df8-959d-ad6296eff914`.`43ca0135-1650-429b-8887-9eabcae2a234` in ( select `8a93bdc5-2214-4f96-b5a7-1ba4c0d396ae` from `07ccc74e-14c3-4685-bb41-c78a069b1a6d` where IsNull( `4f380f26-9af6-4df8-959d-ad6296eff914`.`43ca0135-1650-429b-8887-9eabcae2a234` ) and not( `4f380f26-9af6-4df8-959d-ad6296eff914`.`f7a9a4be-3728-449b-a5ea-df9b957eec67` between 'neud' and 'bpkfo' ) ) ) ) ) ) ) ) group by `4f380f26-9af6-4df8-959d-ad6296eff914`.`f7a9a4be-3728-449b-a5ea-df9b957eec67`;") -} - -func TestIssue38756(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - - tk.MustExec("use test") - tk.MustExec("create table t (c1 int)") - tk.MustExec("insert into t values (1), (2), (3)") - tk.MustQuery("SELECT SQRT(1) FROM t").Check(testkit.Rows("1", "1", "1")) - tk.MustQuery("(SELECT DISTINCT SQRT(1) FROM t)").Check(testkit.Rows("1")) - tk.MustQuery("SELECT DISTINCT cast(1 as double) FROM t").Check(testkit.Rows("1")) -} func TestIssue50043(t *testing.T) { store := testkit.CreateMockStore(t) @@ -4485,4 +4453,3 @@ func TestIssue51324(t *testing.T) { tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1364 Field 'a' doesn't have a default value", "Warning 1364 Field 'b' doesn't have a default value", "Warning 1364 Field 'c' doesn't have a default value", "Warning 1364 Field 'd' doesn't have a default value")) tk.MustQuery("select * from t order by id").Check(testkit.Rows("13 0 0", "21 1 0 1", "22 1 0 1")) } ->>>>>>> e586960027b (table: fix issue of get default value from column when column doesn't have default value (#51309)) diff --git a/pkg/planner/core/expression_rewriter.go b/pkg/planner/core/expression_rewriter.go index f2a0cd05b0b2b..b249edd19eccb 100644 --- a/pkg/planner/core/expression_rewriter.go +++ b/pkg/planner/core/expression_rewriter.go @@ -2297,7 +2297,7 @@ func (er *expressionRewriter) evalDefaultExpr(v *ast.DefaultExpr) { } default: // for other columns, just use what it is - val, er.err = er.b.getDefaultValue(col) + val, er.err = er.b.getDefaultValue(col, false) } if er.err != nil { return diff --git a/pkg/planner/core/planbuilder.go b/pkg/planner/core/planbuilder.go index 5fc63f50ad1a8..930d64128aec8 100644 --- a/pkg/planner/core/planbuilder.go +++ b/pkg/planner/core/planbuilder.go @@ -3855,7 +3855,7 @@ func genAuthErrForGrantStmt(sctx sessionctx.Context, dbName string) error { return ErrDBaccessDenied.FastGenByArgs(u, h, dbName) } -func (b *PlanBuilder) getDefaultValueForInsert(col *table.Column) (*expression.Constant, error) { +func (b *PlanBuilder) getDefaultValue(col *table.Column, isInsert bool) (*expression.Constant, error) { var ( value types.Datum err error @@ -3863,8 +3863,10 @@ func (b *PlanBuilder) getDefaultValueForInsert(col *table.Column) (*expression.C if col.DefaultIsExpr && col.DefaultExpr != nil { value, err = table.EvalColDefaultExpr(b.ctx, col.ToInfo(), col.DefaultExpr) } else { - if err := table.CheckNoDefaultValueForInsert(b.ctx.GetSessionVars().StmtCtx, col.ToInfo()); err != nil { - return nil, err + if isInsert { + if err := table.CheckNoDefaultValueForInsert(b.ctx.GetSessionVars().StmtCtx, col.ToInfo()); err != nil { + return nil, err + } } value, err = table.GetColDefaultValue(b.ctx, col.ToInfo()) } @@ -4154,7 +4156,7 @@ func (b PlanBuilder) getInsertColExpr(ctx context.Context, insertPlan *Insert, m // See note in the end of the function. Only default for generated columns are OK. return nil, nil } - outExpr, err = b.getDefaultValueForInsert(refCol) + outExpr, err = b.getDefaultValue(refCol, true) case *driver.ValueExpr: outExpr = &expression.Constant{ Value: x.Datum, diff --git a/pkg/table/BUILD.bazel b/pkg/table/BUILD.bazel index 0f0273add7154..8f5df5622c166 100644 --- a/pkg/table/BUILD.bazel +++ b/pkg/table/BUILD.bazel @@ -23,11 +23,6 @@ go_library( "//pkg/parser/types", "//pkg/sessionctx", "//pkg/sessionctx/stmtctx", -<<<<<<< HEAD -======= - "//pkg/sessionctx/variable", - "//pkg/table/context", ->>>>>>> e586960027b (table: fix issue of get default value from column when column doesn't have default value (#51309)) "//pkg/types", "//pkg/util/chunk", "//pkg/util/dbterror", diff --git a/pkg/table/column.go b/pkg/table/column.go index 7ec609d05657a..7c315fe318a41 100644 --- a/pkg/table/column.go +++ b/pkg/table/column.go @@ -34,10 +34,6 @@ import ( field_types "github.com/pingcap/tidb/pkg/parser/types" "github.com/pingcap/tidb/pkg/sessionctx" "github.com/pingcap/tidb/pkg/sessionctx/stmtctx" -<<<<<<< HEAD -======= - "github.com/pingcap/tidb/pkg/sessionctx/variable" ->>>>>>> e586960027b (table: fix issue of get default value from column when column doesn't have default value (#51309)) "github.com/pingcap/tidb/pkg/types" "github.com/pingcap/tidb/pkg/util/chunk" "github.com/pingcap/tidb/pkg/util/hack" @@ -539,8 +535,7 @@ func GetColOriginDefaultValueWithoutStrictSQLMode(ctx sessionctx.Context, col *m // But CheckNoDefaultValueForInsert logic should only check before insert. func CheckNoDefaultValueForInsert(sc *stmtctx.StatementContext, col *model.ColumnInfo) error { if mysql.HasNoDefaultValueFlag(col.GetFlag()) && !col.DefaultIsExpr && col.GetDefaultValue() == nil && col.GetType() != mysql.TypeEnum { - ignoreErr := sc.ErrGroupLevel(errctx.ErrGroupBadNull) != errctx.LevelError - if !ignoreErr { + if !sc.BadNullAsWarning { return ErrNoDefaultValue.GenWithStackByArgs(col.Name) } if !mysql.HasNotNullFlag(col.GetFlag()) { @@ -637,13 +632,8 @@ func getColDefaultValue(ctx sessionctx.Context, col *model.ColumnInfo, defaultVa return value, nil } -<<<<<<< HEAD func getColDefaultValueFromNil(ctx sessionctx.Context, col *model.ColumnInfo, args *getColOriginDefaultValue) (types.Datum, error) { - if !mysql.HasNotNullFlag(col.GetFlag()) && !mysql.HasNoDefaultValueFlag(col.GetFlag()) { -======= -func getColDefaultValueFromNil(ctx expression.BuildContext, col *model.ColumnInfo, args *getColOriginDefaultValue) (types.Datum, error) { if !mysql.HasNotNullFlag(col.GetFlag()) { ->>>>>>> e586960027b (table: fix issue of get default value from column when column doesn't have default value (#51309)) return types.Datum{}, nil } if col.GetType() == mysql.TypeEnum { @@ -671,19 +661,14 @@ func getColDefaultValueFromNil(ctx expression.BuildContext, col *model.ColumnInf sc.AppendWarning(ErrNoDefaultValue.FastGenByArgs(col.Name)) return GetZeroValue(col), nil } -<<<<<<< HEAD if sc.BadNullAsWarning { - sc.AppendWarning(ErrColumnCantNull.FastGenByArgs(col.Name)) -======= - ec := sc.ErrCtx() - var err error - if mysql.HasNoDefaultValueFlag(col.GetFlag()) { - err = ErrNoDefaultValue.FastGenByArgs(col.Name) - } else { - err = ErrColumnCantNull.FastGenByArgs(col.Name) - } - if ec.HandleError(err) == nil { ->>>>>>> e586960027b (table: fix issue of get default value from column when column doesn't have default value (#51309)) + var err error + if mysql.HasNoDefaultValueFlag(col.GetFlag()) { + err = ErrNoDefaultValue.FastGenByArgs(col.Name) + } else { + err = ErrColumnCantNull.FastGenByArgs(col.Name) + } + sc.AppendWarning(err) return GetZeroValue(col), nil } return types.Datum{}, ErrNoDefaultValue.GenWithStackByArgs(col.Name)