diff --git a/ddl/db_integration_test.go b/ddl/db_integration_test.go index 0b13fe5971716..2fb65a0a2dbe9 100644 --- a/ddl/db_integration_test.go +++ b/ddl/db_integration_test.go @@ -14,6 +14,7 @@ package ddl_test import ( + "bytes" "context" "fmt" "math" @@ -2917,7 +2918,7 @@ func (s *testIntegrationSuite3) TestCreateTemporaryTable(c *C) { tk.MustExec("begin") tk.MustExec("insert into check_data values (1)") tk.MustExec("create temporary table a_local_temp_table (id int)") - // Although "begin" take a infoschem snapshot, local temporary table inside txn should be always visible. + // Although "begin" take a infoschema snapshot, local temporary table inside txn should be always visible. tk.MustExec("show create table tmp_db.a_local_temp_table") tk.MustExec("rollback") tk.MustQuery("select * from check_data").Check(testkit.Rows()) @@ -3022,3 +3023,125 @@ func (s *testIntegrationSuite3) TestAvoidCreateViewOnLocalTemporaryTable(c *C) { tk.MustExec("create view vv as select * from v0") checkCreateView() } + +func (s *testIntegrationSuite3) TestDropTemporaryTable(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + tk.MustExec("set @@tidb_enable_noop_functions = 1") + + // Check drop temporary table(include meta data and real data. + tk.MustExec("create temporary table if not exists b_local_temp_table (id int)") + tk.MustQuery("select * from b_local_temp_table").Check(testkit.Rows()) + tk.MustExec("drop table b_local_temp_table") + _, err := tk.Exec("select * from b_local_temp_table") + c.Assert(err.Error(), Equals, "[schema:1146]Table 'test.b_local_temp_table' doesn't exist") + // TODO: test drop real data + + // Check if we have a normal and local temporary table in the same db with the same name, + // local temporary table should be dropped instead of the normal table. + tk.MustExec("drop table if exists b_table_local_and_normal") + tk.MustExec("create table if not exists b_table_local_and_normal (id int)") + tk.MustExec("create temporary table if not exists b_table_local_and_normal (id int)") + tk.MustQuery("select * from b_table_local_and_normal").Check(testkit.Rows()) + tk.MustExec("drop table b_table_local_and_normal") + sequenceTable := testGetTableByName(c, tk.Se, "test", "b_table_local_and_normal") + c.Assert(sequenceTable.Meta().TempTableType, Equals, model.TempTableNone) + tk.MustExec("drop table if exists b_table_local_and_normal") + _, err = tk.Exec("select * from b_table_local_and_normal") + c.Assert(err.Error(), Equals, "[schema:1146]Table 'test.b_table_local_and_normal' doesn't exist") + + // Check dropping local temporary tables should not commit current transaction implicitly. + tk.MustExec("drop table if exists check_data_normal_table") + tk.MustExec("create table check_data_normal_table (id int)") + defer tk.MustExec("drop table if exists check_data_normal_table") + tk.MustExec("begin") + tk.MustExec("insert into check_data_normal_table values (1)") + tk.MustExec("create temporary table a_local_temp_table (id int)") + tk.MustExec("create temporary table a_local_temp_table_1 (id int)") + tk.MustExec("create temporary table a_local_temp_table_2 (id int)") + tk.MustExec("show create table a_local_temp_table") + tk.MustExec("show create table a_local_temp_table_1") + tk.MustExec("show create table a_local_temp_table_2") + tk.MustExec("drop table a_local_temp_table, a_local_temp_table_1, a_local_temp_table_2") + tk.MustExec("rollback") + tk.MustQuery("select * from check_data_normal_table").Check(testkit.Rows()) + + // Check dropping local temporary and normal tables should commit current transaction implicitly. + tk.MustExec("drop table if exists check_data_normal_table_1") + tk.MustExec("create table check_data_normal_table_1 (id int)") + defer tk.MustExec("drop table if exists check_data_normal_table_1") + tk.MustExec("begin") + tk.MustExec("insert into check_data_normal_table_1 values (1)") + tk.MustExec("create temporary table a_local_temp_table (id int)") + tk.MustExec("create temporary table a_local_temp_table_1 (id int)") + tk.MustExec("create temporary table a_local_temp_table_2 (id int)") + tk.MustExec("drop table if exists a_normal_table") + tk.MustExec("create table a_normal_table (id int)") + defer tk.MustExec("drop table if exists a_normal_table") + tk.MustExec("show create table a_local_temp_table") + tk.MustExec("show create table a_local_temp_table_1") + tk.MustExec("show create table a_local_temp_table_2") + tk.MustExec("show create table a_normal_table") + tk.MustExec("drop table a_local_temp_table, a_local_temp_table_1, a_local_temp_table_2, a_normal_table") + tk.MustExec("rollback") + tk.MustQuery("select * from check_data_normal_table_1").Check(testkit.Rows("1")) + + // Check drop not exists table. + tk.MustExec("create temporary table a_local_temp_table_3 (id int)") + tk.MustExec("create temporary table a_local_temp_table_4 (id int)") + tk.MustExec("create temporary table a_local_temp_table_5 (id int)") + tk.MustExec("drop table if exists a_normal_table_2") + tk.MustExec("create table a_normal_table_2 (id int)") + defer tk.MustExec("drop table if exists a_normal_table_2") + _, err = tk.Exec("drop table a_local_temp_table_3, a_local_temp_table_4, a_local_temp_table_5, a_normal_table_2, a_local_temp_table_6") + c.Assert(err.Error(), Equals, "[schema:1051]Unknown table 'test.a_local_temp_table_6'") + + tk.MustExec("drop table if exists check_data_normal_table_3") + tk.MustExec("create table check_data_normal_table_3 (id int)") + defer tk.MustExec("drop table if exists check_data_normal_table_3") + tk.MustExec("create temporary table a_local_temp_table_6 (id int)") + _, err = tk.Exec("drop table check_data_normal_table_3, check_data_normal_table_7, a_local_temp_table_6") + c.Assert(err.Error(), Equals, "[schema:1051]Unknown table 'test.check_data_normal_table_7'") + + // Check filter out data from removed local temp tables + tk.MustExec("create temporary table a_local_temp_table_7 (id int)") + ctx := s.ctx + c.Assert(ctx.NewTxn(context.Background()), IsNil) + txn, err := ctx.Txn(true) + c.Assert(err, IsNil) + defer func() { + err := txn.Rollback() + c.Assert(err, IsNil) + }() + sessionVars := tk.Se.GetSessionVars() + sessVarsTempTable := sessionVars.LocalTemporaryTables + localTemporaryTable := sessVarsTempTable.(*infoschema.LocalTemporaryTables) + tbl, exist := localTemporaryTable.TableByName(model.NewCIStr("test"), model.NewCIStr("a_local_temp_table_7")) + c.Assert(exist, IsTrue) + tblInfo := tbl.Meta() + tablePrefix := tablecodec.EncodeTablePrefix(tblInfo.ID) + endTablePrefix := tablecodec.EncodeTablePrefix(tblInfo.ID + 1) + + tk.MustExec("insert into a_local_temp_table_7 values (0)") + tk.MustExec("insert into a_local_temp_table_7 values (2)") + tk.MustExec("begin") + tk.MustExec("insert into a_local_temp_table_7 values (1)") + tk.MustExec("drop table if exists a_local_temp_table_7") + tk.MustExec("commit") + + _, err = tk.Exec("select * from a_local_temp_table_7") + c.Assert(err.Error(), Equals, "[schema:1146]Table 'test.a_local_temp_table_7' doesn't exist") + memData := sessionVars.TemporaryTableData + iter, err := memData.Iter(tablePrefix, endTablePrefix) + c.Assert(err, IsNil) + for iter.Valid() { + key := iter.Key() + if !bytes.HasPrefix(key, tablePrefix) { + break + } + value := iter.Value() + c.Assert(len(value), Equals, 0) + _ = iter.Next() + } + c.Assert(iter.Valid(), IsFalse) +} diff --git a/executor/ddl.go b/executor/ddl.go index b794105ec019d..2ef290fa1bf5a 100644 --- a/executor/ddl.go +++ b/executor/ddl.go @@ -14,6 +14,7 @@ package executor import ( + "bytes" "context" "fmt" "strings" @@ -33,6 +34,7 @@ import ( "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/table/tables" + "github.com/pingcap/tidb/tablecodec" "github.com/pingcap/tidb/util/admin" "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/gcutil" @@ -69,6 +71,39 @@ func (e *DDLExec) toErr(err error) error { return err } +// deleteTemporaryTableRecords delete temporary table data. +func deleteTemporaryTableRecords(memData kv.MemBuffer, tblID int64) error { + if memData == nil { + return kv.ErrNotExist + } + + tblPrefix := tablecodec.EncodeTablePrefix(tblID) + endKey := tablecodec.EncodeTablePrefix(tblID + 1) + + iter, err := memData.Iter(tblPrefix, endKey) + if err != nil { + return err + } + for iter.Valid() { + key := iter.Key() + if !bytes.HasPrefix(key, tblPrefix) { + break + } + + err = memData.Delete(key) + if err != nil { + return err + } + + err = iter.Next() + if err != nil { + return err + } + } + + return nil +} + // Next implements the Executor Next interface. func (e *DDLExec) Next(ctx context.Context, req *chunk.Chunk) (err error) { if e.done { @@ -77,12 +112,37 @@ func (e *DDLExec) Next(ctx context.Context, req *chunk.Chunk) (err error) { e.done = true // For each DDL, we should commit the previous transaction and create a new transaction. - // An exception is create local temporary table. - if s, ok := e.stmt.(*ast.CreateTableStmt); ok { + // Following cases are exceptions + var localTempTablesToDrop []*model.TableInfo + switch s := e.stmt.(type) { + case *ast.CreateTableStmt: if s.TemporaryKeyword == ast.TemporaryLocal { return e.createSessionTemporaryTable(s) } + case *ast.DropTableStmt: + if s.IsView { + break + } + sessVars := e.ctx.GetSessionVars() + sessVarsTempTable := sessVars.LocalTemporaryTables + if sessVarsTempTable == nil { + break + } + localTemporaryTables := sessVarsTempTable.(*infoschema.LocalTemporaryTables) + for tbIdx := len(s.Tables) - 1; tbIdx >= 0; tbIdx-- { + tb := s.Tables[tbIdx] + if tableInfo, ok := localTemporaryTables.TableByName(s.Tables[tbIdx].Schema, s.Tables[tbIdx].Name); ok { + localTempTablesToDrop = append(localTempTablesToDrop, tableInfo.Meta()) + localTemporaryTables.RemoveTable(tb.Schema, tb.Name) + s.Tables = append(s.Tables[:tbIdx], s.Tables[tbIdx+1:]...) + } + } + // if all tables are local temporary, directly drop those tables. + if len(s.Tables) == 0 { + return e.dropLocalTemporaryTables(localTempTablesToDrop) + } } + if err = e.ctx.NewTxn(ctx); err != nil { return err } @@ -111,6 +171,9 @@ func (e *DDLExec) Next(ctx context.Context, req *chunk.Chunk) (err error) { err = e.executeDropView(x) } else { err = e.executeDropTable(x) + if err == nil { + err = e.dropLocalTemporaryTables(localTempTablesToDrop) + } } case *ast.RecoverTableStmt: err = e.executeRecoverTable(x) @@ -143,7 +206,6 @@ func (e *DDLExec) Next(ctx context.Context, req *chunk.Chunk) (err error) { return e.toErr(err) } return err - } dom := domain.GetDomain(e.ctx) @@ -372,6 +434,7 @@ func (e *DDLExec) executeDropSequence(s *ast.DropSequenceStmt) error { // dropTableObject actually applies to `tableObject`, `viewObject` and `sequenceObject`. func (e *DDLExec) dropTableObject(objects []*ast.TableName, obt objectType, ifExists bool) error { var notExistTables []string + sessVars := e.ctx.GetSessionVars() for _, tn := range objects { fullti := ast.Ident{Schema: tn.Schema, Name: tn.Name} _, ok := e.is.SchemaByName(tn.Schema) @@ -438,15 +501,29 @@ func (e *DDLExec) dropTableObject(objects []*ast.TableName, obt objectType, ifEx if len(notExistTables) > 0 && ifExists { for _, table := range notExistTables { if obt == sequenceObject { - e.ctx.GetSessionVars().StmtCtx.AppendNote(infoschema.ErrSequenceDropExists.GenWithStackByArgs(table)) + sessVars.StmtCtx.AppendNote(infoschema.ErrSequenceDropExists.GenWithStackByArgs(table)) } else { - e.ctx.GetSessionVars().StmtCtx.AppendNote(infoschema.ErrTableDropExists.GenWithStackByArgs(table)) + sessVars.StmtCtx.AppendNote(infoschema.ErrTableDropExists.GenWithStackByArgs(table)) } } } return nil } +func (e *DDLExec) dropLocalTemporaryTables(localTempTables []*model.TableInfo) error { + if len(localTempTables) == 0 { + return nil + } + sessVars := e.ctx.GetSessionVars() + for _, tb := range localTempTables { + err := deleteTemporaryTableRecords(sessVars.TemporaryTableData, tb.ID) + if err != nil { + return err + } + } + return nil +} + func (e *DDLExec) executeDropIndex(s *ast.DropIndexStmt) error { ti := ast.Ident{Schema: s.Table.Schema, Name: s.Table.Name} err := domain.GetDomain(e.ctx).DDL().DropIndex(e.ctx, ti, model.NewCIStr(s.IndexName), s.IfExists) diff --git a/session/session.go b/session/session.go index 79252ff379e88..cef9fa7695e53 100644 --- a/session/session.go +++ b/session/session.go @@ -64,7 +64,7 @@ import ( "github.com/pingcap/tidb/plugin" "github.com/pingcap/tidb/privilege" "github.com/pingcap/tidb/privilege/privileges" - txninfo "github.com/pingcap/tidb/session/txninfo" + "github.com/pingcap/tidb/session/txninfo" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/binloginfo" "github.com/pingcap/tidb/sessionctx/stmtctx" @@ -555,13 +555,23 @@ func (s *session) doCommit(ctx context.Context) error { } func (s *session) commitTxnWithTemporaryData(ctx context.Context, txn kv.Transaction) error { - txnTempTables := s.sessionVars.TxnCtx.TemporaryTables + sessVars := s.sessionVars + txnTempTables := sessVars.TxnCtx.TemporaryTables if len(txnTempTables) == 0 { return txn.Commit(ctx) } - sessionData := s.sessionVars.TemporaryTableData - var stage kv.StagingHandle + sessionData := sessVars.TemporaryTableData + var ( + stage kv.StagingHandle + localTempTables *infoschema.LocalTemporaryTables + ) + + if sessVars.LocalTemporaryTables != nil { + localTempTables = sessVars.LocalTemporaryTables.(*infoschema.LocalTemporaryTables) + } else { + localTempTables = new(infoschema.LocalTemporaryTables) + } defer func() { // stage != kv.InvalidStagingHandle means error occurs, we need to cleanup sessionData @@ -578,6 +588,9 @@ func (s *session) commitTxnWithTemporaryData(ctx context.Context, txn kv.Transac if tbl.GetMeta().TempTableType != model.TempTableLocal { continue } + if _, ok := localTempTables.TableByID(tblID); !ok { + continue + } if stage == kv.InvalidStagingHandle { stage = sessionData.Staging()