diff --git a/ast/misc.go b/ast/misc.go index 0cf7e4d6550b8..73e941e6ceab7 100644 --- a/ast/misc.go +++ b/ast/misc.go @@ -595,6 +595,8 @@ const ( AdminCheckIndexRange AdminShowDDLJobQueries AdminChecksumTable + AdminShowSlow + AdminShowNextRowID ) // HandleRange represents a range where handle value >= Begin and < End. diff --git a/executor/admin_test.go b/executor/admin_test.go index d3de154b052a9..b7adec09cfc07 100644 --- a/executor/admin_test.go +++ b/executor/admin_test.go @@ -15,6 +15,7 @@ package executor_test import ( . "github.com/pingcap/check" + "github.com/pingcap/tidb/meta/autoid" "github.com/pingcap/tidb/model" "github.com/pingcap/tidb/table/tables" "github.com/pingcap/tidb/types" @@ -458,3 +459,45 @@ func (s *testSuite) TestAdminCheckTable(c *C) { tk.MustExec(`ALTER TABLE t1 ADD INDEX idx6 (c6)`) tk.MustExec(`admin check table t1`) } + +func (s *testSuite) TestAdminShowNextID(c *C) { + step := int64(10) + autoIDStep := autoid.GetStep() + autoid.SetStep(step) + defer autoid.SetStep(autoIDStep) + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + tk.MustExec("create table t(id int, c int)") + // Start handle is 1. + r := tk.MustQuery("admin show t next_row_id") + r.Check(testkit.Rows("test t _tidb_rowid 1")) + // Row ID is step + 1. + tk.MustExec("insert into t values(1, 1)") + r = tk.MustQuery("admin show t next_row_id") + r.Check(testkit.Rows("test t _tidb_rowid 11")) + // Row ID is original + step. + for i := 0; i < int(step); i++ { + tk.MustExec("insert into t values(10000, 1)") + } + r = tk.MustQuery("admin show t next_row_id") + r.Check(testkit.Rows("test t _tidb_rowid 21")) + + // test for a table with the primary key + tk.MustExec("create table tt(id int primary key auto_increment, c int)") + // Start handle is 1. + r = tk.MustQuery("admin show tt next_row_id") + r.Check(testkit.Rows("test tt id 1")) + // After rebasing auto ID, row ID is 20 + step + 1. + tk.MustExec("insert into tt values(20, 1)") + r = tk.MustQuery("admin show tt next_row_id") + r.Check(testkit.Rows("test tt id 31")) + // test for renaming the table + tk.MustExec("create database test1") + tk.MustExec("rename table test.tt to test1.tt") + tk.MustExec("use test1") + r = tk.MustQuery("admin show tt next_row_id") + r.Check(testkit.Rows("test1 tt id 31")) + tk.MustExec("insert test1.tt values ()") + r = tk.MustQuery("admin show tt next_row_id") + r.Check(testkit.Rows("test1 tt id 41")) +} diff --git a/executor/aggregate_test.go b/executor/aggregate_test.go index 48b6da9e62bbb..ae668d8031021 100644 --- a/executor/aggregate_test.go +++ b/executor/aggregate_test.go @@ -239,8 +239,8 @@ func (s *testSuite) TestAggregation(c *C) { result.Check(testkit.Rows("", "")) result = tk.MustQuery("select count(*) from information_schema.columns") - // When adding new memory columns in information_schema, please update this variable.\ - columnCountOfAllInformationSchemaTables := "754" + // When adding new memory columns in information_schema, please update this variable. + columnCountOfAllInformationSchemaTables := "756" result.Check(testkit.Rows(columnCountOfAllInformationSchemaTables)) tk.MustExec("drop table if exists t1") diff --git a/executor/builder.go b/executor/builder.go index 32daead6b62f8..f6bb03ba971a3 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -104,6 +104,8 @@ func (b *executorBuilder) build(p plan.Plan) Executor { return b.buildSelectLock(v) case *plan.CancelDDLJobs: return b.buildCancelDDLJobs(v) + case *plan.ShowNextRowID: + return b.buildShowNextRowID(v) case *plan.ShowDDL: return b.buildShowDDL(v) case *plan.ShowDDLJobs: @@ -177,6 +179,14 @@ func (b *executorBuilder) buildCancelDDLJobs(v *plan.CancelDDLJobs) Executor { return e } +func (b *executorBuilder) buildShowNextRowID(v *plan.ShowNextRowID) Executor { + e := &ShowNextRowIDExec{ + baseExecutor: newBaseExecutor(b.ctx, v.Schema(), v.ExplainID()), + tblName: v.TableName, + } + return e +} + func (b *executorBuilder) buildShowDDL(v *plan.ShowDDL) Executor { // We get DDLInfo here because for Executors that returns result set, // next will be called after transaction has been committed. diff --git a/executor/executor.go b/executor/executor.go index eec3e876d6c9e..8aa8e4f5539eb 100644 --- a/executor/executor.go +++ b/executor/executor.go @@ -22,6 +22,7 @@ import ( "github.com/cznic/mathutil" "github.com/juju/errors" "github.com/pingcap/tidb/ast" + "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/kv" @@ -49,6 +50,7 @@ var ( _ Executor = &ProjectionExec{} _ Executor = &SelectionExec{} _ Executor = &SelectLockExec{} + _ Executor = &ShowNextRowIDExec{} _ Executor = &ShowDDLExec{} _ Executor = &ShowDDLJobsExec{} _ Executor = &ShowDDLJobQueriesExec{} @@ -221,6 +223,43 @@ func (e *CancelDDLJobsExec) Next(ctx context.Context, chk *chunk.Chunk) error { return nil } +// ShowNextRowIDExec represents a show the next row ID executor. +type ShowNextRowIDExec struct { + baseExecutor + tblName *ast.TableName + done bool +} + +// Next implements the Executor Next interface. +func (e *ShowNextRowIDExec) Next(ctx context.Context, chk *chunk.Chunk) error { + chk.Reset() + if e.done { + return nil + } + is := domain.GetDomain(e.ctx).InfoSchema() + tbl, err := is.TableByName(e.tblName.Schema, e.tblName.Name) + if err != nil { + return errors.Trace(err) + } + colName := model.ExtraHandleName + for _, col := range tbl.Meta().Columns { + if mysql.HasAutoIncrementFlag(col.Flag) { + colName = col.Name + break + } + } + nextGlobalID, err := tbl.Allocator(e.ctx).NextGlobalAutoID(tbl.Meta().ID) + if err != nil { + return errors.Trace(err) + } + chk.AppendString(0, e.tblName.Schema.O) + chk.AppendString(1, e.tblName.Name.O) + chk.AppendString(2, colName.O) + chk.AppendInt64(3, nextGlobalID) + e.done = true + return nil +} + // ShowDDLExec represents a show DDL executor. type ShowDDLExec struct { baseExecutor diff --git a/parser/misc.go b/parser/misc.go index f3b66d7708396..303b5013bdaae 100644 --- a/parser/misc.go +++ b/parser/misc.go @@ -352,6 +352,7 @@ var tokenMap = map[string]int{ "NAMES": names, "NATIONAL": national, "NATURAL": natural, + "NEXT_ROW_ID": nextRowID, "NO": no, "NO_WRITE_TO_BINLOG": noWriteToBinLog, "NONE": none, diff --git a/parser/parser.y b/parser/parser.y index 82be45e7a94a5..ab55e60cb51b6 100644 --- a/parser/parser.y +++ b/parser/parser.y @@ -396,30 +396,31 @@ import ( yearType "YEAR" /* The following tokens belong to NotKeywordToken. */ - addDate "ADDDATE" - bitAnd "BIT_AND" - bitOr "BIT_OR" - bitXor "BIT_XOR" - cast "CAST" - copyKwd "COPY" - count "COUNT" - curTime "CURTIME" - dateAdd "DATE_ADD" - dateSub "DATE_SUB" - extract "EXTRACT" - getFormat "GET_FORMAT" - groupConcat "GROUP_CONCAT" - inplace "INPLACE" - min "MIN" - max "MAX" - now "NOW" - position "POSITION" - subDate "SUBDATE" - sum "SUM" - substring "SUBSTRING" - timestampAdd "TIMESTAMPADD" - timestampDiff "TIMESTAMPDIFF" - trim "TRIM" + addDate "ADDDATE" + bitAnd "BIT_AND" + bitOr "BIT_OR" + bitXor "BIT_XOR" + cast "CAST" + copyKwd "COPY" + count "COUNT" + curTime "CURTIME" + dateAdd "DATE_ADD" + dateSub "DATE_SUB" + extract "EXTRACT" + getFormat "GET_FORMAT" + groupConcat "GROUP_CONCAT" + nextRowID "NEXT_ROW_ID" + inplace "INPLACE" + min "MIN" + max "MAX" + now "NOW" + position "POSITION" + subDate "SUBDATE" + sum "SUM" + substring "SUBSTRING" + timestampAdd "TIMESTAMPADD" + timestampDiff "TIMESTAMPDIFF" + trim "TRIM" /* The following tokens belong to TiDBKeyword. */ admin "ADMIN" @@ -2629,7 +2630,7 @@ TiDBKeyword: NotKeywordToken: "ADDDATE" | "BIT_AND" | "BIT_OR" | "BIT_XOR" | "CAST" | "COPY" | "COUNT" | "CURTIME" | "DATE_ADD" | "DATE_SUB" | "EXTRACT" | "GET_FORMAT" | "GROUP_CONCAT" -| "INPLACE" |"MIN" | "MAX" | "NOW" | "POSITION" | "SUBDATE" | "SUBSTRING" | "SUM" | "TIMESTAMPADD" | "TIMESTAMPDIFF" | "TRIM" +| "INPLACE" |"MIN" | "MAX" | "NOW" | "POSITION" | "SUBDATE" | "SUBSTRING" | "SUM" | "TIMESTAMPADD" | "TIMESTAMPDIFF" | "TRIM" | "NEXT_ROW_ID" /************************************************************************************ * @@ -4924,6 +4925,13 @@ AdminStmt: { $$ = &ast.AdminStmt{Tp: ast.AdminShowDDLJobs} } +| "ADMIN" "SHOW" TableName "NEXT_ROW_ID" + { + $$ = &ast.AdminStmt{ + Tp: ast.AdminShowNextRowID, + Tables: []*ast.TableName{$3.(*ast.TableName)}, + } + } | "ADMIN" "CHECK" "TABLE" TableNameList { $$ = &ast.AdminStmt{ diff --git a/parser/parser_test.go b/parser/parser_test.go index 103a8179331a2..e3e5cddd4d093 100644 --- a/parser/parser_test.go +++ b/parser/parser_test.go @@ -422,6 +422,7 @@ func (s *testParserSuite) TestDMLStmt(c *C) { {"admin checksum table t1, t2;", true}, {"admin cancel ddl jobs 1", true}, {"admin cancel ddl jobs 1, 2", true}, + {"admin show t1 next_row_id", true}, {"admin recover index t1 idx_a", true}, {"admin cleanup index t1 idx_a", true}, diff --git a/plan/common_plans.go b/plan/common_plans.go index e25c68ad4b889..1cc22401596fa 100644 --- a/plan/common_plans.go +++ b/plan/common_plans.go @@ -50,6 +50,12 @@ type ShowDDLJobQueries struct { JobIDs []int64 } +// ShowNextRowID is for showing the next global row ID. +type ShowNextRowID struct { + baseSchemaProducer + TableName *ast.TableName +} + // CheckTable is used for checking table data, built from the 'admin check table' statement. type CheckTable struct { baseSchemaProducer diff --git a/plan/planbuilder.go b/plan/planbuilder.go index d5302f93596ec..b17adfaeab1b7 100644 --- a/plan/planbuilder.go +++ b/plan/planbuilder.go @@ -475,6 +475,10 @@ func (b *planBuilder) buildAdmin(as *ast.AdminStmt) Plan { p := &ChecksumTable{Tables: as.Tables} p.SetSchema(buildChecksumTableSchema()) ret = p + case ast.AdminShowNextRowID: + p := &ShowNextRowID{TableName: as.Tables[0]} + p.SetSchema(buildShowNextRowID()) + ret = p case ast.AdminShowDDL: p := &ShowDDL{} p.SetSchema(buildShowDDLFields()) @@ -628,6 +632,15 @@ func (b *planBuilder) buildAnalyze(as *ast.AnalyzeTableStmt) Plan { return b.buildAnalyzeTable(as) } +func buildShowNextRowID() *expression.Schema { + schema := expression.NewSchema(make([]*expression.Column, 0, 4)...) + schema.Append(buildColumn("", "DB_NAME", mysql.TypeVarchar, mysql.MaxDatabaseNameLength)) + schema.Append(buildColumn("", "TABLE_NAME", mysql.TypeVarchar, mysql.MaxTableNameLength)) + schema.Append(buildColumn("", "COLUMN_NAME", mysql.TypeVarchar, mysql.MaxColumnNameLength)) + schema.Append(buildColumn("", "NEXT_GLOBAL_ROW_ID", mysql.TypeLonglong, 4)) + return schema +} + func buildShowDDLFields() *expression.Schema { schema := expression.NewSchema(make([]*expression.Column, 0, 4)...) schema.Append(buildColumn("", "SCHEMA_VER", mysql.TypeLonglong, 4))