Skip to content

Commit ff0c436

Browse files
zimulalazz-jason
authored andcommitted
*: support for "admin show next_row_id" (#8268) (#8274)
1 parent 691e795 commit ff0c436

File tree

10 files changed

+150
-27
lines changed

10 files changed

+150
-27
lines changed

ast/misc.go

Lines changed: 2 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -595,6 +595,8 @@ const (
595595
AdminCheckIndexRange
596596
AdminShowDDLJobQueries
597597
AdminChecksumTable
598+
AdminShowSlow
599+
AdminShowNextRowID
598600
)
599601

600602
// HandleRange represents a range where handle value >= Begin and < End.

executor/admin_test.go

Lines changed: 43 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -15,6 +15,7 @@ package executor_test
1515

1616
import (
1717
. "github.com/pingcap/check"
18+
"github.com/pingcap/tidb/meta/autoid"
1819
"github.com/pingcap/tidb/model"
1920
"github.com/pingcap/tidb/table/tables"
2021
"github.com/pingcap/tidb/types"
@@ -458,3 +459,45 @@ func (s *testSuite) TestAdminCheckTable(c *C) {
458459
tk.MustExec(`ALTER TABLE t1 ADD INDEX idx6 (c6)`)
459460
tk.MustExec(`admin check table t1`)
460461
}
462+
463+
func (s *testSuite) TestAdminShowNextID(c *C) {
464+
step := int64(10)
465+
autoIDStep := autoid.GetStep()
466+
autoid.SetStep(step)
467+
defer autoid.SetStep(autoIDStep)
468+
tk := testkit.NewTestKit(c, s.store)
469+
tk.MustExec("use test")
470+
tk.MustExec("create table t(id int, c int)")
471+
// Start handle is 1.
472+
r := tk.MustQuery("admin show t next_row_id")
473+
r.Check(testkit.Rows("test t _tidb_rowid 1"))
474+
// Row ID is step + 1.
475+
tk.MustExec("insert into t values(1, 1)")
476+
r = tk.MustQuery("admin show t next_row_id")
477+
r.Check(testkit.Rows("test t _tidb_rowid 11"))
478+
// Row ID is original + step.
479+
for i := 0; i < int(step); i++ {
480+
tk.MustExec("insert into t values(10000, 1)")
481+
}
482+
r = tk.MustQuery("admin show t next_row_id")
483+
r.Check(testkit.Rows("test t _tidb_rowid 21"))
484+
485+
// test for a table with the primary key
486+
tk.MustExec("create table tt(id int primary key auto_increment, c int)")
487+
// Start handle is 1.
488+
r = tk.MustQuery("admin show tt next_row_id")
489+
r.Check(testkit.Rows("test tt id 1"))
490+
// After rebasing auto ID, row ID is 20 + step + 1.
491+
tk.MustExec("insert into tt values(20, 1)")
492+
r = tk.MustQuery("admin show tt next_row_id")
493+
r.Check(testkit.Rows("test tt id 31"))
494+
// test for renaming the table
495+
tk.MustExec("create database test1")
496+
tk.MustExec("rename table test.tt to test1.tt")
497+
tk.MustExec("use test1")
498+
r = tk.MustQuery("admin show tt next_row_id")
499+
r.Check(testkit.Rows("test1 tt id 31"))
500+
tk.MustExec("insert test1.tt values ()")
501+
r = tk.MustQuery("admin show tt next_row_id")
502+
r.Check(testkit.Rows("test1 tt id 41"))
503+
}

executor/aggregate_test.go

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -239,8 +239,8 @@ func (s *testSuite) TestAggregation(c *C) {
239239
result.Check(testkit.Rows("<nil>", "<nil>"))
240240

241241
result = tk.MustQuery("select count(*) from information_schema.columns")
242-
// When adding new memory columns in information_schema, please update this variable.\
243-
columnCountOfAllInformationSchemaTables := "754"
242+
// When adding new memory columns in information_schema, please update this variable.
243+
columnCountOfAllInformationSchemaTables := "756"
244244
result.Check(testkit.Rows(columnCountOfAllInformationSchemaTables))
245245

246246
tk.MustExec("drop table if exists t1")

executor/builder.go

Lines changed: 10 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -104,6 +104,8 @@ func (b *executorBuilder) build(p plan.Plan) Executor {
104104
return b.buildSelectLock(v)
105105
case *plan.CancelDDLJobs:
106106
return b.buildCancelDDLJobs(v)
107+
case *plan.ShowNextRowID:
108+
return b.buildShowNextRowID(v)
107109
case *plan.ShowDDL:
108110
return b.buildShowDDL(v)
109111
case *plan.ShowDDLJobs:
@@ -177,6 +179,14 @@ func (b *executorBuilder) buildCancelDDLJobs(v *plan.CancelDDLJobs) Executor {
177179
return e
178180
}
179181

182+
func (b *executorBuilder) buildShowNextRowID(v *plan.ShowNextRowID) Executor {
183+
e := &ShowNextRowIDExec{
184+
baseExecutor: newBaseExecutor(b.ctx, v.Schema(), v.ExplainID()),
185+
tblName: v.TableName,
186+
}
187+
return e
188+
}
189+
180190
func (b *executorBuilder) buildShowDDL(v *plan.ShowDDL) Executor {
181191
// We get DDLInfo here because for Executors that returns result set,
182192
// next will be called after transaction has been committed.

executor/executor.go

Lines changed: 39 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -22,6 +22,7 @@ import (
2222
"github.com/cznic/mathutil"
2323
"github.com/juju/errors"
2424
"github.com/pingcap/tidb/ast"
25+
"github.com/pingcap/tidb/domain"
2526
"github.com/pingcap/tidb/expression"
2627
"github.com/pingcap/tidb/infoschema"
2728
"github.com/pingcap/tidb/kv"
@@ -49,6 +50,7 @@ var (
4950
_ Executor = &ProjectionExec{}
5051
_ Executor = &SelectionExec{}
5152
_ Executor = &SelectLockExec{}
53+
_ Executor = &ShowNextRowIDExec{}
5254
_ Executor = &ShowDDLExec{}
5355
_ Executor = &ShowDDLJobsExec{}
5456
_ Executor = &ShowDDLJobQueriesExec{}
@@ -221,6 +223,43 @@ func (e *CancelDDLJobsExec) Next(ctx context.Context, chk *chunk.Chunk) error {
221223
return nil
222224
}
223225

226+
// ShowNextRowIDExec represents a show the next row ID executor.
227+
type ShowNextRowIDExec struct {
228+
baseExecutor
229+
tblName *ast.TableName
230+
done bool
231+
}
232+
233+
// Next implements the Executor Next interface.
234+
func (e *ShowNextRowIDExec) Next(ctx context.Context, chk *chunk.Chunk) error {
235+
chk.Reset()
236+
if e.done {
237+
return nil
238+
}
239+
is := domain.GetDomain(e.ctx).InfoSchema()
240+
tbl, err := is.TableByName(e.tblName.Schema, e.tblName.Name)
241+
if err != nil {
242+
return errors.Trace(err)
243+
}
244+
colName := model.ExtraHandleName
245+
for _, col := range tbl.Meta().Columns {
246+
if mysql.HasAutoIncrementFlag(col.Flag) {
247+
colName = col.Name
248+
break
249+
}
250+
}
251+
nextGlobalID, err := tbl.Allocator(e.ctx).NextGlobalAutoID(tbl.Meta().ID)
252+
if err != nil {
253+
return errors.Trace(err)
254+
}
255+
chk.AppendString(0, e.tblName.Schema.O)
256+
chk.AppendString(1, e.tblName.Name.O)
257+
chk.AppendString(2, colName.O)
258+
chk.AppendInt64(3, nextGlobalID)
259+
e.done = true
260+
return nil
261+
}
262+
224263
// ShowDDLExec represents a show DDL executor.
225264
type ShowDDLExec struct {
226265
baseExecutor

parser/misc.go

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -352,6 +352,7 @@ var tokenMap = map[string]int{
352352
"NAMES": names,
353353
"NATIONAL": national,
354354
"NATURAL": natural,
355+
"NEXT_ROW_ID": nextRowID,
355356
"NO": no,
356357
"NO_WRITE_TO_BINLOG": noWriteToBinLog,
357358
"NONE": none,

parser/parser.y

Lines changed: 33 additions & 25 deletions
Original file line numberDiff line numberDiff line change
@@ -396,30 +396,31 @@ import (
396396
yearType "YEAR"
397397

398398
/* The following tokens belong to NotKeywordToken. */
399-
addDate "ADDDATE"
400-
bitAnd "BIT_AND"
401-
bitOr "BIT_OR"
402-
bitXor "BIT_XOR"
403-
cast "CAST"
404-
copyKwd "COPY"
405-
count "COUNT"
406-
curTime "CURTIME"
407-
dateAdd "DATE_ADD"
408-
dateSub "DATE_SUB"
409-
extract "EXTRACT"
410-
getFormat "GET_FORMAT"
411-
groupConcat "GROUP_CONCAT"
412-
inplace "INPLACE"
413-
min "MIN"
414-
max "MAX"
415-
now "NOW"
416-
position "POSITION"
417-
subDate "SUBDATE"
418-
sum "SUM"
419-
substring "SUBSTRING"
420-
timestampAdd "TIMESTAMPADD"
421-
timestampDiff "TIMESTAMPDIFF"
422-
trim "TRIM"
399+
addDate "ADDDATE"
400+
bitAnd "BIT_AND"
401+
bitOr "BIT_OR"
402+
bitXor "BIT_XOR"
403+
cast "CAST"
404+
copyKwd "COPY"
405+
count "COUNT"
406+
curTime "CURTIME"
407+
dateAdd "DATE_ADD"
408+
dateSub "DATE_SUB"
409+
extract "EXTRACT"
410+
getFormat "GET_FORMAT"
411+
groupConcat "GROUP_CONCAT"
412+
nextRowID "NEXT_ROW_ID"
413+
inplace "INPLACE"
414+
min "MIN"
415+
max "MAX"
416+
now "NOW"
417+
position "POSITION"
418+
subDate "SUBDATE"
419+
sum "SUM"
420+
substring "SUBSTRING"
421+
timestampAdd "TIMESTAMPADD"
422+
timestampDiff "TIMESTAMPDIFF"
423+
trim "TRIM"
423424

424425
/* The following tokens belong to TiDBKeyword. */
425426
admin "ADMIN"
@@ -2629,7 +2630,7 @@ TiDBKeyword:
26292630

26302631
NotKeywordToken:
26312632
"ADDDATE" | "BIT_AND" | "BIT_OR" | "BIT_XOR" | "CAST" | "COPY" | "COUNT" | "CURTIME" | "DATE_ADD" | "DATE_SUB" | "EXTRACT" | "GET_FORMAT" | "GROUP_CONCAT"
2632-
| "INPLACE" |"MIN" | "MAX" | "NOW" | "POSITION" | "SUBDATE" | "SUBSTRING" | "SUM" | "TIMESTAMPADD" | "TIMESTAMPDIFF" | "TRIM"
2633+
| "INPLACE" |"MIN" | "MAX" | "NOW" | "POSITION" | "SUBDATE" | "SUBSTRING" | "SUM" | "TIMESTAMPADD" | "TIMESTAMPDIFF" | "TRIM" | "NEXT_ROW_ID"
26332634

26342635
/************************************************************************************
26352636
*
@@ -4924,6 +4925,13 @@ AdminStmt:
49244925
{
49254926
$$ = &ast.AdminStmt{Tp: ast.AdminShowDDLJobs}
49264927
}
4928+
| "ADMIN" "SHOW" TableName "NEXT_ROW_ID"
4929+
{
4930+
$$ = &ast.AdminStmt{
4931+
Tp: ast.AdminShowNextRowID,
4932+
Tables: []*ast.TableName{$3.(*ast.TableName)},
4933+
}
4934+
}
49274935
| "ADMIN" "CHECK" "TABLE" TableNameList
49284936
{
49294937
$$ = &ast.AdminStmt{

parser/parser_test.go

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -422,6 +422,7 @@ func (s *testParserSuite) TestDMLStmt(c *C) {
422422
{"admin checksum table t1, t2;", true},
423423
{"admin cancel ddl jobs 1", true},
424424
{"admin cancel ddl jobs 1, 2", true},
425+
{"admin show t1 next_row_id", true},
425426
{"admin recover index t1 idx_a", true},
426427
{"admin cleanup index t1 idx_a", true},
427428

plan/common_plans.go

Lines changed: 6 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -50,6 +50,12 @@ type ShowDDLJobQueries struct {
5050
JobIDs []int64
5151
}
5252

53+
// ShowNextRowID is for showing the next global row ID.
54+
type ShowNextRowID struct {
55+
baseSchemaProducer
56+
TableName *ast.TableName
57+
}
58+
5359
// CheckTable is used for checking table data, built from the 'admin check table' statement.
5460
type CheckTable struct {
5561
baseSchemaProducer

plan/planbuilder.go

Lines changed: 13 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -475,6 +475,10 @@ func (b *planBuilder) buildAdmin(as *ast.AdminStmt) Plan {
475475
p := &ChecksumTable{Tables: as.Tables}
476476
p.SetSchema(buildChecksumTableSchema())
477477
ret = p
478+
case ast.AdminShowNextRowID:
479+
p := &ShowNextRowID{TableName: as.Tables[0]}
480+
p.SetSchema(buildShowNextRowID())
481+
ret = p
478482
case ast.AdminShowDDL:
479483
p := &ShowDDL{}
480484
p.SetSchema(buildShowDDLFields())
@@ -628,6 +632,15 @@ func (b *planBuilder) buildAnalyze(as *ast.AnalyzeTableStmt) Plan {
628632
return b.buildAnalyzeTable(as)
629633
}
630634

635+
func buildShowNextRowID() *expression.Schema {
636+
schema := expression.NewSchema(make([]*expression.Column, 0, 4)...)
637+
schema.Append(buildColumn("", "DB_NAME", mysql.TypeVarchar, mysql.MaxDatabaseNameLength))
638+
schema.Append(buildColumn("", "TABLE_NAME", mysql.TypeVarchar, mysql.MaxTableNameLength))
639+
schema.Append(buildColumn("", "COLUMN_NAME", mysql.TypeVarchar, mysql.MaxColumnNameLength))
640+
schema.Append(buildColumn("", "NEXT_GLOBAL_ROW_ID", mysql.TypeLonglong, 4))
641+
return schema
642+
}
643+
631644
func buildShowDDLFields() *expression.Schema {
632645
schema := expression.NewSchema(make([]*expression.Column, 0, 4)...)
633646
schema.Append(buildColumn("", "SCHEMA_VER", mysql.TypeLonglong, 4))

0 commit comments

Comments
 (0)