Skip to content

Commit

Permalink
*: support for "admin show next_row_id" (#8268) (#8274)
Browse files Browse the repository at this point in the history
  • Loading branch information
zimulala authored and zz-jason committed Nov 14, 2018
1 parent 691e795 commit ff0c436
Show file tree
Hide file tree
Showing 10 changed files with 150 additions and 27 deletions.
2 changes: 2 additions & 0 deletions ast/misc.go
Original file line number Diff line number Diff line change
Expand Up @@ -595,6 +595,8 @@ const (
AdminCheckIndexRange
AdminShowDDLJobQueries
AdminChecksumTable
AdminShowSlow
AdminShowNextRowID
)

// HandleRange represents a range where handle value >= Begin and < End.
Expand Down
43 changes: 43 additions & 0 deletions executor/admin_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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"
Expand Down Expand Up @@ -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"))
}
4 changes: 2 additions & 2 deletions executor/aggregate_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -239,8 +239,8 @@ func (s *testSuite) TestAggregation(c *C) {
result.Check(testkit.Rows("<nil>", "<nil>"))

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")
Expand Down
10 changes: 10 additions & 0 deletions executor/builder.go
Original file line number Diff line number Diff line change
Expand Up @@ -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:
Expand Down Expand Up @@ -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.
Expand Down
39 changes: 39 additions & 0 deletions executor/executor.go
Original file line number Diff line number Diff line change
Expand Up @@ -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"
Expand Down Expand Up @@ -49,6 +50,7 @@ var (
_ Executor = &ProjectionExec{}
_ Executor = &SelectionExec{}
_ Executor = &SelectLockExec{}
_ Executor = &ShowNextRowIDExec{}
_ Executor = &ShowDDLExec{}
_ Executor = &ShowDDLJobsExec{}
_ Executor = &ShowDDLJobQueriesExec{}
Expand Down Expand Up @@ -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
Expand Down
1 change: 1 addition & 0 deletions parser/misc.go
Original file line number Diff line number Diff line change
Expand Up @@ -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,
Expand Down
58 changes: 33 additions & 25 deletions parser/parser.y
Original file line number Diff line number Diff line change
Expand Up @@ -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"
Expand Down Expand Up @@ -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"

/************************************************************************************
*
Expand Down Expand Up @@ -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{
Expand Down
1 change: 1 addition & 0 deletions parser/parser_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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},

Expand Down
6 changes: 6 additions & 0 deletions plan/common_plans.go
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down
13 changes: 13 additions & 0 deletions plan/planbuilder.go
Original file line number Diff line number Diff line change
Expand Up @@ -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())
Expand Down Expand Up @@ -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))
Expand Down

0 comments on commit ff0c436

Please sign in to comment.