diff --git a/ast/misc.go b/ast/misc.go index c4493a24ccd66..e9e20b162e0f0 100644 --- a/ast/misc.go +++ b/ast/misc.go @@ -628,6 +628,7 @@ const ( 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 0a1282c582388..fa3ed027cb605 100644 --- a/executor/admin_test.go +++ b/executor/admin_test.go @@ -18,6 +18,7 @@ import ( . "github.com/pingcap/check" "github.com/pingcap/tidb/executor" + "github.com/pingcap/tidb/meta/autoid" "github.com/pingcap/tidb/model" "github.com/pingcap/tidb/table/tables" "github.com/pingcap/tidb/types" @@ -515,3 +516,45 @@ func (s *testSuite) TestAdminCheckPrimaryIndex(c *C) { tk.MustExec("insert into t values(1, 1, 1), (9223372036854775807, 2, 2);") tk.MustExec("admin check index t idx;") } + +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 9b2d861907bcf..f8a098097575c 100644 --- a/executor/aggregate_test.go +++ b/executor/aggregate_test.go @@ -239,7 +239,7 @@ func (s *testSuite) TestAggregation(c *C) { result = tk.MustQuery("select count(*) from information_schema.columns") // When adding new memory columns in information_schema, please update this variable. - columnCountOfAllInformationSchemaTables := "757" + columnCountOfAllInformationSchemaTables := "759" result.Check(testkit.Rows(columnCountOfAllInformationSchemaTables)) tk.MustExec("drop table if exists t1") diff --git a/executor/builder.go b/executor/builder.go index 68af3a6bfd888..2af348a383579 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -112,6 +112,8 @@ func (b *executorBuilder) build(p plannercore.Plan) Executor { return b.buildSelectLock(v) case *plannercore.CancelDDLJobs: return b.buildCancelDDLJobs(v) + case *plannercore.ShowNextRowID: + return b.buildShowNextRowID(v) case *plannercore.ShowDDL: return b.buildShowDDL(v) case *plannercore.ShowDDLJobs: @@ -185,6 +187,14 @@ func (b *executorBuilder) buildCancelDDLJobs(v *plannercore.CancelDDLJobs) Execu return e } +func (b *executorBuilder) buildShowNextRowID(v *plannercore.ShowNextRowID) Executor { + e := &ShowNextRowIDExec{ + baseExecutor: newBaseExecutor(b.ctx, v.Schema(), v.ExplainID()), + tblName: v.TableName, + } + return e +} + func (b *executorBuilder) buildShowDDL(v *plannercore.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 2ea2381db9ec8..0d97743e7b93c 100644 --- a/executor/executor.go +++ b/executor/executor.go @@ -55,6 +55,7 @@ var ( _ Executor = &ProjectionExec{} _ Executor = &SelectionExec{} _ Executor = &SelectLockExec{} + _ Executor = &ShowNextRowIDExec{} _ Executor = &ShowDDLExec{} _ Executor = &ShowDDLJobsExec{} _ Executor = &ShowDDLJobQueriesExec{} @@ -200,6 +201,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 51c623016dd05..4190932fb75f7 100644 --- a/parser/misc.go +++ b/parser/misc.go @@ -355,6 +355,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 a94c9259c9e58..6326d2cb30028 100644 --- a/parser/parser.y +++ b/parser/parser.y @@ -420,6 +420,7 @@ import ( extract "EXTRACT" getFormat "GET_FORMAT" groupConcat "GROUP_CONCAT" + nextRowID "NEXT_ROW_ID" inplace "INPLACE" internal "INTERNAL" min "MIN" @@ -2838,7 +2839,7 @@ TiDBKeyword: NotKeywordToken: "ADDDATE" | "BIT_AND" | "BIT_OR" | "BIT_XOR" | "CAST" | "COPY" | "COUNT" | "CURTIME" | "DATE_ADD" | "DATE_SUB" | "EXTRACT" | "GET_FORMAT" | "GROUP_CONCAT" | "INPLACE" | "INTERNAL" -|"MIN" | "MAX" | "MAX_EXECUTION_TIME" | "NOW" | "RECENT" | "POSITION" | "SUBDATE" | "SUBSTRING" | "SUM" | "TIMESTAMPADD" | "TIMESTAMPDIFF" | "TOP" | "TRIM" +|"MIN" | "MAX" | "MAX_EXECUTION_TIME" | "NOW" | "RECENT" | "POSITION" | "SUBDATE" | "SUBSTRING" | "SUM" | "TIMESTAMPADD" | "TIMESTAMPDIFF" | "TOP" | "TRIM" | "NEXT_ROW_ID" /************************************************************************************ * @@ -5173,6 +5174,13 @@ AdminStmt: JobNumber: $5.(int64), } } +| "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 15370afa33d93..af7a8ba4f2bbc 100644 --- a/parser/parser_test.go +++ b/parser/parser_test.go @@ -439,6 +439,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}, {"admin show slow top 3", true}, diff --git a/planner/core/common_plans.go b/planner/core/common_plans.go index 6a4db7fe3795b..7f750c67cb260 100644 --- a/planner/core/common_plans.go +++ b/planner/core/common_plans.go @@ -60,6 +60,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/planner/core/planbuilder.go b/planner/core/planbuilder.go index f218ecf944a7d..3601e4b712532 100644 --- a/planner/core/planbuilder.go +++ b/planner/core/planbuilder.go @@ -476,6 +476,10 @@ func (b *planBuilder) buildAdmin(as *ast.AdminStmt) (Plan, error) { 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()) @@ -706,6 +710,15 @@ func (b *planBuilder) buildAnalyze(as *ast.AnalyzeTableStmt) (Plan, error) { return b.buildAnalyzeTable(as), nil } +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))