diff --git a/ddl/backfilling.go b/ddl/backfilling.go index 0736cbb58215f..7761003d78e23 100644 --- a/ddl/backfilling.go +++ b/ddl/backfilling.go @@ -471,6 +471,9 @@ func (dc *ddlCtx) handleRangeTasks(sessPool *sessionPool, t table.Table, workers physicalTableID := reorgInfo.PhysicalTableID var prefix kv.Key + if tbl, ok := t.(table.PartitionedTable); ok { + t = tbl.GetPartition(physicalTableID) + } if reorgInfo.mergingTmpIdx { prefix = t.IndexPrefix() } else { diff --git a/ddl/column.go b/ddl/column.go index 6beba60a35d5c..10db5120e9351 100644 --- a/ddl/column.go +++ b/ddl/column.go @@ -1020,9 +1020,30 @@ func BuildElements(changingCol *model.ColumnInfo, changingIdxs []*model.IndexInf return elements } -func (w *worker) updatePhysicalTableRow(t table.PhysicalTable, reorgInfo *reorgInfo) error { +func (w *worker) updatePhysicalTableRow(t table.Table, reorgInfo *reorgInfo) error { logutil.BgLogger().Info("[ddl] start to update table row", zap.String("job", reorgInfo.Job.String()), zap.String("reorgInfo", reorgInfo.String())) - return w.writePhysicalTableRecord(w.sessPool, t, typeUpdateColumnWorker, reorgInfo) + if tbl, ok := t.(table.PartitionedTable); ok { + done := false + for !done { + p := tbl.GetPartition(reorgInfo.PhysicalTableID) + if p == nil { + return dbterror.ErrCancelledDDLJob.GenWithStack("Can not find partition id %d for table %d", reorgInfo.PhysicalTableID, t.Meta().ID) + } + err := w.writePhysicalTableRecord(w.sessPool, p, typeUpdateColumnWorker, reorgInfo) + if err != nil { + return err + } + done, err = w.updateReorgInfo(tbl, reorgInfo) + if err != nil { + return errors.Trace(err) + } + } + return nil + } + if tbl, ok := t.(table.PhysicalTable); ok { + return w.writePhysicalTableRecord(w.sessPool, tbl, typeUpdateColumnWorker, reorgInfo) + } + return dbterror.ErrCancelledDDLJob.GenWithStack("internal error for phys tbl id: %d tbl id: %d", reorgInfo.PhysicalTableID, t.Meta().ID) } // TestReorgGoroutineRunning is only used in test to indicate the reorg goroutine has been started. @@ -1044,22 +1065,25 @@ func (w *worker) updateCurrentElement(t table.Table, reorgInfo *reorgInfo) error } } }) - // TODO: Support partition tables. if bytes.Equal(reorgInfo.currElement.TypeKey, meta.ColumnElementKey) { - //nolint:forcetypeassert - err := w.updatePhysicalTableRow(t.(table.PhysicalTable), reorgInfo) + err := w.updatePhysicalTableRow(t, reorgInfo) if err != nil { return errors.Trace(err) } } + var physTbl table.PhysicalTable + if tbl, ok := t.(table.PartitionedTable); ok { + physTbl = tbl.GetPartition(reorgInfo.PhysicalTableID) + } else if tbl, ok := t.(table.PhysicalTable); ok { + physTbl = tbl + } // Get the original start handle and end handle. currentVer, err := getValidCurrentVersion(reorgInfo.d.store) if err != nil { return errors.Trace(err) } - //nolint:forcetypeassert - originalStartHandle, originalEndHandle, err := getTableRange(reorgInfo.d.jobContext(reorgInfo.Job), reorgInfo.d, t.(table.PhysicalTable), currentVer.Ver, reorgInfo.Job.Priority) + originalStartHandle, originalEndHandle, err := getTableRange(reorgInfo.d.jobContext(reorgInfo.Job), reorgInfo.d, physTbl, currentVer.Ver, reorgInfo.Job.Priority) if err != nil { return errors.Trace(err) } diff --git a/ddl/db_partition_test.go b/ddl/db_partition_test.go index ff75ba2f41b10..93ed264deb6e8 100644 --- a/ddl/db_partition_test.go +++ b/ddl/db_partition_test.go @@ -4528,3 +4528,131 @@ func TestPartitionTableWithAnsiQuotes(t *testing.T) { ` PARTITION "p4" VALUES LESS THAN ('\\''\t\n','\\''\t\n'),` + "\n" + ` PARTITION "pMax" VALUES LESS THAN (MAXVALUE,MAXVALUE))`)) } +func TestAlterModifyColumnOnPartitionedTable(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("create database AlterPartTable") + tk.MustExec("use AlterPartTable") + tk.MustExec(`create table t (a int unsigned PRIMARY KEY, b varchar(255), key (b))`) + tk.MustExec(`insert into t values (7, "07"), (8, "08"),(23,"23"),(34,"34💥"),(46,"46"),(57,"57")`) + tk.MustQuery(`show create table t`).Check(testkit.Rows( + "t CREATE TABLE `t` (\n" + + " `a` int(10) unsigned NOT NULL,\n" + + " `b` varchar(255) DEFAULT NULL,\n" + + " PRIMARY KEY (`a`) /*T![clustered_index] CLUSTERED */,\n" + + " KEY `b` (`b`)\n" + + ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin")) + // TODO: Why does it allow 💥 as a latin1 character? + tk.MustQuery(`select hex(b) from t where a = 34`).Check(testkit.Rows("3334F09F92A5")) + tk.MustExec(`alter table t modify b varchar(200) charset latin1`) + tk.MustQuery(`show create table t`).Check(testkit.Rows( + "t CREATE TABLE `t` (\n" + + " `a` int(10) unsigned NOT NULL,\n" + + " `b` varchar(200) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,\n" + + " PRIMARY KEY (`a`) /*T![clustered_index] CLUSTERED */,\n" + + " KEY `b` (`b`)\n" + + ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin")) + tk.MustQuery(`select hex(b) from t where a = 34`).Check(testkit.Rows("3334F09F92A5")) + tk.MustQuery(`select * from t`).Sort().Check(testkit.Rows(""+ + "23 23", + "34 34💥", + "46 46", + "57 57", + "7 07", + "8 08")) + tk.MustQuery(`select * from t order by b`).Check(testkit.Rows(""+ + "7 07", + "8 08", + "23 23", + "34 34💥", + "46 46", + "57 57")) + tk.MustExec(`alter table t change b c varchar(200) charset utf8mb4`) + tk.MustExec(`drop table t`) + tk.MustExec(`create table t (a int unsigned PRIMARY KEY, b varchar(255), key (b)) partition by range (a) ` + + `(partition p0 values less than (10),` + + ` partition p1 values less than (20),` + + ` partition p2 values less than (30),` + + ` partition pMax values less than (MAXVALUE))`) + tk.MustExec(`insert into t values (7, "07"), (8, "08"),(23,"23"),(34,"34💥"),(46,"46"),(57,"57")`) + tk.MustQuery(`select * from t`).Sort().Check(testkit.Rows(""+ + "23 23", + "34 34💥", + "46 46", + "57 57", + "7 07", + "8 08")) + tk.MustQuery(`select * from t order by b`).Check(testkit.Rows(""+ + "7 07", + "8 08", + "23 23", + "34 34💥", + "46 46", + "57 57")) + tk.MustQuery(`show create table t`).Check(testkit.Rows( + "t CREATE TABLE `t` (\n" + + " `a` int(10) unsigned NOT NULL,\n" + + " `b` varchar(255) DEFAULT NULL,\n" + + " PRIMARY KEY (`a`) /*T![clustered_index] CLUSTERED */,\n" + + " KEY `b` (`b`)\n" + + ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin\n" + + "PARTITION BY RANGE (`a`)\n" + + "(PARTITION `p0` VALUES LESS THAN (10),\n" + + " PARTITION `p1` VALUES LESS THAN (20),\n" + + " PARTITION `p2` VALUES LESS THAN (30),\n" + + " PARTITION `pMax` VALUES LESS THAN (MAXVALUE))")) + tk.MustExec(`alter table t modify b varchar(200) charset latin1`) + tk.MustQuery(`show create table t`).Check(testkit.Rows( + "t CREATE TABLE `t` (\n" + + " `a` int(10) unsigned NOT NULL,\n" + + " `b` varchar(200) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,\n" + + " PRIMARY KEY (`a`) /*T![clustered_index] CLUSTERED */,\n" + + " KEY `b` (`b`)\n" + + ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin\n" + + "PARTITION BY RANGE (`a`)\n" + + "(PARTITION `p0` VALUES LESS THAN (10),\n" + + " PARTITION `p1` VALUES LESS THAN (20),\n" + + " PARTITION `p2` VALUES LESS THAN (30),\n" + + " PARTITION `pMax` VALUES LESS THAN (MAXVALUE))")) + tk.MustQuery(`select * from t`).Sort().Check(testkit.Rows(""+ + "23 23", + "34 34💥", + "46 46", + "57 57", + "7 07", + "8 08")) + tk.MustQuery(`select * from t order by b`).Check(testkit.Rows(""+ + "7 07", + "8 08", + "23 23", + "34 34💥", + "46 46", + "57 57")) + tk.MustExec(`alter table t change b c varchar(150) charset utf8mb4`) + tk.MustQuery(`show create table t`).Check(testkit.Rows( + "t CREATE TABLE `t` (\n" + + " `a` int(10) unsigned NOT NULL,\n" + + " `c` varchar(150) DEFAULT NULL,\n" + + " PRIMARY KEY (`a`) /*T![clustered_index] CLUSTERED */,\n" + + " KEY `b` (`c`)\n" + + ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin\n" + + "PARTITION BY RANGE (`a`)\n" + + "(PARTITION `p0` VALUES LESS THAN (10),\n" + + " PARTITION `p1` VALUES LESS THAN (20),\n" + + " PARTITION `p2` VALUES LESS THAN (30),\n" + + " PARTITION `pMax` VALUES LESS THAN (MAXVALUE))")) + tk.MustQuery(`select * from t`).Sort().Check(testkit.Rows(""+ + "23 23", + "34 34💥", + "46 46", + "57 57", + "7 07", + "8 08")) + tk.MustQuery(`select * from t order by c`).Check(testkit.Rows(""+ + "7 07", + "8 08", + "23 23", + "34 34💥", + "46 46", + "57 57")) +} diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index 153567e2c9fe1..54f8243b6df8c 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -4543,9 +4543,6 @@ func GetModifiableColumnJob( if err = isGeneratedRelatedColumn(t.Meta(), newCol.ColumnInfo, col.ColumnInfo); err != nil { return nil, errors.Trace(err) } - if t.Meta().Partition != nil { - return nil, dbterror.ErrUnsupportedModifyColumn.GenWithStackByArgs("table is partition table") - } } // We don't support modifying column from not_auto_increment to auto_increment. diff --git a/ddl/failtest/fail_db_test.go b/ddl/failtest/fail_db_test.go index e4d8ea7e58342..bde5e9b1b9569 100644 --- a/ddl/failtest/fail_db_test.go +++ b/ddl/failtest/fail_db_test.go @@ -493,8 +493,6 @@ func TestModifyColumn(t *testing.T) { tk.MustExec("admin check table t") // Test unsupported statements. - tk.MustExec("create table t1(a int) partition by hash (a) partitions 2") - tk.MustGetErrMsg("alter table t1 modify column a mediumint", "[ddl:8200]Unsupported modify column: table is partition table") tk.MustExec("create table t2(id int, a int, b int generated always as (abs(a)) virtual, c int generated always as (a+1) stored)") tk.MustGetErrMsg("alter table t2 modify column b mediumint", "[ddl:8200]Unsupported modify column: newCol IsGenerated false, oldCol IsGenerated true") tk.MustGetErrMsg("alter table t2 modify column c mediumint", "[ddl:8200]Unsupported modify column: newCol IsGenerated false, oldCol IsGenerated true") @@ -531,7 +529,7 @@ func TestModifyColumn(t *testing.T) { tk.MustExec("insert into t5 values (1,1),(2,2),(3,3),(4,4),(5,5);") tk.MustExec("alter table t5 modify a int not null;") - tk.MustExec("drop table t, t1, t2, t3, t4, t5") + tk.MustExec("drop table t, t2, t3, t4, t5") } func TestPartitionAddPanic(t *testing.T) { diff --git a/expression/expression.go b/expression/expression.go index e8f38910c1264..e4b8ae764a2bd 100644 --- a/expression/expression.go +++ b/expression/expression.go @@ -818,6 +818,10 @@ func EvaluateExprWithNull(ctx sessionctx.Context, schema *Schema, expr Expressio if MaybeOverOptimized4PlanCache(ctx, []Expression{expr}) { return expr } + if ctx.GetSessionVars().StmtCtx.InNullRejectCheck { + expr, _ = evaluateExprWithNullInNullRejectCheck(ctx, schema, expr) + return expr + } return evaluateExprWithNull(ctx, schema, expr) } @@ -842,6 +846,70 @@ func evaluateExprWithNull(ctx sessionctx.Context, schema *Schema, expr Expressio return expr } +// evaluateExprWithNullInNullRejectCheck sets columns in schema as null and calculate the final result of the scalar function. +// If the Expression is a non-constant value, it means the result is unknown. +// The returned bool values indicates whether the value is influenced by the Null Constant transformed from schema column +// when the value is Null Constant. +func evaluateExprWithNullInNullRejectCheck(ctx sessionctx.Context, schema *Schema, expr Expression) (Expression, bool) { + switch x := expr.(type) { + case *ScalarFunction: + args := make([]Expression, len(x.GetArgs())) + nullFromSets := make([]bool, len(x.GetArgs())) + for i, arg := range x.GetArgs() { + args[i], nullFromSets[i] = evaluateExprWithNullInNullRejectCheck(ctx, schema, arg) + } + + // allNullFromSet indicates whether all arguments are Null Constant and the Null Constant is affected by the column of the schema. + allNullFromSet := true + for i := range args { + if cons, ok := args[i].(*Constant); ok && cons.Value.IsNull() && !nullFromSets[i] { + allNullFromSet = false + break + } + } + + // allArgsNullFromSet indicates whether all Null Constant are affected by the column of the schema + allArgsNullFromSet := true + for i := range args { + if cons, ok := args[i].(*Constant); ok && cons.Value.IsNull() && nullFromSets[i] { + continue + } + allArgsNullFromSet = false + } + + // If all the args are Null Constant and affected by the column schema, then we should keep it. + // Otherwise, we shouldn't let Null Constant which affected by the column schema participate in computing in `And` and `OR` + // due to the result of `AND` and `OR are uncertain if one of the arguments is NULL. + if !allArgsNullFromSet { + for i := range args { + if cons, ok := args[i].(*Constant); ok && cons.Value.IsNull() && nullFromSets[i] { + if x.FuncName.L == ast.LogicAnd { + args[i] = NewOne() + } + if x.FuncName.L == ast.LogicOr { + args[i] = NewZero() + } + } + } + } + c := NewFunctionInternal(ctx, x.FuncName.L, x.RetType.Clone(), args...) + cons, ok := c.(*Constant) + // If the return expr is Null Constant, and all the Null Constant arguments are affected by column schema, + // then we think the result Null Constant is also affected by the column schema + return c, ok && cons.Value.IsNull() && allNullFromSet + case *Column: + if !schema.Contains(x) { + return x, false + } + return &Constant{Value: types.Datum{}, RetType: types.NewFieldType(mysql.TypeNull)}, true + case *Constant: + if x.DeferredExpr != nil { + return FoldConstant(x), false + } + } + return expr, false +} + // TableInfo2SchemaAndNames converts the TableInfo to the schema and name slice. func TableInfo2SchemaAndNames(ctx sessionctx.Context, dbName model.CIStr, tbl *model.TableInfo) (*Schema, []*types.FieldName, error) { cols, names, err := ColumnInfos2ColumnsAndNames(ctx, dbName, tbl.Name, tbl.Cols(), tbl) diff --git a/planner/core/plan_test.go b/planner/core/plan_test.go index 242d868a8c571..ebbb9c7373401 100644 --- a/planner/core/plan_test.go +++ b/planner/core/plan_test.go @@ -1076,3 +1076,19 @@ func TestNullEQConditionPlan(t *testing.T) { {"Point_Get_5", "root", "handle:0"}, }) } + +// https://github.com/pingcap/tidb/issues/38304 +func TestOuterJoinOnNull(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("CREATE TABLE t0(c0 BLOB(5), c1 BLOB(5));") + tk.MustExec("CREATE TABLE t1 (c0 BOOL);") + tk.MustExec("INSERT INTO t1 VALUES(false);") + tk.MustExec("INSERT INTO t0(c0, c1) VALUES ('>', true);") + tk.MustQuery("SELECT * FROM t0 LEFT OUTER JOIN t1 ON NULL; ").Check(testkit.Rows("> 1 ")) + tk.MustQuery("SELECT NOT '2' =(t1.c0 AND t0.c1 IS NULL) FROM t0 LEFT OUTER JOIN t1 ON NULL; ").Check(testkit.Rows("1")) + tk.MustQuery("SELECT * FROM t0 LEFT JOIN t1 ON NULL WHERE NOT '2' =(t1.c0 AND t0.c1 IS NULL); ").Check(testkit.Rows("> 1 ")) + tk.MustQuery("SELECT * FROM t0 LEFT JOIN t1 ON NULL WHERE t1.c0 or true; ").Check(testkit.Rows("> 1 ")) + tk.MustQuery("SELECT * FROM t0 LEFT JOIN t1 ON NULL WHERE not(t1.c0 and false); ").Check(testkit.Rows("> 1 ")) +} diff --git a/planner/core/rule_predicate_push_down.go b/planner/core/rule_predicate_push_down.go index 96d62942f2346..bebed1cab141a 100644 --- a/planner/core/rule_predicate_push_down.go +++ b/planner/core/rule_predicate_push_down.go @@ -440,16 +440,20 @@ func isNullRejected(ctx sessionctx.Context, schema *expression.Schema, expr expr } sc := ctx.GetSessionVars().StmtCtx sc.InNullRejectCheck = true - result := expression.EvaluateExprWithNull(ctx, schema, expr) - sc.InNullRejectCheck = false - x, ok := result.(*expression.Constant) - if !ok { - return false - } - if x.Value.IsNull() { - return true - } else if isTrue, err := x.Value.ToBool(sc); err == nil && isTrue == 0 { - return true + defer func() { + sc.InNullRejectCheck = false + }() + for _, cond := range expression.SplitCNFItems(expr) { + result := expression.EvaluateExprWithNull(ctx, schema, cond) + x, ok := result.(*expression.Constant) + if !ok { + continue + } + if x.Value.IsNull() { + return true + } else if isTrue, err := x.Value.ToBool(sc); err == nil && isTrue == 0 { + return true + } } return false } diff --git a/store/mockstore/unistore/cophandler/cop_handler.go b/store/mockstore/unistore/cophandler/cop_handler.go index 5f375f2bfdc30..2b32c168329bd 100644 --- a/store/mockstore/unistore/cophandler/cop_handler.go +++ b/store/mockstore/unistore/cophandler/cop_handler.go @@ -401,7 +401,7 @@ func newRowDecoder(columnInfos []*tipb.ColumnInfo, fieldTps []*types.FieldType, if primaryCols != nil { pkCols = primaryCols } else { - pkCols = []int64{0} + pkCols = []int64{-1} } } def := func(i int, chk *chunk.Chunk) error {