Skip to content

Commit

Permalink
*: Merge global temporary table into release-5.1 (#25724)
Browse files Browse the repository at this point in the history
  • Loading branch information
tiancaiamao authored Jun 29, 2021
1 parent 8acd5c8 commit 3939e67
Show file tree
Hide file tree
Showing 27 changed files with 734 additions and 171 deletions.
16 changes: 16 additions & 0 deletions bindinfo/bind_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -30,6 +30,7 @@ import (
"github.com/pingcap/parser/terror"
"github.com/pingcap/tidb/bindinfo"
"github.com/pingcap/tidb/domain"
"github.com/pingcap/tidb/errno"
"github.com/pingcap/tidb/kv"
"github.com/pingcap/tidb/meta/autoid"
"github.com/pingcap/tidb/metrics"
Expand Down Expand Up @@ -2098,3 +2099,18 @@ func (s *testSuite) TestBindingWithoutCharset(c *C) {
c.Assert(rows[0][0], Equals, "select * from `test` . `t` where `a` = ?")
c.Assert(rows[0][1], Equals, "SELECT * FROM `test`.`t` WHERE `a` = 'aa'")
}

func (s *testSuite) TestTemporaryTable(c *C) {
tk := testkit.NewTestKit(c, s.store)
s.cleanBindingEnv(tk)
tk.MustExec("use test")
tk.MustExec("drop table if exists t")
tk.MustExec("set tidb_enable_global_temporary_table = true")
tk.MustExec("create global temporary table t(a int, b int, key(a), key(b)) on commit delete rows")
tk.MustExec("create table t2(a int, b int, key(a), key(b))")
tk.MustGetErrCode("create session binding for select * from t where b = 123 using select * from t ignore index(b) where b = 123;", errno.ErrOptOnTemporaryTable)
tk.MustGetErrCode("create binding for insert into t select * from t2 where t2.b = 1 and t2.c > 1 using insert into t select /*+ use_index(t2,c) */ * from t2 where t2.b = 1 and t2.c > 1", errno.ErrOptOnTemporaryTable)
tk.MustGetErrCode("create binding for replace into t select * from t2 where t2.b = 1 and t2.c > 1 using replace into t select /*+ use_index(t2,c) */ * from t2 where t2.b = 1 and t2.c > 1", errno.ErrOptOnTemporaryTable)
tk.MustGetErrCode("create binding for update t set a = 1 where b = 1 and c > 1 using update /*+ use_index(t, c) */ t set a = 1 where b = 1 and c > 1", errno.ErrOptOnTemporaryTable)
tk.MustGetErrCode("create binding for delete from t where b = 1 and c > 1 using delete /*+ use_index(t, c) */ from t where b = 1 and c > 1", errno.ErrOptOnTemporaryTable)
}
46 changes: 28 additions & 18 deletions ddl/ddl_api.go
Original file line number Diff line number Diff line change
Expand Up @@ -1642,12 +1642,15 @@ func checkTableInfoValid(tblInfo *model.TableInfo) error {
return checkInvisibleIndexOnPK(tblInfo)
}

func buildTableInfoWithLike(ident ast.Ident, referTblInfo *model.TableInfo, s *ast.CreateTableStmt) (*model.TableInfo, error) {
func buildTableInfoWithLike(ctx sessionctx.Context, ident ast.Ident, referTblInfo *model.TableInfo, s *ast.CreateTableStmt) (*model.TableInfo, error) {
// Check the referred table is a real table object.
if referTblInfo.IsSequence() || referTblInfo.IsView() {
return nil, ErrWrongObject.GenWithStackByArgs(ident.Schema, referTblInfo.Name, "BASE TABLE")
}
tblInfo := *referTblInfo
if err := setTemporaryType(ctx, &tblInfo, s); err != nil {
return nil, errors.Trace(err)
}
// Check non-public column and adjust column offset.
newColumns := referTblInfo.Cols()
newIndices := make([]*model.IndexInfo, 0, len(tblInfo.Indices))
Expand Down Expand Up @@ -1735,22 +1738,8 @@ func buildTableInfoWithStmt(ctx sessionctx.Context, s *ast.CreateTableStmt, dbCh
if err != nil {
return nil, errors.Trace(err)
}
switch s.TemporaryKeyword {
case ast.TemporaryGlobal:
tbInfo.TempTableType = model.TempTableGlobal
if !ctx.GetSessionVars().EnableGlobalTemporaryTable {
return nil, errors.New("global temporary table is experimental and it is switched off by tidb_enable_global_temporary_table")
}
// "create global temporary table ... on commit preserve rows"
if !s.OnCommitDelete {
return nil, errors.Trace(errUnsupportedOnCommitPreserve)
}
case ast.TemporaryLocal:
// TODO: set "tbInfo.TempTableType = model.TempTableLocal" after local temporary table is supported.
tbInfo.TempTableType = model.TempTableNone
ctx.GetSessionVars().StmtCtx.AppendWarning(errors.New("local TEMPORARY TABLE is not supported yet, TEMPORARY will be parsed but ignored"))
case ast.TemporaryNone:
tbInfo.TempTableType = model.TempTableNone
if err = setTemporaryType(ctx, tbInfo, s); err != nil {
return nil, errors.Trace(err)
}

if err = setTableAutoRandomBits(ctx, tbInfo, colDefs); err != nil {
Expand Down Expand Up @@ -1812,7 +1801,7 @@ func (d *ddl) CreateTable(ctx sessionctx.Context, s *ast.CreateTableStmt) (err e
// build tableInfo
var tbInfo *model.TableInfo
if s.ReferTable != nil {
tbInfo, err = buildTableInfoWithLike(ident, referTbl.Meta(), s)
tbInfo, err = buildTableInfoWithLike(ctx, ident, referTbl.Meta(), s)
} else {
tbInfo, err = buildTableInfoWithStmt(ctx, s, schema.Charset, schema.Collate)
}
Expand All @@ -1832,6 +1821,27 @@ func (d *ddl) CreateTable(ctx sessionctx.Context, s *ast.CreateTableStmt) (err e
return d.CreateTableWithInfo(ctx, schema.Name, tbInfo, onExist, false /*tryRetainID*/)
}

func setTemporaryType(ctx sessionctx.Context, tbInfo *model.TableInfo, s *ast.CreateTableStmt) error {
switch s.TemporaryKeyword {
case ast.TemporaryGlobal:
tbInfo.TempTableType = model.TempTableGlobal
if !ctx.GetSessionVars().EnableGlobalTemporaryTable {
return errors.New("global temporary table is experimental and it is switched off by tidb_enable_global_temporary_table")
}
// "create global temporary table ... on commit preserve rows"
if !s.OnCommitDelete {
return errors.Trace(errUnsupportedOnCommitPreserve)
}
case ast.TemporaryLocal:
// TODO: set "tbInfo.TempTableType = model.TempTableLocal" after local temporary table is supported.
tbInfo.TempTableType = model.TempTableNone
ctx.GetSessionVars().StmtCtx.AppendWarning(errors.New("local TEMPORARY TABLE is not supported yet, TEMPORARY will be parsed but ignored"))
default:
tbInfo.TempTableType = model.TempTableNone
}
return nil
}

func (d *ddl) CreateTableWithInfo(
ctx sessionctx.Context,
dbName model.CIStr,
Expand Down
113 changes: 112 additions & 1 deletion ddl/serial_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -525,16 +525,127 @@ func (s *testSerialSuite) TestCreateTableWithLike(c *C) {

tk.MustExec("drop database ctwl_db")
tk.MustExec("drop database ctwl_db1")
}

func (s *testSerialSuite) TestCreateTableWithLikeAtTemporaryMode(c *C) {
tk := testkit.NewTestKit(c, s.store)

// Test create table like at temporary mode.
tk.MustExec("set tidb_enable_global_temporary_table=true")
tk.MustExec("use test")
tk.MustExec("drop table if exists temporary_table;")
tk.MustExec("create global temporary table temporary_table (a int, b int,index(a)) on commit delete rows")
tk.MustExec("drop table if exists temporary_table_t1;")
_, err = tk.Exec("create table temporary_table_t1 like temporary_table")
_, err := tk.Exec("create table temporary_table_t1 like temporary_table")
c.Assert(err.Error(), Equals, core.ErrOptOnTemporaryTable.GenWithStackByArgs("create table like").Error())
tk.MustExec("drop table if exists temporary_table;")

// Test create temporary table like.
// Test auto_random.
tk.MustExec("drop table if exists auto_random_table")
_, err = tk.Exec("create table auto_random_table (a bigint primary key auto_random(3), b varchar(255));")
defer tk.MustExec("drop table if exists auto_random_table")
tk.MustExec("drop table if exists auto_random_temporary_global")
_, err = tk.Exec("create global temporary table auto_random_temporary_global like auto_random_table on commit delete rows;")
c.Assert(err.Error(), Equals, core.ErrOptOnTemporaryTable.GenWithStackByArgs("auto_random").Error())

// Test pre split regions.
tk.MustExec("drop table if exists table_pre_split")
_, err = tk.Exec("create table table_pre_split(id int) shard_row_id_bits = 2 pre_split_regions=2;")
defer tk.MustExec("drop table if exists table_pre_split")
tk.MustExec("drop table if exists temporary_table_pre_split")
_, err = tk.Exec("create global temporary table temporary_table_pre_split like table_pre_split ON COMMIT DELETE ROWS;")
c.Assert(err.Error(), Equals, core.ErrOptOnTemporaryTable.GenWithStackByArgs("pre split regions").Error())

// Test shard_row_id_bits.
tk.MustExec("drop table if exists shard_row_id_table, shard_row_id_temporary_table, shard_row_id_table_plus, shard_row_id_temporary_table_plus")
_, err = tk.Exec("create table shard_row_id_table (a int) shard_row_id_bits = 5;")
_, err = tk.Exec("create global temporary table shard_row_id_temporary_table like shard_row_id_table on commit delete rows;")
c.Assert(err.Error(), Equals, core.ErrOptOnTemporaryTable.GenWithStackByArgs("shard_row_id_bits").Error())
tk.MustExec("create table shard_row_id_table_plus (a int);")
tk.MustExec("create global temporary table shard_row_id_temporary_table_plus (a int) on commit delete rows;")
defer tk.MustExec("drop table if exists shard_row_id_table, shard_row_id_temporary_table, shard_row_id_table_plus, shard_row_id_temporary_table_plus")
_, err = tk.Exec("alter table shard_row_id_temporary_table_plus shard_row_id_bits = 4;")
c.Assert(err.Error(), Equals, ddl.ErrOptOnTemporaryTable.GenWithStackByArgs("shard_row_id_bits").Error())

// Test partition.
tk.MustExec("drop table if exists global_partition_table;")
tk.MustExec("create table global_partition_table (a int, b int) partition by hash(a) partitions 3;")
defer tk.MustExec("drop table if exists global_partition_table;")
tk.MustGetErrCode("create global temporary table global_partition_temp_table like global_partition_table ON COMMIT DELETE ROWS;",
errno.ErrPartitionNoTemporary)
// Test virtual columns.
tk.MustExec("drop table if exists test_gv_ddl, test_gv_ddl_temp")
tk.MustExec(`create table test_gv_ddl(a int, b int as (a+8) virtual, c int as (b + 2) stored)`)
tk.MustExec(`create global temporary table test_gv_ddl_temp like test_gv_ddl on commit delete rows;`)
defer tk.MustExec("drop table if exists test_gv_ddl_temp, test_gv_ddl")
is := tk.Se.(sessionctx.Context).GetInfoSchema().(infoschema.InfoSchema)
table, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("test_gv_ddl"))
c.Assert(err, IsNil)
testCases := []struct {
generatedExprString string
generatedStored bool
}{
{"", false},
{"`a` + 8", false},
{"`b` + 2", true},
}
for i, column := range table.Meta().Columns {
c.Assert(column.GeneratedExprString, Equals, testCases[i].generatedExprString)
c.Assert(column.GeneratedStored, Equals, testCases[i].generatedStored)
}
result := tk.MustQuery(`DESC test_gv_ddl_temp`)
result.Check(testkit.Rows(`a int(11) YES <nil> `, `b int(11) YES <nil> VIRTUAL GENERATED`, `c int(11) YES <nil> STORED GENERATED`))
tk.MustExec("begin;")
tk.MustExec("insert into test_gv_ddl_temp values (1, default, default)")
tk.MustQuery("select * from test_gv_ddl_temp").Check(testkit.Rows("1 9 11"))
_, err = tk.Exec("commit")
c.Assert(err, IsNil)

// Test foreign key.
tk.MustExec("drop table if exists test_foreign_key, t1")
tk.MustExec("create table t1 (a int, b int);")
tk.MustExec("create table test_foreign_key (c int,d int,foreign key (d) references t1 (b));")
defer tk.MustExec("drop table if exists test_foreign_key, t1;")
tk.MustExec("create global temporary table test_foreign_key_temp like test_foreign_key on commit delete rows;")
is = tk.Se.(sessionctx.Context).GetInfoSchema().(infoschema.InfoSchema)
table, err = is.TableByName(model.NewCIStr("test"), model.NewCIStr("test_foreign_key_temp"))
c.Assert(err, IsNil)
tableInfo := table.Meta()
c.Assert(len(tableInfo.ForeignKeys), Equals, 0)

// Issue 25613.
// Test from->normal, to->normal.
tk.MustExec("drop table if exists tb1, tb2")
tk.MustExec("create table tb1(id int);")
tk.MustExec("create table tb2 like tb1")
defer tk.MustExec("drop table if exists tb1, tb2")
tk.MustQuery("show create table tb2;").Check(testkit.Rows("tb2 CREATE TABLE `tb2` (\n" +
" `id` int(11) DEFAULT NULL\n" +
") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin"))

// Test from->normal, to->global temporary.
tk.MustExec("drop table if exists tb3, tb4")
tk.MustExec("create table tb3(id int);")
tk.MustExec("create global temporary table tb4 like tb3 on commit delete rows;")
defer tk.MustExec("drop table if exists tb3, tb4")
tk.MustQuery("show create table tb4;").Check(testkit.Rows("tb4 CREATE GLOBAL TEMPORARY TABLE `tb4` (\n" +
" `id` int(11) DEFAULT NULL\n" +
") ENGINE=memory DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin ON COMMIT DELETE ROWS"))

// Test from->global temporary, to->normal.
tk.MustExec("drop table if exists tb5, tb6")
tk.MustExec("create global temporary table tb5(id int) on commit delete rows;")
_, err = tk.Exec("create table tb6 like tb5;")
c.Assert(err.Error(), Equals, core.ErrOptOnTemporaryTable.GenWithStackByArgs("create table like").Error())
defer tk.MustExec("drop table if exists tb5, tb6")

// Test from->global temporary, to->global temporary.
tk.MustExec("drop table if exists tb7, tb8")
tk.MustExec("create global temporary table tb7(id int) on commit delete rows;")
_, err = tk.Exec("create global temporary table tb8 like tb7 on commit delete rows;")
c.Assert(err.Error(), Equals, core.ErrOptOnTemporaryTable.GenWithStackByArgs("create table like").Error())
defer tk.MustExec("drop table if exists tb7, tb8")
}

// TestCancelAddIndex1 tests canceling ddl job when the add index worker is not started.
Expand Down
1 change: 1 addition & 0 deletions errno/errcode.go
Original file line number Diff line number Diff line change
Expand Up @@ -923,6 +923,7 @@ const (
ErrTxnTooLarge = 8004
ErrWriteConflictInTiDB = 8005
ErrOptOnTemporaryTable = 8006
ErrDropTableOnTemporaryTable = 8007
ErrUnsupportedReloadPlugin = 8018
ErrUnsupportedReloadPluginVar = 8019
ErrTableLocked = 8020
Expand Down
1 change: 1 addition & 0 deletions errno/errname.go
Original file line number Diff line number Diff line change
Expand Up @@ -925,6 +925,7 @@ var MySQLErrName = map[uint16]*mysql.ErrMessage{
ErrForUpdateCantRetry: mysql.Message("[%d] can not retry select for update statement", nil),
ErrAdminCheckTable: mysql.Message("TiDB admin check table failed.", nil),
ErrOptOnTemporaryTable: mysql.Message("`%s` is unsupported on temporary tables.", nil),
ErrDropTableOnTemporaryTable: mysql.Message("`drop global temporary table` applies on global temporary table only.", nil),
ErrTxnTooLarge: mysql.Message("Transaction is too large, size: %d", nil),
ErrWriteConflictInTiDB: mysql.Message("Write conflict, txnStartTS %d is stale", nil),
ErrInvalidPluginID: mysql.Message("Wrong plugin id: %s, valid plugin id is [name]-[version], both name and version should not contain '-'", nil),
Expand Down
15 changes: 15 additions & 0 deletions errors.toml
Original file line number Diff line number Diff line change
Expand Up @@ -981,6 +981,11 @@ error = '''
`%-.192s`.`%-.192s` contains view recursion
'''

["planner:1562"]
error = '''
Cannot create temporary table with partitions
'''

["planner:1706"]
error = '''
Primary key/partition key update is not allowed since the table is updated both as '%-.192s' and '%-.192s'.
Expand Down Expand Up @@ -1156,6 +1161,11 @@ error = '''
`%s` is unsupported on temporary tables.
'''

["planner:8007"]
error = '''
`drop global temporary table` applies on global temporary table only.
'''

["planner:8108"]
error = '''
Unsupported type %T
Expand Down Expand Up @@ -1386,6 +1396,11 @@ error = '''
Unknown column '%-.192s' in '%-.192s'
'''

["table:1114"]
error = '''
The table '%-.192s' is full
'''

["table:1192"]
error = '''
Can't execute the given command because you have active locked tables or an active transaction
Expand Down
2 changes: 2 additions & 0 deletions executor/adapter.go
Original file line number Diff line number Diff line change
Expand Up @@ -796,6 +796,8 @@ func (a *ExecStmt) buildExecutor() (Executor, error) {
}

b := newExecutorBuilder(ctx, a.InfoSchema, a.Ti, a.SnapshotTS, a.ExplicitStaleness, a.TxnScope)
b.snapshotTS = a.SnapshotTS
b.explicitStaleness = a.ExplicitStaleness
e := b.build(a.Plan)
if b.err != nil {
return nil, errors.Trace(b.err)
Expand Down
Loading

0 comments on commit 3939e67

Please sign in to comment.