Skip to content

Commit

Permalink
Merge master into feature branch (#38463)
Browse files Browse the repository at this point in the history
* mockkv: make pk id = -1 if no primary key column is used (#38443)

* ddl: support modify column on partitioned table (#38302)

close #38297

* planner: revise isnullRejected check for `And` and `OR` (#38430)

close #38304

Co-authored-by: xiongjiwei <xiongjiwei1996@outlook.com>
Co-authored-by: Song Gao <disxiaofei@163.com>
  • Loading branch information
3 people authored Oct 13, 2022
1 parent a10bb9e commit 0d316ef
Show file tree
Hide file tree
Showing 9 changed files with 262 additions and 24 deletions.
3 changes: 3 additions & 0 deletions ddl/backfilling.go
Original file line number Diff line number Diff line change
Expand Up @@ -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 {
Expand Down
38 changes: 31 additions & 7 deletions ddl/column.go
Original file line number Diff line number Diff line change
Expand Up @@ -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.
Expand All @@ -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)
}
Expand Down
128 changes: 128 additions & 0 deletions ddl/db_partition_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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"))
}
3 changes: 0 additions & 3 deletions ddl/ddl_api.go
Original file line number Diff line number Diff line change
Expand Up @@ -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.
Expand Down
4 changes: 1 addition & 3 deletions ddl/failtest/fail_db_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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")
Expand Down Expand Up @@ -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) {
Expand Down
68 changes: 68 additions & 0 deletions expression/expression.go
Original file line number Diff line number Diff line change
Expand Up @@ -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)
}

Expand All @@ -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)
Expand Down
16 changes: 16 additions & 0 deletions planner/core/plan_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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 <nil>"))
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 <nil>"))
tk.MustQuery("SELECT * FROM t0 LEFT JOIN t1 ON NULL WHERE t1.c0 or true; ").Check(testkit.Rows("> 1 <nil>"))
tk.MustQuery("SELECT * FROM t0 LEFT JOIN t1 ON NULL WHERE not(t1.c0 and false); ").Check(testkit.Rows("> 1 <nil>"))
}
24 changes: 14 additions & 10 deletions planner/core/rule_predicate_push_down.go
Original file line number Diff line number Diff line change
Expand Up @@ -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
}
Expand Down
2 changes: 1 addition & 1 deletion store/mockstore/unistore/cophandler/cop_handler.go
Original file line number Diff line number Diff line change
Expand Up @@ -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 {
Expand Down

0 comments on commit 0d316ef

Please sign in to comment.