Skip to content

Commit

Permalink
Merge branch 'master' of https://github.com/pingcap/tidb into issue#1…
Browse files Browse the repository at this point in the history
…5702
  • Loading branch information
Reminiscent committed Aug 4, 2021
2 parents 0ef4f7a + 31403ad commit 897ffe7
Show file tree
Hide file tree
Showing 21 changed files with 461 additions and 337 deletions.
35 changes: 6 additions & 29 deletions cmd/explaintest/r/generated_columns.result
Original file line number Diff line number Diff line change
Expand Up @@ -105,37 +105,14 @@ PARTITION p5 VALUES LESS THAN (6),
PARTITION max VALUES LESS THAN MAXVALUE);
EXPLAIN format = 'brief' SELECT * FROM sgc3 WHERE a <= 1;
id estRows task access object operator info
PartitionUnion 6646.67 root
├─TableReader 3323.33 root data:Selection
│ └─Selection 3323.33 cop[tikv] le(test.sgc3.a, 1)
│ └─TableFullScan 10000.00 cop[tikv] table:sgc3, partition:p0 keep order:false, stats:pseudo
└─TableReader 3323.33 root data:Selection
└─Selection 3323.33 cop[tikv] le(test.sgc3.a, 1)
└─TableFullScan 10000.00 cop[tikv] table:sgc3, partition:p1 keep order:false, stats:pseudo
TableReader 3323.33 root partition:p0,p1 data:Selection
└─Selection 3323.33 cop[tikv] le(test.sgc3.a, 1)
└─TableFullScan 10000.00 cop[tikv] table:sgc3 keep order:false, stats:pseudo
EXPLAIN format = 'brief' SELECT * FROM sgc3 WHERE a < 7;
id estRows task access object operator info
PartitionUnion 23263.33 root
├─TableReader 3323.33 root data:Selection
│ └─Selection 3323.33 cop[tikv] lt(test.sgc3.a, 7)
│ └─TableFullScan 10000.00 cop[tikv] table:sgc3, partition:p0 keep order:false, stats:pseudo
├─TableReader 3323.33 root data:Selection
│ └─Selection 3323.33 cop[tikv] lt(test.sgc3.a, 7)
│ └─TableFullScan 10000.00 cop[tikv] table:sgc3, partition:p1 keep order:false, stats:pseudo
├─TableReader 3323.33 root data:Selection
│ └─Selection 3323.33 cop[tikv] lt(test.sgc3.a, 7)
│ └─TableFullScan 10000.00 cop[tikv] table:sgc3, partition:p2 keep order:false, stats:pseudo
├─TableReader 3323.33 root data:Selection
│ └─Selection 3323.33 cop[tikv] lt(test.sgc3.a, 7)
│ └─TableFullScan 10000.00 cop[tikv] table:sgc3, partition:p3 keep order:false, stats:pseudo
├─TableReader 3323.33 root data:Selection
│ └─Selection 3323.33 cop[tikv] lt(test.sgc3.a, 7)
│ └─TableFullScan 10000.00 cop[tikv] table:sgc3, partition:p4 keep order:false, stats:pseudo
├─TableReader 3323.33 root data:Selection
│ └─Selection 3323.33 cop[tikv] lt(test.sgc3.a, 7)
│ └─TableFullScan 10000.00 cop[tikv] table:sgc3, partition:p5 keep order:false, stats:pseudo
└─TableReader 3323.33 root data:Selection
└─Selection 3323.33 cop[tikv] lt(test.sgc3.a, 7)
└─TableFullScan 10000.00 cop[tikv] table:sgc3, partition:max keep order:false, stats:pseudo
TableReader 3323.33 root partition:all data:Selection
└─Selection 3323.33 cop[tikv] lt(test.sgc3.a, 7)
└─TableFullScan 10000.00 cop[tikv] table:sgc3 keep order:false, stats:pseudo
DROP TABLE IF EXISTS t1;
CREATE TABLE t1(a INT, b INT AS (a+1) VIRTUAL, c INT AS (b+1) VIRTUAL, d INT AS (c+1) VIRTUAL, KEY(b), INDEX IDX(c, d));
INSERT INTO t1 (a) VALUES (0);
Expand Down
22 changes: 7 additions & 15 deletions cmd/explaintest/r/select.result
Original file line number Diff line number Diff line change
Expand Up @@ -359,25 +359,17 @@ insert into th values (0,0),(1,1),(2,2),(3,3),(4,4),(5,5),(6,6),(7,7),(8,8);
insert into th values (-1,-1),(-2,-2),(-3,-3),(-4,-4),(-5,-5),(-6,-6),(-7,-7),(-8,-8);
desc select * from th where a=-2;
id estRows task access object operator info
TableReader_9 10.00 root data:Selection_8
└─Selection_8 10.00 cop[tikv] eq(test.th.a, -2)
└─TableFullScan_7 10000.00 cop[tikv] table:th, partition:p2 keep order:false, stats:pseudo
TableReader_7 10.00 root partition:p2 data:Selection_6
└─Selection_6 10.00 cop[tikv] eq(test.th.a, -2)
└─TableFullScan_5 10000.00 cop[tikv] table:th keep order:false, stats:pseudo
desc select * from th;
id estRows task access object operator info
PartitionUnion_9 30000.00 root
├─TableReader_11 10000.00 root data:TableFullScan_10
│ └─TableFullScan_10 10000.00 cop[tikv] table:th, partition:p0 keep order:false, stats:pseudo
├─TableReader_13 10000.00 root data:TableFullScan_12
│ └─TableFullScan_12 10000.00 cop[tikv] table:th, partition:p1 keep order:false, stats:pseudo
└─TableReader_15 10000.00 root data:TableFullScan_14
└─TableFullScan_14 10000.00 cop[tikv] table:th, partition:p2 keep order:false, stats:pseudo
TableReader_5 10000.00 root partition:all data:TableFullScan_4
└─TableFullScan_4 10000.00 cop[tikv] table:th keep order:false, stats:pseudo
desc select * from th partition (p2,p1);
id estRows task access object operator info
PartitionUnion_8 20000.00 root
├─TableReader_10 10000.00 root data:TableFullScan_9
│ └─TableFullScan_9 10000.00 cop[tikv] table:th, partition:p1 keep order:false, stats:pseudo
└─TableReader_12 10000.00 root data:TableFullScan_11
└─TableFullScan_11 10000.00 cop[tikv] table:th, partition:p2 keep order:false, stats:pseudo
TableReader_5 10000.00 root partition:p1,p2 data:TableFullScan_4
└─TableFullScan_4 10000.00 cop[tikv] table:th keep order:false, stats:pseudo
drop table if exists t;
create table t(a int, b int);
explain format = 'brief' select a != any (select a from t t2) from t t1;
Expand Down
97 changes: 97 additions & 0 deletions ddl/db_integration_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -3134,3 +3134,100 @@ func (s *testIntegrationSuite3) TestDropTemporaryTable(c *C) {
c.Assert(err.Error(), Equals, "[schema:1051]Unknown table 'test.a_local_temp_table_9_not_exist'")
tk.MustQuery("select * from a_local_temp_table_8").Check(testkit.Rows())
}

func (s *testIntegrationSuite3) TestTruncateLocalTemporaryTable(c *C) {
tk := testkit.NewTestKit(c, s.store)
tk.MustExec("use test")
tk.MustExec("set @@tidb_enable_noop_functions = 1")

tk.MustExec("drop table if exists t1, tn")
tk.MustExec("create table t1 (id int)")
tk.MustExec("create table tn (id int)")
tk.MustExec("insert into t1 values(10), (11), (12)")
tk.MustExec("create temporary table t1 (id int primary key auto_increment)")
tk.MustExec("create temporary table t2 (id int primary key)")
tk.MustExec("create database test2")
tk.MustExec("create temporary table test2.t2 (id int)")

// truncate table out of txn
tk.MustExec("insert into t1 values(1), (2), (3)")
tk.MustExec("insert into t2 values(4), (5), (6)")
tk.MustExec("insert into test2.t2 values(7), (8), (9)")
tk.MustExec("truncate table t1")
tk.MustQuery("select * from t1").Check(testkit.Rows())
tk.MustExec("insert into t1 values()")
// auto_increment will be reset for truncate
tk.MustQuery("select * from t1").Check(testkit.Rows("1"))
tk.MustQuery("select * from t2").Check(testkit.Rows("4", "5", "6"))
tk.MustExec("truncate table t2")
tk.MustQuery("select * from t2").Check(testkit.Rows())
tk.MustQuery("select * from test2.t2").Check(testkit.Rows("7", "8", "9"))
tk.MustExec("drop table t1")
tk.MustQuery("select * from t1").Check(testkit.Rows("10", "11", "12"))
tk.MustExec("create temporary table t1 (id int primary key auto_increment)")

// truncate table with format dbName.tableName
tk.MustExec("insert into t2 values(4), (5), (6)")
tk.MustExec("insert into test2.t2 values(7), (8), (9)")
tk.MustExec("truncate table test2.t2")
tk.MustQuery("select * from test2.t2").Check(testkit.Rows())
tk.MustQuery("select * from t2").Check(testkit.Rows("4", "5", "6"))
tk.MustExec("truncate table test.t2")
tk.MustQuery("select * from t2").Check(testkit.Rows())

// truncate table in txn
tk.MustExec("insert into t1 values(1), (2), (3)")
tk.MustExec("insert into t2 values(4), (5), (6)")
tk.MustExec("begin")
tk.MustExec("insert into t1 values(11), (12)")
tk.MustExec("insert into t2 values(24), (25)")
tk.MustExec("delete from t1 where id=2")
tk.MustExec("delete from t2 where id=4")
tk.MustExec("truncate table t1")
tk.MustQuery("select * from t1").Check(testkit.Rows())
tk.MustExec("insert into t1 values()")
// auto_increment will be reset for truncate
tk.MustQuery("select * from t1").Check(testkit.Rows("1"))
tk.MustQuery("select * from t2").Check(testkit.Rows("5", "6", "24", "25"))

// since transaction already committed by truncate, so query after rollback will get same result
tk.MustExec("rollback")
tk.MustQuery("select * from t1").Check(testkit.Rows("1"))
tk.MustQuery("select * from t2").Check(testkit.Rows("5", "6", "24", "25"))

// truncate a temporary table will not effect the normal table with the same name
tk.MustExec("drop table t1")
tk.MustQuery("select * from t1").Check(testkit.Rows("10", "11", "12"))
tk.MustExec("create temporary table t1 (id int primary key auto_increment)")

// truncate temporary table will clear session data
localTemporaryTables := tk.Se.GetSessionVars().LocalTemporaryTables.(*infoschema.LocalTemporaryTables)
tb1, exist := localTemporaryTables.TableByName(model.NewCIStr("test"), model.NewCIStr("t1"))
tbl1Info := tb1.Meta()
tablePrefix := tablecodec.EncodeTablePrefix(tbl1Info.ID)
endTablePrefix := tablecodec.EncodeTablePrefix(tbl1Info.ID + 1)
c.Assert(exist, IsTrue)
tk.MustExec("insert into t1 values(1), (2), (3)")
tk.MustExec("begin")
tk.MustExec("insert into t1 values(5), (6), (7)")
tk.MustExec("truncate table t1")
iter, err := tk.Se.GetSessionVars().TemporaryTableData.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)

// truncate after drop database should be successful
tk.MustExec("create temporary table test2.t3 (id int)")
tk.MustExec("insert into test2.t3 values(1)")
tk.MustExec("drop database test2")
tk.MustExec("truncate table test2.t3")
tk.MustQuery("select * from test2.t3").Check(testkit.Rows())
}
108 changes: 82 additions & 26 deletions executor/ddl.go
Original file line number Diff line number Diff line change
Expand Up @@ -33,6 +33,7 @@ import (
"github.com/pingcap/tidb/meta/autoid"
"github.com/pingcap/tidb/planner/core"
"github.com/pingcap/tidb/sessionctx/variable"
"github.com/pingcap/tidb/table"
"github.com/pingcap/tidb/table/tables"
"github.com/pingcap/tidb/tablecodec"
"github.com/pingcap/tidb/util/admin"
Expand Down Expand Up @@ -104,6 +105,27 @@ func deleteTemporaryTableRecords(memData kv.MemBuffer, tblID int64) error {
return nil
}

func (e *DDLExec) getLocalTemporaryTables() *infoschema.LocalTemporaryTables {
tempTables := e.ctx.GetSessionVars().LocalTemporaryTables
if tempTables != nil {
return tempTables.(*infoschema.LocalTemporaryTables)
}
return nil
}

func (e *DDLExec) getLocalTemporaryTable(schema model.CIStr, table model.CIStr) (table.Table, bool) {
tbl, err := e.ctx.GetInfoSchema().(infoschema.InfoSchema).TableByName(schema, table)
if infoschema.ErrTableNotExists.Equal(err) {
return nil, false
}

if tbl.Meta().TempTableType != model.TempTableLocal {
return nil, false
}

return tbl, true
}

// Next implements the Executor Next interface.
func (e *DDLExec) Next(ctx context.Context, req *chunk.Chunk) (err error) {
if e.done {
Expand Down Expand Up @@ -218,10 +240,40 @@ func (e *DDLExec) Next(ctx context.Context, req *chunk.Chunk) (err error) {

func (e *DDLExec) executeTruncateTable(s *ast.TruncateTableStmt) error {
ident := ast.Ident{Schema: s.Table.Schema, Name: s.Table.Name}
if _, exist := e.getLocalTemporaryTable(s.Table.Schema, s.Table.Name); exist {
return e.executeTruncateLocalTemporaryTable(s)
}
err := domain.GetDomain(e.ctx).DDL().TruncateTable(e.ctx, ident)
return err
}

func (e *DDLExec) executeTruncateLocalTemporaryTable(s *ast.TruncateTableStmt) error {
tbl, exists := e.getLocalTemporaryTable(s.Table.Schema, s.Table.Name)
if !exists {
return infoschema.ErrTableNotExists.GenWithStackByArgs(s.Table.Schema, s.Table.Name)
}

tblInfo := tbl.Meta()

newTbl, err := e.newTemporaryTableFromTableInfo(tblInfo.Clone())
if err != nil {
return err
}

localTempTables := e.getLocalTemporaryTables()
localTempTables.RemoveTable(s.Table.Schema, s.Table.Name)
if err := localTempTables.AddTable(s.Table.Schema, newTbl); err != nil {
return err
}

err = deleteTemporaryTableRecords(e.ctx.GetSessionVars().TemporaryTableData, tblInfo.ID)
if err != nil {
return err
}

return nil
}

func (e *DDLExec) executeRenameTable(s *ast.RenameTableStmt) error {
isAlterTable := false
var err error
Expand Down Expand Up @@ -286,31 +338,7 @@ func (e *DDLExec) createSessionTemporaryTable(s *ast.CreateTableStmt) error {
return err
}

dom := domain.GetDomain(e.ctx)
// Local temporary table uses a real table ID.
// We could mock a table ID, but the mocked ID might be identical to an existing
// real table, and then we'll get into trouble.
err = kv.RunInNewTxn(context.Background(), dom.Store(), true, func(ctx context.Context, txn kv.Transaction) error {
m := meta.NewMeta(txn)
tblID, err := m.GenGlobalID()
if err != nil {
return errors.Trace(err)
}
tbInfo.ID = tblID
tbInfo.State = model.StatePublic
return nil
})
if err != nil {
return err
}

// AutoID is allocated in mocked..
alloc := autoid.NewAllocatorFromTempTblInfo(tbInfo)
allocs := make([]autoid.Allocator, 0, 1)
if alloc != nil {
allocs = append(allocs, alloc)
}
tbl, err := tables.TableFromMeta(allocs, tbInfo)
tbl, err := e.newTemporaryTableFromTableInfo(tbInfo)
if err != nil {
return err
}
Expand All @@ -333,7 +361,7 @@ func (e *DDLExec) createSessionTemporaryTable(s *ast.CreateTableStmt) error {
sessVars.TemporaryTableData = bufferTxn.GetMemBuffer()
}

err = localTempTables.AddTable(dbInfo, tbl)
err = localTempTables.AddTable(dbInfo.Name, tbl)

if err != nil && s.IfNotExists && infoschema.ErrTableExists.Equal(err) {
e.ctx.GetSessionVars().StmtCtx.AppendNote(err)
Expand All @@ -343,6 +371,34 @@ func (e *DDLExec) createSessionTemporaryTable(s *ast.CreateTableStmt) error {
return err
}

func (e *DDLExec) newTemporaryTableFromTableInfo(tbInfo *model.TableInfo) (table.Table, error) {
dom := domain.GetDomain(e.ctx)
// Local temporary table uses a real table ID.
// We could mock a table ID, but the mocked ID might be identical to an existing
// real table, and then we'll get into trouble.
err := kv.RunInNewTxn(context.Background(), dom.Store(), true, func(ctx context.Context, txn kv.Transaction) error {
m := meta.NewMeta(txn)
tblID, err := m.GenGlobalID()
if err != nil {
return errors.Trace(err)
}
tbInfo.ID = tblID
tbInfo.State = model.StatePublic
return nil
})
if err != nil {
return nil, err
}

// AutoID is allocated in mocked..
alloc := autoid.NewAllocatorFromTempTblInfo(tbInfo)
allocs := make([]autoid.Allocator, 0, 1)
if alloc != nil {
allocs = append(allocs, alloc)
}
return tables.TableFromMeta(allocs, tbInfo)
}

func (e *DDLExec) executeCreateView(s *ast.CreateViewStmt) error {
ret := &core.PreprocessorReturn{}
err := core.Preprocess(e.ctx, s.Select, core.WithPreprocessorReturn(ret))
Expand Down
17 changes: 17 additions & 0 deletions executor/insert_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1726,3 +1726,20 @@ func (s *testSuite13) TestGlobalTempTableParallel(c *C) {
}
wg.Wait()
}

func (s *testSuite13) TestIssue26762(c *C) {
tk := testkit.NewTestKit(c, s.store)
tk.MustExec(`use test`)
tk.MustExec("drop table if exists t1;")
tk.MustExec("create table t1(c1 date);")
_, err := tk.Exec("insert into t1 values('2020-02-31');")
c.Assert(err.Error(), Equals, `[table:1292]Incorrect date value: '2020-02-31' for column 'c1' at row 1`)

tk.MustExec("set @@sql_mode='ALLOW_INVALID_DATES';")
tk.MustExec("insert into t1 values('2020-02-31');")
tk.MustQuery("select * from t1").Check(testkit.Rows("2020-02-31"))

tk.MustExec("set @@sql_mode='STRICT_TRANS_TABLES';")
_, err = tk.Exec("insert into t1 values('2020-02-31');")
c.Assert(err.Error(), Equals, `[table:1292]Incorrect date value: '2020-02-31' for column 'c1' at row 1`)
}
14 changes: 1 addition & 13 deletions executor/stale_txn_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -865,8 +865,7 @@ func (s *testStaleTxnSuite) TestSetTransactionInfoSchema(c *C) {
c.Assert(tk.Se.GetInfoSchema().SchemaMetaVersion(), Equals, schemaVer3)
}

func (s *testStaleTxnSuite) TestStaleSelect(c *C) {
c.Skip("unstable, skip it and fix it before 20210702")
func (s *testStaleTxnSerialSuite) TestStaleSelect(c *C) {
tk := testkit.NewTestKit(c, s.store)
tk.MustExec("use test")
tk.MustExec("drop table if exists t")
Expand Down Expand Up @@ -916,20 +915,9 @@ func (s *testStaleTxnSuite) TestStaleSelect(c *C) {
tk.MustExec("insert into t values (4, 5)")
time.Sleep(10 * time.Millisecond)
tk.MustQuery("execute s").Check(staleRows)

// test dynamic timestamp stale select
time3 := time.Now()
tk.MustExec("alter table t add column d int")
tk.MustExec("insert into t values (4, 4, 4)")
time.Sleep(tolerance)
time4 := time.Now()
staleRows = testkit.Rows("1 <nil>", "2 <nil>", "3 <nil>", "4 5")
tk.MustQuery(fmt.Sprintf("select * from t as of timestamp CURRENT_TIMESTAMP(3) - INTERVAL %d MICROSECOND", time4.Sub(time3).Microseconds())).Check(staleRows)

// test prepared dynamic timestamp stale select
time5 := time.Now()
tk.MustExec(fmt.Sprintf(`prepare v from "select * from t as of timestamp CURRENT_TIMESTAMP(3) - INTERVAL %d MICROSECOND"`, time5.Sub(time3).Microseconds()))
tk.MustQuery("execute v").Check(staleRows)

// test point get
time6 := time.Now()
Expand Down
4 changes: 2 additions & 2 deletions infoschema/infoschema.go
Original file line number Diff line number Diff line change
Expand Up @@ -445,8 +445,8 @@ func (is *LocalTemporaryTables) TableByID(id int64) (tbl table.Table, ok bool) {
}

// AddTable add a table
func (is *LocalTemporaryTables) AddTable(schema *model.DBInfo, tbl table.Table) error {
schemaTables := is.ensureSchema(schema.Name)
func (is *LocalTemporaryTables) AddTable(schema model.CIStr, tbl table.Table) error {
schemaTables := is.ensureSchema(schema)

tblMeta := tbl.Meta()
if _, ok := schemaTables.tables[tblMeta.Name.L]; ok {
Expand Down
Loading

0 comments on commit 897ffe7

Please sign in to comment.