Skip to content

Commit

Permalink
ddl : ddl forbid partition table switch into cache and add view featu…
Browse files Browse the repository at this point in the history
…re test (#29256)
  • Loading branch information
JayLZhou authored Nov 1, 2021
1 parent 3032d87 commit 01e4bdd
Show file tree
Hide file tree
Showing 6 changed files with 129 additions and 52 deletions.
110 changes: 110 additions & 0 deletions ddl/db_cache_test.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,110 @@
// Copyright 2021 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.

package ddl_test

import (
. "github.com/pingcap/check"
"github.com/pingcap/tidb/ddl"
"github.com/pingcap/tidb/domain"
"github.com/pingcap/tidb/errno"
"github.com/pingcap/tidb/parser/terror"
"github.com/pingcap/tidb/util/testkit"
)

// test alter table cache
func (s *testDBSuite2) TestAlterTableCache(c *C) {
tk := testkit.NewTestKit(c, s.store)
tk2 := testkit.NewTestKit(c, s.store)
tk.MustExec("use test")
tk.MustExec("drop table if exists t1")
tk2.MustExec("use test")
/* Test of cache table */
tk.MustExec("create table t1 ( n int auto_increment primary key)")
tk.MustGetErrCode("alter table t1 ca", errno.ErrParse)
tk.MustGetErrCode("alter table t2 cache", errno.ErrNoSuchTable)
tk.MustExec("alter table t1 cache")
checkTableCache(c, tk.Se, "test", "t1")
tk.MustExec("drop table if exists t1")
/*Test can't skip schema checker*/
tk.MustExec("drop table if exists t1,t2")
tk.MustExec("CREATE TABLE t1 (a int)")
tk.MustExec("CREATE TABLE t2 (a int)")
tk.MustExec("begin")
tk.MustExec("insert into t1 set a=1;")
tk2.MustExec("alter table t1 cache;")
_, err := tk.Exec("commit")
c.Assert(terror.ErrorEqual(domain.ErrInfoSchemaChanged, err), IsTrue)
/* Test can skip schema checker */
tk.MustExec("begin")
tk.MustExec("insert into t1 set a=2;")
tk2.MustExec("alter table t2 cache")
tk.MustExec("commit")
// Test if a table is not exists
tk.MustExec("drop table if exists t")
tk.MustGetErrCode("alter table t cache", errno.ErrNoSuchTable)
tk.MustExec("create table t (a int)")
tk.MustExec("alter table t cache")
// Multiple alter cache is okay
tk.MustExec("alter table t cache")
tk.MustExec("alter table t cache")
// Test a temporary table
tk.MustExec("drop table if exists t")
tk.MustExec("create temporary table t (id int primary key auto_increment, u int unique, v int)")
tk.MustExec("drop table if exists tmp1")
// local temporary table alter is not supported
tk.MustGetErrCode("alter table t cache", errno.ErrUnsupportedDDLOperation)
// test global temporary table
tk.MustExec("create global temporary table tmp1 " +
"(id int not null primary key, code int not null, value int default null, unique key code(code))" +
"on commit delete rows")
tk.MustGetErrMsg("alter table tmp1 cache", ddl.ErrOptOnTemporaryTable.GenWithStackByArgs("alter temporary table cache").Error())

}

func (s *testDBSuite2) TestAlterPartitionCache(c *C) {
tk := testkit.NewTestKit(c, s.store)
tk.MustExec("use test;")
tk.MustExec("drop table if exists cache_partition_table;")
tk.MustExec("create table cache_partition_table (a int, b int) partition by hash(a) partitions 3;")
tk.MustGetErrCode("alter table cache_partition_table cache", errno.ErrOptOnCacheTable)
defer tk.MustExec("drop table if exists cache_partition_table;")
tk.MustExec("drop table if exists cache_partition_range_table;")
tk.MustExec(`create table cache_partition_range_table (c1 smallint(6) not null, c2 char(5) default null) partition by range ( c1 ) (
partition p0 values less than (10),
partition p1 values less than (20),
partition p2 values less than (30),
partition p3 values less than (MAXVALUE)
);`)
tk.MustGetErrCode("alter table cache_partition_range_table cache;", errno.ErrOptOnCacheTable)
defer tk.MustExec("drop table if exists cache_partition_range_table;")
tk.MustExec("drop table if exists partition_list_table;")
tk.MustExec("set @@session.tidb_enable_list_partition = ON")
tk.MustExec(`create table cache_partition_list_table (id int) partition by list (id) (
partition p0 values in (1,2),
partition p1 values in (3,4),
partition p3 values in (5,null)
);`)
tk.MustGetErrCode("alter table cache_partition_list_table cache", errno.ErrOptOnCacheTable)
tk.MustExec("drop table if exists cache_partition_list_table;")
}

func (s *testDBSuite2) TestAlterViewTableCache(c *C) {
tk := testkit.NewTestKit(c, s.store)
tk.MustExec("use test;")
tk.MustExec("drop table if exists cache_view_t")
tk.MustExec("create table cache_view_t (id int);")
tk.MustExec("create view v as select * from cache_view_t")
tk.MustGetErrCode("alter table v cache", errno.ErrWrongObject)
}
50 changes: 0 additions & 50 deletions ddl/db_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -5883,56 +5883,6 @@ func (s *testDBSuite2) TestTableLocksLostCommit(c *C) {
tk.MustExec("unlock tables")
}

// test alter table cache
func (s *testDBSuite2) TestAlterTableCache(c *C) {
tk := testkit.NewTestKit(c, s.store)
tk2 := testkit.NewTestKit(c, s.store)
tk.MustExec("use test")
tk.MustExec("drop table if exists t1")
tk2.MustExec("use test")
/* Test of cache table */
tk.MustExec("create table t1 ( n int auto_increment primary key)")
tk.MustGetErrCode("alter table t1 ca", errno.ErrParse)
tk.MustGetErrCode("alter table t2 cache", errno.ErrNoSuchTable)
tk.MustExec("alter table t1 cache")
checkTableCache(c, tk.Se, "test", "t1")
tk.MustExec("drop table if exists t1")
/*Test can't skip schema checker*/
tk.MustExec("drop table if exists t1,t2")
tk.MustExec("CREATE TABLE t1 (a int)")
tk.MustExec("CREATE TABLE t2 (a int)")
tk.MustExec("begin")
tk.MustExec("insert into t1 set a=1;")
tk2.MustExec("alter table t1 cache;")
_, err := tk.Exec("commit")
c.Assert(terror.ErrorEqual(domain.ErrInfoSchemaChanged, err), IsTrue)
/* Test can skip schema checker */
tk.MustExec("begin")
tk.MustExec("insert into t1 set a=2;")
tk2.MustExec("alter table t2 cache")
tk.MustExec("commit")
// Test if a table is not exists
tk.MustExec("drop table if exists t")
tk.MustGetErrCode("alter table t cache", errno.ErrNoSuchTable)
tk.MustExec("create table t (a int)")
tk.MustExec("alter table t cache")
// Multiple alter cache is okay
tk.MustExec("alter table t cache")
tk.MustExec("alter table t cache")
// Test a temporary table
tk.MustExec("drop table if exists t")
tk.MustExec("create temporary table t (id int primary key auto_increment, u int unique, v int)")
tk.MustExec("drop table if exists tmp1")
// local temporary table alter is not supported
tk.MustGetErrCode("alter table t cache", errno.ErrUnsupportedDDLOperation)
// test global temporary table
tk.MustExec("create global temporary table tmp1 " +
"(id int not null primary key, code int not null, value int default null, unique key code(code))" +
"on commit delete rows")
tk.MustGetErrMsg("alter table tmp1 cache", ddl.ErrOptOnTemporaryTable.GenWithStackByArgs("alter temporary table cache").Error())

}

// test write local lock
func (s *testDBSuite2) TestWriteLocal(c *C) {
tk := testkit.NewTestKit(c, s.store)
Expand Down
7 changes: 6 additions & 1 deletion ddl/ddl_api.go
Original file line number Diff line number Diff line change
Expand Up @@ -6640,8 +6640,13 @@ func (d *ddl) AlterTableCache(ctx sessionctx.Context, ti ast.Ident) (err error)
if t.Meta().TableCacheStatusType == model.TableCacheStatusEnable {
return nil
}

if t.Meta().TempTableType != model.TempTableNone {
return ErrOptOnTemporaryTable.GenWithStackByArgs("alter temporary table cache")
return errors.Trace(ErrOptOnTemporaryTable.GenWithStackByArgs("alter temporary table cache"))
}

if t.Meta().Partition != nil {
return errors.Trace(ErrOptOnCacheTable.GenWithStackByArgs("partition mode"))
}
job := &model.Job{
SchemaID: schema.ID,
Expand Down
3 changes: 2 additions & 1 deletion ddl/error.go
Original file line number Diff line number Diff line change
Expand Up @@ -293,7 +293,8 @@ var (

// ErrOptOnTemporaryTable returns when exec unsupported opt at temporary mode
ErrOptOnTemporaryTable = dbterror.ClassDDL.NewStd(mysql.ErrOptOnTemporaryTable)

// ErrOptOnCacheTable returns when exec unsupported opt at cache mode
ErrOptOnCacheTable = dbterror.ClassDDL.NewStd(mysql.ErrOptOnCacheTable)
errUnsupportedOnCommitPreserve = dbterror.ClassDDL.NewStdErr(mysql.ErrUnsupportedDDLOperation, parser_mysql.Message("TiDB doesn't support ON COMMIT PRESERVE ROWS for now", nil))
errUnsupportedClusteredSecondaryKey = dbterror.ClassDDL.NewStdErr(mysql.ErrUnsupportedDDLOperation, parser_mysql.Message("CLUSTERED/NONCLUSTERED keyword is only supported for primary key", nil))

Expand Down
6 changes: 6 additions & 0 deletions ddl/table.go
Original file line number Diff line number Diff line change
Expand Up @@ -1475,9 +1475,15 @@ func onAlterCacheTable(t *meta.Meta, job *model.Job) (ver int64, err error) {
job.FinishTableJob(model.JobStateDone, model.StatePublic, ver, tbInfo)
return ver, nil
}

if tbInfo.TempTableType != model.TempTableNone {
return ver, errors.Trace(ErrOptOnTemporaryTable.GenWithStackByArgs("alter temporary table cache"))
}

if tbInfo.Partition != nil {
return ver, errors.Trace(ErrOptOnCacheTable.GenWithStackByArgs("partition mode"))
}

switch tbInfo.TableCacheStatusType {
case model.TableCacheStatusDisable:
// disable -> switching
Expand Down
5 changes: 5 additions & 0 deletions errors.toml
Original file line number Diff line number Diff line change
Expand Up @@ -711,6 +711,11 @@ error = '''
Placement policy '%-.192s' is still in use
'''

["ddl:8242"]
error = '''
'%s' is unsupported on cache tables.
'''

["domain:8027"]
error = '''
Information schema is out of date: schema failed to update in 1 lease, please make sure TiDB can connect to TiKV
Expand Down

0 comments on commit 01e4bdd

Please sign in to comment.