Skip to content

Commit

Permalink
Merge branch 'ref_40330_2' of https://github.com/Yisaer/tidb into ref…
Browse files Browse the repository at this point in the history
…_40330_2
  • Loading branch information
Yisaer committed Jan 17, 2023
2 parents 948e555 + ea0cf1f commit e846448
Show file tree
Hide file tree
Showing 15 changed files with 314 additions and 24 deletions.
4 changes: 4 additions & 0 deletions ddl/column.go
Original file line number Diff line number Diff line change
Expand Up @@ -552,6 +552,10 @@ func (w *worker) onModifyColumn(d *ddlCtx, t *meta.Meta, job *model.Job) (ver in
job.State = model.JobStateCancelled
return ver, errors.Trace(err)
}
if tblInfo.Partition != nil {
job.State = model.JobStateCancelled
return ver, errors.Trace(dbterror.ErrUnsupportedModifyColumn.GenWithStackByArgs("table is partition table"))
}

changingCol := modifyInfo.changingCol
if changingCol == nil {
Expand Down
38 changes: 38 additions & 0 deletions ddl/db_partition_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@ import (
"fmt"
"math/rand"
"strings"
"sync"
"sync/atomic"
"testing"
"time"
Expand Down Expand Up @@ -4528,6 +4529,43 @@ func TestPartitionTableWithAnsiQuotes(t *testing.T) {
` PARTITION "pMax" VALUES LESS THAN (MAXVALUE,MAXVALUE))`))
}

func TestIssue40135Ver2(t *testing.T) {
store, dom := testkit.CreateMockStoreAndDomain(t)
tk := testkit.NewTestKit(t, store)
tk.MustExec("use test")

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

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

tk.MustExec("CREATE TABLE t40135 ( a int DEFAULT NULL, b varchar(32) DEFAULT 'md', index(a)) PARTITION BY HASH (a) PARTITIONS 6")
tk.MustExec("insert into t40135 values (1, 'md'), (2, 'ma'), (3, 'md'), (4, 'ma'), (5, 'md'), (6, 'ma')")
one := true
hook := &ddl.TestDDLCallback{Do: dom}
var checkErr error
var wg sync.WaitGroup
wg.Add(1)
hook.OnJobRunBeforeExported = func(job *model.Job) {
if job.SchemaState == model.StateDeleteOnly {
tk3.MustExec("delete from t40135 where a = 1")
}
if one {
one = false
go func() {
_, checkErr = tk1.Exec("alter table t40135 modify column a int NULL")
wg.Done()
}()
}
}
dom.DDL().SetHook(hook)
tk.MustExec("alter table t40135 modify column a bigint NULL DEFAULT '6243108' FIRST")
wg.Wait()
require.ErrorContains(t, checkErr, "[ddl:8200]Unsupported modify column: table is partition table")
tk.MustExec("admin check table t40135")
}

func TestAlterModifyPartitionColTruncateWarning(t *testing.T) {
t.Skip("waiting for supporting Modify Partition Column again")
store := testkit.CreateMockStore(t)
Expand Down
25 changes: 25 additions & 0 deletions ddl/fktest/foreign_key_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1810,3 +1810,28 @@ func TestForeignKeyAndConcurrentDDL(t *testing.T) {
}
}
}

func TestForeignKeyAndRenameIndex(t *testing.T) {
store := testkit.CreateMockStore(t)
tk := testkit.NewTestKit(t, store)
tk.MustExec("set @@foreign_key_checks=1;")
tk.MustExec("use test")
tk.MustExec("create table t1 (id int key, b int, index idx1(b));")
tk.MustExec("create table t2 (id int key, b int, constraint fk foreign key (b) references t1(b));")
tk.MustExec("insert into t1 values (1,1),(2,2)")
tk.MustExec("insert into t2 values (1,1),(2,2)")
tk.MustGetDBError("insert into t2 values (3,3)", plannercore.ErrNoReferencedRow2)
tk.MustGetDBError("delete from t1 where id=1", plannercore.ErrRowIsReferenced2)
tk.MustExec("alter table t1 rename index idx1 to idx2")
tk.MustExec("alter table t2 rename index fk to idx")
tk.MustGetDBError("insert into t2 values (3,3)", plannercore.ErrNoReferencedRow2)
tk.MustGetDBError("delete from t1 where id=1", plannercore.ErrRowIsReferenced2)
tk.MustExec("alter table t2 drop foreign key fk")
tk.MustExec("alter table t2 add foreign key fk (b) references t1(b) on delete cascade on update cascade")
tk.MustExec("alter table t1 rename index idx2 to idx3")
tk.MustExec("alter table t2 rename index idx to idx0")
tk.MustExec("delete from t1 where id=1")
tk.MustQuery("select * from t1").Check(testkit.Rows("2 2"))
tk.MustQuery("select * from t2").Check(testkit.Rows("2 2"))
tk.MustExec("admin check table t1,t2")
}
1 change: 1 addition & 0 deletions ddl/table.go
Original file line number Diff line number Diff line change
Expand Up @@ -301,6 +301,7 @@ func onCreateView(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, _ error)
if oldTbInfoID > 0 && orReplace {
err = t.DropTableOrView(schemaID, oldTbInfoID)
if err != nil {
job.State = model.JobStateCancelled
return ver, errors.Trace(err)
}
err = t.GetAutoIDAccessors(schemaID, oldTbInfoID).Del()
Expand Down
73 changes: 73 additions & 0 deletions ddl/table_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -237,6 +237,79 @@ func TestTable(t *testing.T) {
testDropSchema(t, testkit.NewTestKit(t, store).Session(), d, dbInfo)
}

func TestCreateView(t *testing.T) {
store, domain := testkit.CreateMockStoreAndDomainWithSchemaLease(t, testLease)

d := domain.DDL()
dbInfo, err := testSchemaInfo(store, "test_table")
require.NoError(t, err)
testCreateSchema(t, testkit.NewTestKit(t, store).Session(), domain.DDL(), dbInfo)

ctx := testkit.NewTestKit(t, store).Session()

tblInfo, err := testTableInfo(store, "t", 3)
require.NoError(t, err)
job := testCreateTable(t, ctx, d, dbInfo, tblInfo)
testCheckTableState(t, store, dbInfo, tblInfo, model.StatePublic)
testCheckJobDone(t, store, job.ID, true)

// Create a view
newTblInfo0, err := testTableInfo(store, "v", 3)
require.NoError(t, err)
job = &model.Job{
SchemaID: dbInfo.ID,
TableID: tblInfo.ID,
Type: model.ActionCreateView,
BinlogInfo: &model.HistoryInfo{},
Args: []interface{}{newTblInfo0},
}
ctx.SetValue(sessionctx.QueryString, "skip")
err = d.DoDDLJob(ctx, job)
require.NoError(t, err)

v := getSchemaVer(t, ctx)
tblInfo.State = model.StatePublic
checkHistoryJobArgs(t, ctx, job.ID, &historyJobArgs{ver: v, tbl: newTblInfo0})
tblInfo.State = model.StateNone
testCheckTableState(t, store, dbInfo, tblInfo, model.StatePublic)
testCheckJobDone(t, store, job.ID, true)

// Replace a view
newTblInfo1, err := testTableInfo(store, "v", 3)
require.NoError(t, err)
job = &model.Job{
SchemaID: dbInfo.ID,
TableID: tblInfo.ID,
Type: model.ActionCreateView,
BinlogInfo: &model.HistoryInfo{},
Args: []interface{}{newTblInfo1, true, newTblInfo0.ID},
}
ctx.SetValue(sessionctx.QueryString, "skip")
err = d.DoDDLJob(ctx, job)
require.NoError(t, err)

v = getSchemaVer(t, ctx)
tblInfo.State = model.StatePublic
checkHistoryJobArgs(t, ctx, job.ID, &historyJobArgs{ver: v, tbl: newTblInfo1})
tblInfo.State = model.StateNone
testCheckTableState(t, store, dbInfo, tblInfo, model.StatePublic)
testCheckJobDone(t, store, job.ID, true)

// Replace a view with a non-existing table id
newTblInfo2, err := testTableInfo(store, "v", 3)
require.NoError(t, err)
job = &model.Job{
SchemaID: dbInfo.ID,
TableID: tblInfo.ID,
Type: model.ActionCreateView,
BinlogInfo: &model.HistoryInfo{},
Args: []interface{}{newTblInfo2, true, newTblInfo0.ID},
}
ctx.SetValue(sessionctx.QueryString, "skip")
err = d.DoDDLJob(ctx, job)
require.Error(t, err)
}

func checkTableCacheTest(t *testing.T, store kv.Storage, dbInfo *model.DBInfo, tblInfo *model.TableInfo) {
ctx := kv.WithInternalSourceType(context.Background(), kv.InternalTxnDDL)
require.NoError(t, kv.RunInNewTxn(ctx, store, false, func(ctx context.Context, txn kv.Transaction) error {
Expand Down
94 changes: 94 additions & 0 deletions executor/fktest/foreign_key_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -2745,3 +2745,97 @@ func TestForeignKeyMetaInKeyColumnUsage(t *testing.T) {
"INFORMATION_SCHEMA.KEY_COLUMN_USAGE where CONSTRAINT_SCHEMA='test' and TABLE_NAME='t2' and REFERENCED_TABLE_SCHEMA is not null and REFERENCED_COLUMN_NAME is not null;").
Check(testkit.Rows("fk test t2 a test t1 a", "fk test t2 b test t1 b"))
}

func TestForeignKeyAndGeneratedColumn(t *testing.T) {
store := testkit.CreateMockStore(t)
tk := testkit.NewTestKit(t, store)
tk.MustExec("set @@foreign_key_checks=1")
tk.MustExec("use test")
// Test foreign key with parent column is virtual generated column.
tk.MustExec("create table t1 (a int, b int as (a+1) virtual, index(b));")
tk.MustGetErrMsg("create table t2 (a int, b int, constraint fk foreign key(b) references t1(b));", "[schema:3733]Foreign key 'fk' uses virtual column 'b' which is not supported.")
// Test foreign key with child column is virtual generated column.
tk.MustExec("drop table t1")
tk.MustExec("create table t1 (a int key);")
tk.MustGetErrMsg("create table t2 (a int, c int as (a+1) virtual, constraint fk foreign key(c) references t1(a));", "[schema:3733]Foreign key 'fk' uses virtual column 'c' which is not supported.")
// Test foreign key with parent column is stored generated column.
tk.MustExec("drop table if exists t1,t2")
tk.MustExec("create table t1 (a int, b int as (a) stored, index(b));")
tk.MustExec("create table t2 (a int, b int, constraint fk foreign key(b) references t1(b) on delete cascade on update cascade);")
tk.MustExec("insert into t1 (a) values (1),(2)")
tk.MustExec("insert into t2 (a) values (1),(2)")
tk.MustExec("update t2 set b=a")
tk.MustExec("insert into t2 values (1,1),(2,2)")
tk.MustGetDBError("insert into t2 values (3,3)", plannercore.ErrNoReferencedRow2)
tk.MustQuery("select * from t2 order by a").Check(testkit.Rows("1 1", "1 1", "2 2", "2 2"))
tk.MustExec("update t1 set a=a+10 where a=1")
tk.MustQuery("select * from t1 order by a").Check(testkit.Rows("2 2", "11 11"))
tk.MustQuery("select * from t2 order by a").Check(testkit.Rows("1 11", "1 11", "2 2", "2 2"))
tk.MustExec("delete from t1 where a=2")
tk.MustQuery("select * from t1 order by a").Check(testkit.Rows("11 11"))
tk.MustQuery("select * from t2 order by a").Check(testkit.Rows("1 11", "1 11"))
// Test foreign key with parent and child column is stored generated column.
tk.MustExec("drop table if exists t1,t2")
tk.MustExec("create table t1 (a int, b int as (a) stored, index(b));")
tk.MustGetErrMsg("create table t2 (a int, b int as (a) stored, constraint fk foreign key(b) references t1(b) on update cascade);", "[ddl:3104]Cannot define foreign key with ON UPDATE CASCADE clause on a generated column.")
tk.MustGetErrMsg("create table t2 (a int, b int as (a) stored, constraint fk foreign key(b) references t1(b) on delete set null);", "[ddl:3104]Cannot define foreign key with ON DELETE SET NULL clause on a generated column.")
tk.MustExec("create table t2 (a int, b int as (a) stored, constraint fk foreign key(b) references t1(b));")
tk.MustExec("insert into t1 (a) values (1),(2)")
tk.MustExec("insert into t2 (a) values (1),(2)")
tk.MustGetDBError("insert into t2 (a) values (3)", plannercore.ErrNoReferencedRow2)
tk.MustQuery("select * from t2 order by a").Check(testkit.Rows("1 1", "2 2"))
tk.MustGetDBError("delete from t1 where b=1", plannercore.ErrRowIsReferenced2)
tk.MustGetDBError("update t1 set a=a+10 where a=1", plannercore.ErrRowIsReferenced2)
tk.MustExec("alter table t2 drop foreign key fk")
tk.MustExec("alter table t2 add foreign key fk (b) references t1(b) on delete cascade")
tk.MustExec("delete from t1 where a=1")
tk.MustQuery("select * from t1 order by a").Check(testkit.Rows("2 2"))
tk.MustQuery("select * from t2 order by a").Check(testkit.Rows("2 2"))
}

func TestForeignKeyAndExpressionIndex(t *testing.T) {
store := testkit.CreateMockStore(t)
tk := testkit.NewTestKit(t, store)
tk.MustExec("set @@foreign_key_checks=1")
tk.MustExec("use test")
tk.MustExec("create table t1 (a int, b int, index idx1 (b), index idx2 ((b*2)));")
tk.MustExec("create table t2 (a int, b int, index((b*2)), constraint fk foreign key(b) references t1(b));")
tk.MustExec("insert into t1 values (1,1),(2,2)")
tk.MustExec("insert into t2 values (1,1),(2,2)")
tk.MustGetDBError("insert into t2 values (3,3)", plannercore.ErrNoReferencedRow2)
tk.MustGetDBError("update t1 set b=b+10 where b=1", plannercore.ErrRowIsReferenced2)
tk.MustGetDBError("delete from t1 where b=1", plannercore.ErrRowIsReferenced2)
tk.MustGetErrMsg("alter table t1 drop index idx1", "[ddl:1553]Cannot drop index 'idx1': needed in a foreign key constraint")
tk.MustGetErrMsg("alter table t2 drop index fk", "[ddl:1553]Cannot drop index 'fk': needed in a foreign key constraint")
tk.MustExec("alter table t2 drop foreign key fk")
tk.MustExec("alter table t2 add foreign key fk (b) references t1(b) on delete set null on update cascade")
tk.MustExec("update t1 set b=b+10 where b=1")
tk.MustExec("delete from t1 where b=2")
tk.MustQuery("select * from t1 order by a").Check(testkit.Rows("1 11"))
tk.MustQuery("select * from t2 order by a").Check(testkit.Rows("1 11", "2 <nil>"))
tk.MustExec("admin check table t1")
tk.MustExec("admin check table t2")
}

func TestForeignKeyAndMultiValuedIndex(t *testing.T) {
store := testkit.CreateMockStore(t)
tk := testkit.NewTestKit(t, store)
tk.MustExec("set @@foreign_key_checks=1")
tk.MustExec("use test")
tk.MustExec("create table t1 (id int primary key, a json, b int generated always as (a->'$.id') stored, index idx1(b), index idx2((cast(a ->'$.data' as signed array))))")
tk.MustExec("create table t2 (id int, b int, constraint fk foreign key(b) references t1(b));")
tk.MustExec(`insert into t1 (id, a) values (1, '{"id": "1", "data": [1,11,111]}')`)
tk.MustExec(`insert into t1 (id, a) values (2, '{"id": "2", "data": [2,22,222]}')`)
tk.MustExec("insert into t2 values (1,1),(2,2)")
tk.MustGetDBError("insert into t2 values (3,3)", plannercore.ErrNoReferencedRow2)
tk.MustGetDBError(`update t1 set a='{"id": "10", "data": [1,11,111]}' where id=1`, plannercore.ErrRowIsReferenced2)
tk.MustGetDBError(`delete from t1 where id=1`, plannercore.ErrRowIsReferenced2)
tk.MustExec("alter table t2 drop foreign key fk")
tk.MustExec("alter table t2 add foreign key fk (b) references t1(b) on delete set null on update cascade")
tk.MustExec(`update t1 set a='{"id": "10", "data": [1,11,111]}' where id=1`)
tk.MustExec(`delete from t1 where id=2`)
tk.MustQuery("select id,b from t1 order by id").Check(testkit.Rows("1 10"))
tk.MustQuery("select id,b from t2 order by id").Check(testkit.Rows("1 10", "2 <nil>"))
tk.MustExec("admin check table t1")
tk.MustExec("admin check table t2")
}
1 change: 1 addition & 0 deletions planner/core/find_best_task.go
Original file line number Diff line number Diff line change
Expand Up @@ -1136,6 +1136,7 @@ func (ds *DataSource) convertToIndexMergeScan(prop *property.PhysicalProperty, c
cop.tablePlan = ts
cop.idxMergePartPlans = scans
cop.idxMergeIsIntersection = path.IndexMergeIsIntersection
cop.idxMergeAccessMVIndex = path.IndexMergeAccessMVIndex
if remainingFilters != nil {
cop.rootTaskConds = remainingFilters
}
Expand Down
2 changes: 1 addition & 1 deletion planner/core/indexmerge_path.go
Original file line number Diff line number Diff line change
Expand Up @@ -618,7 +618,7 @@ func (ds *DataSource) generateIndexMerge4MVIndex(normalPathCnt int, filters []ex

// buildPartialPathUp4MVIndex builds these partial paths up to a complete index merge path.
func (ds *DataSource) buildPartialPathUp4MVIndex(partialPaths []*util.AccessPath, isIntersection bool, remainingFilters []expression.Expression) *util.AccessPath {
indexMergePath := &util.AccessPath{PartialIndexPaths: partialPaths}
indexMergePath := &util.AccessPath{PartialIndexPaths: partialPaths, IndexMergeAccessMVIndex: true}
indexMergePath.IndexMergeIsIntersection = isIntersection
indexMergePath.TableFilters = remainingFilters

Expand Down
13 changes: 13 additions & 0 deletions planner/core/indexmerge_path_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -136,3 +136,16 @@ index i_int((cast(j->'$.int' as signed array))))`)
result.Check(testkit.Rows(output[i].Plan...))
}
}

func TestMVIndexIndexMergePlanCache(t *testing.T) {
store := testkit.CreateMockStore(t)
tk := testkit.NewTestKit(t, store)
tk.MustExec("use test")
tk.MustExec(`create table t(j json, index kj((cast(j as signed array))))`)

tk.MustExec("prepare st from 'select /*+ use_index_merge(t, kj) */ * from t where (1 member of (j))'")
tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1105 skip plan-cache: query accesses generated columns is un-cacheable"))
tk.MustExec("execute st")
tk.MustExec("execute st")
tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("0"))
}
20 changes: 0 additions & 20 deletions planner/core/optimizer.go
Original file line number Diff line number Diff line change
Expand Up @@ -1001,26 +1001,6 @@ func propagateProbeParents(plan PhysicalPlan, probeParents []PhysicalPlan) {
}
}

// useTiFlash used to check whether the plan use the TiFlash engine.
func useTiFlash(p PhysicalPlan) bool {
switch x := p.(type) {
case *PhysicalTableReader:
switch x.StoreType {
case kv.TiFlash:
return true
default:
return false
}
default:
if len(p.Children()) > 0 {
for _, plan := range p.Children() {
return useTiFlash(plan)
}
}
}
return false
}

func enableParallelApply(sctx sessionctx.Context, plan PhysicalPlan) PhysicalPlan {
if !sctx.GetSessionVars().EnableParallelApply {
return plan
Expand Down
2 changes: 2 additions & 0 deletions planner/core/physical_plans.go
Original file line number Diff line number Diff line change
Expand Up @@ -551,6 +551,8 @@ type PhysicalIndexMergeReader struct {
// IsIntersectionType means whether it's intersection type or union type.
// Intersection type is for expressions connected by `AND` and union type is for `OR`.
IsIntersectionType bool
// AccessMVIndex indicates whether this IndexMergeReader access a MVIndex.
AccessMVIndex bool

// PartialPlans flats the partialPlans to construct executor pb.
PartialPlans [][]PhysicalPlan
Expand Down
Loading

0 comments on commit e846448

Please sign in to comment.