Skip to content

Commit

Permalink
ddl: add region and split compatibility for temporary table (#24684)
Browse files Browse the repository at this point in the history
  • Loading branch information
Howie59 authored May 25, 2021
1 parent 0204954 commit 7966992
Show file tree
Hide file tree
Showing 10 changed files with 75 additions and 0 deletions.
6 changes: 6 additions & 0 deletions ddl/ddl_api.go
Original file line number Diff line number Diff line change
Expand Up @@ -1931,6 +1931,9 @@ func (d *ddl) CreateTableWithInfo(
// preSplitAndScatter performs pre-split and scatter of the table's regions.
// If `pi` is not nil, will only split region for `pi`, this is used when add partition.
func (d *ddl) preSplitAndScatter(ctx sessionctx.Context, tbInfo *model.TableInfo, pi *model.PartitionInfo) {
if tbInfo.TempTableType != model.TempTableNone {
return
}
sp, ok := d.store.(kv.SplittableStore)
if !ok || atomic.LoadUint32(&EnableSplitTableRegion) == 0 {
return
Expand Down Expand Up @@ -2220,6 +2223,9 @@ func handleTableOptions(options []*ast.TableOption, tbInfo *model.TableInfo) err
}
tbInfo.MaxShardRowIDBits = tbInfo.ShardRowIDBits
case ast.TableOptionPreSplitRegion:
if tbInfo.TempTableType != model.TempTableNone {
return errors.Trace(ErrOptOnTemporaryTable.GenWithStackByArgs("pre split regions"))
}
tbInfo.PreSplitRegions = op.UintValue
case ast.TableOptionCharset, ast.TableOptionCollate:
// We don't handle charset and collate here since they're handled in `getCharsetAndCollateInTableOption`.
Expand Down
3 changes: 3 additions & 0 deletions ddl/error.go
Original file line number Diff line number Diff line change
Expand Up @@ -279,6 +279,9 @@ var (
// ErrPartitionNoTemporary returns when partition at temporary mode
ErrPartitionNoTemporary = dbterror.ClassDDL.NewStd(mysql.ErrPartitionNoTemporary)

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

errUnsupportedOnCommitPreserve = dbterror.ClassDDL.NewStdErr(mysql.ErrUnsupportedDDLOperation, parser_mysql.Message("TiDB doesn't support ON COMMIT PRESERVE ROWS for now", nil))
errUnsupportedEngineTemporary = dbterror.ClassDDL.NewStdErr(mysql.ErrUnsupportedDDLOperation, parser_mysql.Message("TiDB doesn't support this kind of engine for temporary table", nil))
)
10 changes: 10 additions & 0 deletions ddl/serial_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -37,6 +37,7 @@ import (
"github.com/pingcap/tidb/kv"
"github.com/pingcap/tidb/meta"
"github.com/pingcap/tidb/meta/autoid"
"github.com/pingcap/tidb/planner/core"
"github.com/pingcap/tidb/session"
"github.com/pingcap/tidb/sessionctx"
"github.com/pingcap/tidb/sessionctx/variable"
Expand Down Expand Up @@ -524,6 +525,15 @@ func (s *testSerialSuite) TestCreateTableWithLike(c *C) {

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

// Test create table like at temporary mode.
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")
c.Assert(err.Error(), Equals, core.ErrOptOnTemporaryTable.GenWithStackByArgs("create table like").Error())
tk.MustExec("drop table if exists temporary_table;")
}

// 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 @@ -922,6 +922,7 @@ const (
ErrAdminCheckTable = 8003
ErrTxnTooLarge = 8004
ErrWriteConflictInTiDB = 8005
ErrOptOnTemporaryTable = 8006
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 @@ -924,6 +924,7 @@ var MySQLErrName = map[uint16]*mysql.ErrMessage{
ErrMemExceedThreshold: mysql.Message("%s holds %dB memory, exceeds threshold %dB.%s", nil),
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),
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
10 changes: 10 additions & 0 deletions errors.toml
Original file line number Diff line number Diff line change
Expand Up @@ -401,6 +401,11 @@ error = '''
Unknown SEQUENCE: '%-.300s'
'''

["ddl:8006"]
error = '''
`%s` is unsupported on temporary tables.
'''

["ddl:8200"]
error = '''
Unsupported partition by range columns
Expand Down Expand Up @@ -1136,6 +1141,11 @@ error = '''
Variable '%s' cannot be set using SET_VAR hint.
'''

["planner:8006"]
error = '''
`%s` is unsupported on temporary tables.
'''

["planner:8108"]
error = '''
Unsupported type %T
Expand Down
17 changes: 17 additions & 0 deletions executor/executor_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -5194,6 +5194,23 @@ func (s *testSplitTable) TestShowTableRegion(c *C) {
// Test show table regions.
tk.MustQuery(`split table t_regions between (-10000) and (10000) regions 4;`).Check(testkit.Rows("4 1"))
re := tk.MustQuery("show table t_regions regions")

// Test show table regions and split table on temporary table.
tk.MustExec("drop table if exists t_regions_temporary_table")
tk.MustExec("create global temporary table t_regions_temporary_table (a int key, b int, c int, index idx(b), index idx2(c)) ON COMMIT DELETE ROWS;")
// Test show table regions.
_, err = tk.Exec("show table t_regions_temporary_table regions")
c.Assert(err.Error(), Equals, plannercore.ErrOptOnTemporaryTable.GenWithStackByArgs("show table regions").Error())
// Test split table.
_, err = tk.Exec("split table t_regions_temporary_table between (-10000) and (10000) regions 4;")
c.Assert(err.Error(), Equals, plannercore.ErrOptOnTemporaryTable.GenWithStackByArgs("split table").Error())
_, err = tk.Exec("split partition table t_regions_temporary_table partition (p1,p2) index idx between (0) and (20000) regions 2;")
c.Assert(err.Error(), Equals, plannercore.ErrOptOnTemporaryTable.GenWithStackByArgs("split table").Error())
tk.MustExec("drop table if exists t_regions_temporary_table")
// Test pre split regions
_, err = tk.Exec("create global temporary table temporary_table_pre_split(id int ) pre_split_regions=2 ON COMMIT DELETE ROWS;")
c.Assert(err.Error(), Equals, ddl.ErrOptOnTemporaryTable.GenWithStackByArgs("pre split regions").Error())

rows := re.Rows()
// Table t_regions should have 5 regions now.
// 4 regions to store record data.
Expand Down
1 change: 1 addition & 0 deletions planner/core/errors.go
Original file line number Diff line number Diff line change
Expand Up @@ -97,4 +97,5 @@ var (
ErrAccessDenied = dbterror.ClassOptimizer.NewStdErr(mysql.ErrAccessDenied, mysql.MySQLErrName[mysql.ErrAccessDeniedNoPassword])
ErrBadNull = dbterror.ClassOptimizer.NewStd(mysql.ErrBadNull)
ErrNotSupportedWithSem = dbterror.ClassOptimizer.NewStd(mysql.ErrNotSupportedWithSem)
ErrOptOnTemporaryTable = dbterror.ClassOptimizer.NewStd(mysql.ErrOptOnTemporaryTable)
)
11 changes: 11 additions & 0 deletions planner/core/planbuilder.go
Original file line number Diff line number Diff line change
Expand Up @@ -2207,6 +2207,14 @@ func (b *PlanBuilder) buildShow(ctx context.Context, show *ast.ShowStmt) (Plan,
err = ErrDBaccessDenied.GenWithStackByArgs(user.AuthUsername, user.AuthHostname, mysql.SystemDB)
}
b.visitInfo = appendVisitInfo(b.visitInfo, mysql.SelectPriv, mysql.SystemDB, "", "", err)
case ast.ShowRegions:
tableInfo, err := b.is.TableByName(show.Table.Schema, show.Table.Name)
if err != nil {
return nil, err
}
if tableInfo.Meta().TempTableType != model.TempTableNone {
return nil, ErrOptOnTemporaryTable.GenWithStackByArgs("show table regions")
}
}
schema, names := buildShowSchema(show, isView, isSequence)
p.SetSchema(schema)
Expand Down Expand Up @@ -3102,6 +3110,9 @@ func (b *PlanBuilder) buildIndexAdvise(node *ast.IndexAdviseStmt) Plan {
}

func (b *PlanBuilder) buildSplitRegion(node *ast.SplitRegionStmt) (Plan, error) {
if node.Table.TableInfo.TempTableType != model.TempTableNone {
return nil, ErrOptOnTemporaryTable.GenWithStackByArgs("split table")
}
if node.SplitSyntaxOpt != nil && node.SplitSyntaxOpt.HasPartition && node.Table.TableInfo.Partition == nil {
return nil, ErrPartitionClauseOnNonpartitioned
}
Expand Down
15 changes: 15 additions & 0 deletions planner/core/preprocess.go
Original file line number Diff line number Diff line change
Expand Up @@ -585,6 +585,21 @@ func (p *preprocessor) checkAdminCheckTableGrammar(stmt *ast.AdminStmt) {
}

func (p *preprocessor) checkCreateTableGrammar(stmt *ast.CreateTableStmt) {
if stmt.ReferTable != nil {
schema := model.NewCIStr(p.ctx.GetSessionVars().CurrentDB)
if stmt.ReferTable.Schema.String() != "" {
schema = stmt.ReferTable.Schema
}
tableInfo, err := p.is.TableByName(schema, stmt.ReferTable.Name)
if err != nil {
p.err = err
return
}
if tableInfo.Meta().TempTableType != model.TempTableNone {
p.err = ErrOptOnTemporaryTable.GenWithStackByArgs("create table like")
return
}
}
tName := stmt.Table.Name.String()
if isIncorrectName(tName) {
p.err = ddl.ErrWrongTableName.GenWithStackByArgs(tName)
Expand Down

0 comments on commit 7966992

Please sign in to comment.