Skip to content

Commit

Permalink
Merge branch 'master' into seperate_plan_expr_ctx
Browse files Browse the repository at this point in the history
  • Loading branch information
lcwangchao committed Feb 28, 2024
2 parents 7058e46 + e586960 commit bcd04eb
Show file tree
Hide file tree
Showing 12 changed files with 252 additions and 22 deletions.
5 changes: 4 additions & 1 deletion pkg/ddl/db_integration_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1173,7 +1173,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", "<nil>"))

// for failing tests
sql := "alter table db_not_exist.test_alter_column alter column b set default 'c'"
Expand Down
16 changes: 14 additions & 2 deletions pkg/ddl/partition.go
Original file line number Diff line number Diff line change
Expand Up @@ -3578,6 +3578,14 @@ func checkExchangePartitionRecordValidation(w *worker, ptbl, ntbl table.Table, p
buf.WriteString(pi.Expr)
buf.WriteString(", %?) != %?")
paramList = append(paramList, pi.Num, index)
if index != 0 {
// TODO: if hash result can't be NULL, we can remove the check part.
// For example hash(id), but id is defined not NULL.
buf.WriteString(" or mod(")
buf.WriteString(pi.Expr)
buf.WriteString(", %?) is null")
paramList = append(paramList, pi.Num, index)
}
}
case model.PartitionTypeRange:
// Table has only one partition and has the maximum value
Expand Down Expand Up @@ -3691,13 +3699,17 @@ func buildCheckSQLConditionForRangeExprPartition(pi *model.PartitionInfo, index
paramList = append(paramList, driver.UnwrapFromSingleQuotes(pi.Definitions[index].LessThan[0]))
} else if index == len(pi.Definitions)-1 && strings.EqualFold(pi.Definitions[index].LessThan[0], partitionMaxValue) {
buf.WriteString(pi.Expr)
buf.WriteString(" < %?")
buf.WriteString(" < %? or ")
buf.WriteString(pi.Expr)
buf.WriteString(" is null")
paramList = append(paramList, driver.UnwrapFromSingleQuotes(pi.Definitions[index-1].LessThan[0]))
} else {
buf.WriteString(pi.Expr)
buf.WriteString(" < %? or ")
buf.WriteString(pi.Expr)
buf.WriteString(" >= %?")
buf.WriteString(" >= %? or ")
buf.WriteString(pi.Expr)
buf.WriteString(" is null")
paramList = append(paramList, driver.UnwrapFromSingleQuotes(pi.Definitions[index-1].LessThan[0]), driver.UnwrapFromSingleQuotes(pi.Definitions[index].LessThan[0]))
}
return buf.String(), paramList
Expand Down
2 changes: 1 addition & 1 deletion pkg/ddl/tests/partition/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -8,7 +8,7 @@ go_test(
"main_test.go",
],
flaky = True,
shard_count = 44,
shard_count = 45,
deps = [
"//pkg/config",
"//pkg/ddl",
Expand Down
39 changes: 39 additions & 0 deletions pkg/ddl/tests/partition/db_partition_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -3413,3 +3413,42 @@ func TestAlterLastIntervalPartition(t *testing.T) {
}

// TODO: check EXCHANGE how it handles null (for all types of partitioning!!!)
func TestExchangeValidateHandleNullValue(t *testing.T) {
store := testkit.CreateMockStore(t)

tk := testkit.NewTestKit(t, store)
tk.MustExec("use test")

tk.MustExec(`CREATE TABLE t1 (id int, c varchar(128)) PARTITION BY HASH (id) PARTITIONS 3`)
tk.MustExec(`CREATE TABLE t2 (id int, c varchar(128))`)
tk.MustExec(`insert into t1 values(null, 'a1')`)
tk.MustExec(`insert into t2 values(null, 'b2')`)
tk.MustQuery(`select id, c from t1 partition(p0)`).Check(testkit.Rows("<nil> a1"))
tk.MustContainErrMsg(`alter table t1 EXCHANGE PARTITION p1 WITH TABLE t2`,
"[ddl:1737]Found a row that does not match the partition")
tk.MustExec(`alter table t1 EXCHANGE PARTITION p0 WITH TABLE t2`)

tk.MustExec(`CREATE TABLE t3 (id int, c date) PARTITION BY HASH (year(c)) PARTITIONS 12`)
tk.MustExec(`CREATE TABLE t4 (id int, c date)`)
tk.MustExec(`insert into t3 values(1, null)`)
tk.MustExec(`insert into t4 values(2, null)`)
tk.MustQuery(`select id, c from t3 partition(p0)`).Check(testkit.Rows("1 <nil>"))
tk.MustContainErrMsg(`alter table t3 EXCHANGE PARTITION p1 WITH TABLE t4`,
"[ddl:1737]Found a row that does not match the partition")
tk.MustExec(`alter table t3 EXCHANGE PARTITION p0 WITH TABLE t4`)

tk.MustExec(`CREATE TABLE t5 (id int, c varchar(128)) partition by range (id)(
partition p0 values less than (10),
partition p1 values less than (20),
partition p2 values less than (maxvalue))`)
tk.MustExec(`CREATE TABLE t6 (id int, c varchar(128))`)
tk.MustExec(`insert into t5 values(null, 'a5')`)
tk.MustExec(`insert into t6 values(null, 'b6')`)
tk.MustQuery(`select id, c from t5 partition(p0)`).Check(testkit.Rows("<nil> a5"))
tk.MustContainErrMsg(`alter table t5 EXCHANGE PARTITION p1 WITH TABLE t6`,
"[ddl:1737]Found a row that does not match the partition")
tk.MustContainErrMsg(`alter table t5 EXCHANGE PARTITION p2 WITH TABLE t6`,
"[ddl:1737]Found a row that does not match the partition")
tk.MustExec(`alter table t5 EXCHANGE PARTITION p0 WITH TABLE t6`)
// TODO: add "partition by range columns(a, b, c)" test cases.
}
1 change: 1 addition & 0 deletions pkg/errctx/context.go
Original file line number Diff line number Diff line change
Expand Up @@ -211,6 +211,7 @@ func init() {
ErrGroupBadNull: {
errno.ErrBadNull,
errno.ErrWarnNullToNotnull,
errno.ErrNoDefaultForField,
},
ErrGroupDividedByZero: {
errno.ErrDivisionByZero,
Expand Down
3 changes: 3 additions & 0 deletions pkg/executor/insert_common.go
Original file line number Diff line number Diff line change
Expand Up @@ -584,6 +584,9 @@ func (e *InsertValues) getColDefaultValue(idx int, col *table.Column) (d types.D
if col.DefaultIsExpr && col.DefaultExpr != nil {
defaultVal, err = table.EvalColDefaultExpr(e.Ctx().GetExprCtx(), col.ToInfo(), col.DefaultExpr)
} else {
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())
}
if err != nil {
Expand Down
101 changes: 101 additions & 0 deletions pkg/executor/test/executor/executor_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -2795,3 +2795,104 @@ func TestIssue38756(t *testing.T) {
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 <nil> <nil>", "2 <nil> <nil>", "3 <nil> <nil>"))

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 <nil> <nil>", "2 <nil> <nil>", "3 <nil> <nil>", "5 <nil> <nil>"))

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 <nil> <nil>", "2 <nil> <nil>", "3 <nil> <nil>", "4 <nil> <nil>", "5 <nil> <nil>", "6 <nil> <nil>"))
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 <nil> <nil>", "12 <nil> <nil>", "13 <nil> <nil>", "14 <nil> <nil>", "15 <nil> <nil>", "16 <nil> <nil>"))

// 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 <nil> <nil> 0 0", "21 1 <nil> 0 1", "22 1 <nil> 0 1"))
}
7 changes: 5 additions & 2 deletions pkg/planner/core/planbuilder.go
Original file line number Diff line number Diff line change
Expand Up @@ -3549,14 +3549,17 @@ func genAuthErrForGrantStmt(sctx PlanContext, dbName string) error {
return plannererrors.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
)
if col.DefaultIsExpr && col.DefaultExpr != nil {
value, err = table.EvalColDefaultExpr(b.ctx.GetExprCtx(), 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.GetExprCtx(), col.ToInfo())
}
if err != nil {
Expand Down Expand Up @@ -3845,7 +3848,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,
Expand Down
1 change: 1 addition & 0 deletions pkg/table/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,7 @@ go_library(
"//pkg/parser/mysql",
"//pkg/parser/types",
"//pkg/sessionctx",
"//pkg/sessionctx/stmtctx",
"//pkg/sessionctx/variable",
"//pkg/table/context",
"//pkg/types",
Expand Down
48 changes: 32 additions & 16 deletions pkg/table/column.go
Original file line number Diff line number Diff line change
Expand Up @@ -34,6 +34,7 @@ import (
"github.com/pingcap/tidb/pkg/parser/mysql"
field_types "github.com/pingcap/tidb/pkg/parser/types"
"github.com/pingcap/tidb/pkg/sessionctx"
"github.com/pingcap/tidb/pkg/sessionctx/stmtctx"
"github.com/pingcap/tidb/pkg/sessionctx/variable"
"github.com/pingcap/tidb/pkg/types"
"github.com/pingcap/tidb/pkg/util/chunk"
Expand Down Expand Up @@ -536,6 +537,23 @@ func GetColOriginDefaultValueWithoutStrictSQLMode(ctx expression.BuildContext, c
})
}

// 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 expression.BuildContext, col *model.ColumnInfo) (types.Datum, error) {
defaultValue := col.GetDefaultValue()
Expand Down Expand Up @@ -624,22 +642,19 @@ func getColDefaultValue(ctx expression.BuildContext, col *model.ColumnInfo, defa
}

func getColDefaultValueFromNil(ctx expression.BuildContext, col *model.ColumnInfo, args *getColOriginDefaultValue) (types.Datum, error) {
if !mysql.HasNotNullFlag(col.GetFlag()) && !mysql.HasNoDefaultValueFlag(col.GetFlag()) {
if !mysql.HasNotNullFlag(col.GetFlag()) {
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
}
Expand All @@ -653,15 +668,16 @@ func getColDefaultValueFromNil(ctx expression.BuildContext, col *model.ColumnInf
}
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
}
ec := sc.ErrCtx()
if ec.HandleError(ErrColumnCantNull.FastGenByArgs(col.Name)) == nil {
var err error
if mysql.HasNoDefaultValueFlag(col.GetFlag()) {
err = ErrNoDefaultValue.FastGenByArgs(col.Name)
} else {
err = ErrColumnCantNull.FastGenByArgs(col.Name)
}
if ec.HandleError(err) == nil {
return GetZeroValue(col), nil
}
return types.Datum{}, ErrNoDefaultValue.GenWithStackByArgs(col.Name)
Expand Down
33 changes: 33 additions & 0 deletions tools/check/go-compile-without-link.sh
Original file line number Diff line number Diff line change
@@ -0,0 +1,33 @@
#!/bin/bash
# Copyright 2024 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.

# See https://gist.github.com/howardjohn/c0f5d0bc293ef7d7fada533a2c9ffaf4
# Usage: go test -exec=true -toolexec=go-compile-without-link -vet=off ./...
# Preferably as an alias like `alias go-test-compile='go test -exec=true -toolexec=go-compile-without-link -vet=off'`
# This will compile all tests, but not link them (which is the least cacheable part)

if [[ "${2}" == "-V=full" ]]; then
"$@"
exit 0
fi
case "$(basename ${1})" in
link)
# Output a dummy file
touch "${3}"
;;
# We could skip vet as well, but it can be done with -vet=off if desired
*)
"$@"
esac
Loading

0 comments on commit bcd04eb

Please sign in to comment.