Skip to content

Commit

Permalink
Merge branch 'master' into resource-control-privilege
Browse files Browse the repository at this point in the history
  • Loading branch information
tiancaiamao authored Jan 17, 2023
2 parents ea5aca5 + a2e1e9b commit 77b8710
Show file tree
Hide file tree
Showing 16 changed files with 239 additions and 23 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")
}
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
44 changes: 42 additions & 2 deletions planner/core/plan_cache.go
Original file line number Diff line number Diff line change
Expand Up @@ -286,7 +286,9 @@ func generateNewPlan(ctx context.Context, sctx sessionctx.Context, isNonPrepared
}

// check whether this plan is cacheable.
checkPlanCacheability(sctx, p, len(paramTypes))
if stmtCtx.UseCache {
checkPlanCacheability(sctx, p, len(paramTypes))
}

// put this plan into the plan cache.
if stmtCtx.UseCache {
Expand Down Expand Up @@ -341,7 +343,10 @@ func checkPlanCacheability(sctx sessionctx.Context, p Plan, paramNum int) {
return
}

// TODO: plans accessing MVIndex are un-cacheable
if accessMVIndexWithIndexMerge(pp) {
stmtCtx.SetSkipPlanCache(errors.New("skip plan-cache: the plan with IndexMerge accessing Multi-Valued Index is un-cacheable"))
return
}
}

// RebuildPlan4CachedPlan will rebuild this plan under current user parameters.
Expand Down Expand Up @@ -725,6 +730,41 @@ func containTableDual(p PhysicalPlan) bool {
return childContainTableDual
}

func accessMVIndexWithIndexMerge(p PhysicalPlan) bool {
if idxMerge, ok := p.(*PhysicalIndexMergeReader); ok {
if idxMerge.AccessMVIndex {
return true
}
}

for _, c := range p.Children() {
if accessMVIndexWithIndexMerge(c) {
return true
}
}
return false
}

// 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
}

// GetBindSQL4PlanCache used to get the bindSQL for plan cache to build the plan cache key.
func GetBindSQL4PlanCache(sctx sessionctx.Context, stmt *PlanCacheStmt) (string, bool) {
useBinding := sctx.GetSessionVars().UsePlanBaselines
Expand Down
2 changes: 2 additions & 0 deletions planner/core/task.go
Original file line number Diff line number Diff line change
Expand Up @@ -86,6 +86,7 @@ type copTask struct {

idxMergePartPlans []PhysicalPlan
idxMergeIsIntersection bool
idxMergeAccessMVIndex bool

// rootTaskConds stores select conditions containing virtual columns.
// These conditions can't push to TiKV, so we have to add a selection for rootTask
Expand Down Expand Up @@ -688,6 +689,7 @@ func (t *copTask) convertToRootTaskImpl(ctx sessionctx.Context) *rootTask {
partialPlans: t.idxMergePartPlans,
tablePlan: t.tablePlan,
IsIntersectionType: t.idxMergeIsIntersection,
AccessMVIndex: t.idxMergeAccessMVIndex,
}.Init(ctx, t.idxMergePartPlans[0].SelectBlockOffset())
p.PartitionInfo = t.partitionInfo
setTableScanToTableRowIDScan(p.tablePlan)
Expand Down
2 changes: 2 additions & 0 deletions planner/util/path.go
Original file line number Diff line number Diff line change
Expand Up @@ -54,6 +54,8 @@ type AccessPath struct {
// It's only valid for a IndexMerge path.
// Intersection type is for expressions connected by `AND` and union type is for `OR`.
IndexMergeIsIntersection bool
// IndexMergeAccessMVIndex indicates whether this IndexMerge path accesses a MVIndex.
IndexMergeAccessMVIndex bool

StoreType kv.StoreType

Expand Down
1 change: 1 addition & 0 deletions resourcemanager/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -13,6 +13,7 @@ go_library(
"//resourcemanager/util",
"//util",
"//util/cpu",
"@com_github_google_uuid//:uuid",
"@com_github_pingcap_log//:log",
"@org_uber_go_zap//:zap",
],
Expand Down
11 changes: 11 additions & 0 deletions resourcemanager/rm.go
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,7 @@ package resourcemanager
import (
"time"

"github.com/google/uuid"
"github.com/pingcap/tidb/resourcemanager/scheduler"
"github.com/pingcap/tidb/resourcemanager/util"
tidbutil "github.com/pingcap/tidb/util"
Expand All @@ -26,6 +27,11 @@ import (
// GlobalResourceManager is a global resource manager
var GlobalResourceManager = NewResourceManger()

// RandomName is to get a random name for register pool. It is just for test.
func RandomName() string {
return uuid.New().String()
}

// ResourceManager is a resource manager
type ResourceManager struct {
poolMap *util.ShardPoolMap
Expand Down Expand Up @@ -85,3 +91,8 @@ func (r *ResourceManager) registerPool(name string, pool *util.PoolContainer) er
func (r *ResourceManager) Unregister(name string) {
r.poolMap.Del(name)
}

// Reset is to Reset resource manager. it is just for test.
func (r *ResourceManager) Reset() {
r.poolMap = util.NewShardPoolMap()
}
1 change: 1 addition & 0 deletions testkit/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@ go_library(
"//parser/ast",
"//parser/terror",
"//planner/core",
"//resourcemanager",
"//session",
"//session/txninfo",
"//sessionctx/variable",
Expand Down
Loading

0 comments on commit 77b8710

Please sign in to comment.