From cf164cf4b64e3ae7dbe2afc48580656ecc2d27c9 Mon Sep 17 00:00:00 2001 From: tangenta Date: Thu, 29 Oct 2020 13:25:14 +0800 Subject: [PATCH 1/7] *: separate auto_increment ID allocator from _tidb_rowid allocator --- ddl/db_integration_test.go | 12 +++-- ddl/db_test.go | 10 ++-- ddl/ddl.go | 1 + ddl/ddl_api.go | 20 +++++--- ddl/ddl_worker.go | 2 +- ddl/table.go | 50 +++++++----------- errno/errcode.go | 1 + errno/errname.go | 1 + errors.toml | 5 ++ executor/ddl.go | 21 ++++---- executor/executor.go | 12 +++-- executor/executor_test.go | 5 +- executor/infoschema_reader.go | 2 +- executor/insert_common.go | 4 +- executor/insert_test.go | 4 +- executor/seqtest/seq_executor_test.go | 14 ++--- executor/show.go | 2 +- executor/write.go | 2 +- infoschema/builder.go | 11 +++- meta/autoid/autoid.go | 26 +++++++--- meta/autoid/errors.go | 1 + meta/meta.go | 73 +++++++++++++++++++-------- meta/meta_test.go | 2 +- session/session_test.go | 22 +++----- table/table.go | 4 +- table/tables/tables_test.go | 12 +---- 26 files changed, 177 insertions(+), 142 deletions(-) diff --git a/ddl/db_integration_test.go b/ddl/db_integration_test.go index 1ec26a3b2daa2..9ed4be975ac82 100644 --- a/ddl/db_integration_test.go +++ b/ddl/db_integration_test.go @@ -2321,13 +2321,13 @@ func (s *testIntegrationSuite3) TestCreateTableWithAutoIdCache(c *C) { c.Assert(err, IsNil) tk.MustExec("insert into t(b) values(NULL)") - tk.MustQuery("select b, _tidb_rowid from t").Check(testkit.Rows("1 2")) + tk.MustQuery("select b, _tidb_rowid from t").Check(testkit.Rows("1 1")) tk.MustExec("delete from t") // Invalid the allocator cache, insert will trigger a new cache. tk.MustExec("rename table t to t1;") tk.MustExec("insert into t1(b) values(NULL)") - tk.MustQuery("select b, _tidb_rowid from t1").Check(testkit.Rows("101 102")) + tk.MustQuery("select b, _tidb_rowid from t1").Check(testkit.Rows("101 101")) tk.MustExec("delete from t1") // Test alter auto_id_cache. @@ -2337,13 +2337,13 @@ func (s *testIntegrationSuite3) TestCreateTableWithAutoIdCache(c *C) { c.Assert(tblInfo.Meta().AutoIdCache, Equals, int64(200)) tk.MustExec("insert into t1(b) values(NULL)") - tk.MustQuery("select b, _tidb_rowid from t1").Check(testkit.Rows("201 202")) + tk.MustQuery("select b, _tidb_rowid from t1").Check(testkit.Rows("201 201")) tk.MustExec("delete from t1") // Invalid the allocator cache, insert will trigger a new cache. tk.MustExec("rename table t1 to t;") tk.MustExec("insert into t(b) values(NULL)") - tk.MustQuery("select b, _tidb_rowid from t").Check(testkit.Rows("401 402")) + tk.MustQuery("select b, _tidb_rowid from t").Check(testkit.Rows("401 401")) tk.MustExec("delete from t") tk.MustExec("drop table if exists t;") @@ -2354,7 +2354,9 @@ func (s *testIntegrationSuite3) TestCreateTableWithAutoIdCache(c *C) { c.Assert(tblInfo.Meta().AutoIdCache, Equals, int64(3)) // Test insert batch size(4 here) greater than the customized autoid step(3 here). - tk.MustExec("insert into t(a) values(NULL),(NULL),(NULL),(NULL)") + // We split them into 2 statements to trigger the cache for the second time. + tk.MustExec("insert into t(a) values(NULL),(NULL),(NULL);") + tk.MustExec("insert into t(a) values(NULL);") tk.MustQuery("select a from t").Check(testkit.Rows("1", "2", "3", "4")) tk.MustExec("delete from t") diff --git a/ddl/db_test.go b/ddl/db_test.go index 0f4065f0f84a4..84a4bf53c68d7 100644 --- a/ddl/db_test.go +++ b/ddl/db_test.go @@ -5122,10 +5122,9 @@ func (s *testSerialDBSuite) TestAlterShardRowIDBits(c *C) { tk.MustExec(fmt.Sprintf("alter table t1 auto_increment = %d;", 1<<56)) tk.MustExec("insert into t1 set a=1;") - // Test increase shard_row_id_bits failed by overflow global auto ID. + // Test rebase auto_increment does not affect shard_row_id_bits. _, err := tk.Exec("alter table t1 SHARD_ROW_ID_BITS = 10;") - c.Assert(err, NotNil) - c.Assert(err.Error(), Equals, "[autoid:1467]shard_row_id_bits 10 will cause next global auto ID 72057594037932936 overflow") + c.Assert(err, IsNil) // Test reduce shard_row_id_bits will be ok. tk.MustExec("alter table t1 SHARD_ROW_ID_BITS = 3;") @@ -5134,7 +5133,7 @@ func (s *testSerialDBSuite) TestAlterShardRowIDBits(c *C) { c.Assert(tbl.Meta().MaxShardRowIDBits == maxShardRowIDBits, IsTrue) c.Assert(tbl.Meta().ShardRowIDBits == shardRowIDBits, IsTrue) } - checkShardRowID(5, 3) + checkShardRowID(10, 3) // Test reduce shard_row_id_bits but calculate overflow should use the max record shard_row_id_bits. tk.MustExec("drop table if exists t1") @@ -5143,8 +5142,7 @@ func (s *testSerialDBSuite) TestAlterShardRowIDBits(c *C) { checkShardRowID(10, 5) tk.MustExec(fmt.Sprintf("alter table t1 auto_increment = %d;", 1<<56)) _, err = tk.Exec("insert into t1 set a=1;") - c.Assert(err, NotNil) - c.Assert(err.Error(), Equals, "[autoid:1467]Failed to read auto-increment value from storage engine") + c.Assert(err, IsNil) } // port from mysql diff --git a/ddl/ddl.go b/ddl/ddl.go index 03890a12e97cc..5bd989baa7bc3 100644 --- a/ddl/ddl.go +++ b/ddl/ddl.go @@ -618,4 +618,5 @@ type RecoverInfo struct { SnapshotTS uint64 CurAutoIncID int64 CurAutoRandID int64 + CurRowID int64 } diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index 60637350c00f0..973947ba93c97 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -1793,7 +1793,7 @@ func (d *ddl) CreateTableWithInfo( // Default tableAutoIncID base is 0. // If the first ID is expected to greater than 1, we need to do rebase. newEnd := tbInfo.AutoIncID - 1 - if err = d.handleAutoIncID(tbInfo, schema.ID, newEnd, autoid.RowIDAllocType); err != nil { + if err = d.handleAutoIncID(tbInfo, schema.ID, newEnd, autoid.AutoIncrementType); err != nil { return errors.Trace(err) } } @@ -1860,8 +1860,8 @@ func (d *ddl) RecoverTable(ctx sessionctx.Context, recoverInfo *RecoverInfo) (er SchemaName: schema.Name.L, Type: model.ActionRecoverTable, BinlogInfo: &model.HistoryInfo{}, - Args: []interface{}{tbInfo, recoverInfo.CurAutoIncID, recoverInfo.DropJobID, - recoverInfo.SnapshotTS, recoverTableCheckFlagNone, recoverInfo.CurAutoRandID}, + Args: []interface{}{tbInfo, recoverInfo.CurRowID, recoverInfo.DropJobID, + recoverInfo.SnapshotTS, recoverTableCheckFlagNone, recoverInfo.CurAutoRandID, recoverInfo.CurAutoIncID}, } err = d.doDDLJob(ctx, job) err = d.callHookOnChanged(err) @@ -2434,7 +2434,7 @@ func (d *ddl) AlterTable(ctx sessionctx.Context, ident ast.Ident, specs []*ast.A } err = d.ShardRowID(ctx, ident, opt.UintValue) case ast.TableOptionAutoIncrement: - err = d.RebaseAutoID(ctx, ident, int64(opt.UintValue), autoid.RowIDAllocType) + err = d.RebaseAutoID(ctx, ident, int64(opt.UintValue), autoid.AutoIncrementType) case ast.TableOptionAutoIdCache: if opt.UintValue > uint64(math.MaxInt64) { // TODO: Refine this error. @@ -2517,7 +2517,7 @@ func (d *ddl) RebaseAutoID(ctx sessionctx.Context, ident ast.Ident, newBase int6 return errors.Trace(ErrInvalidAutoRandom.GenWithStackByArgs(errMsg)) } actionType = model.ActionRebaseAutoRandomBase - case autoid.RowIDAllocType: + case autoid.AutoIncrementType: actionType = model.ActionRebaseAutoID } @@ -2552,15 +2552,21 @@ func (d *ddl) ShardRowID(ctx sessionctx.Context, tableIdent ast.Ident, uVal uint if err != nil { return errors.Trace(err) } - if uVal == t.Meta().ShardRowIDBits { + tblInfo := t.Meta() + if uVal == tblInfo.ShardRowIDBits { // Nothing need to do. return nil } - if uVal > 0 && t.Meta().PKIsHandle { + noRowID := tblInfo.PKIsHandle || tblInfo.IsCommonHandle + if uVal > 0 && noRowID { return errUnsupportedShardRowIDBits } err = verifyNoOverflowShardBits(d.sessPool, t, uVal) if err != nil { + // Ignore the 'shard_row_id_bits' option if there is no _tidb_rowid allocator. + if errors.ErrorEqual(err, autoid.ErrAutoIDAllocatorNotFound) { + return nil + } return err } job := &model.Job{ diff --git a/ddl/ddl_worker.go b/ddl/ddl_worker.go index 6506506158ad4..335e1595faf04 100644 --- a/ddl/ddl_worker.go +++ b/ddl/ddl_worker.go @@ -655,7 +655,7 @@ func (w *worker) runDDLJob(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, case model.ActionTruncateTable: ver, err = onTruncateTable(d, t, job) case model.ActionRebaseAutoID: - ver, err = onRebaseRowIDType(d.store, t, job) + ver, err = onRebaseAutoIncrementIDType(d.store, t, job) case model.ActionRebaseAutoRandomBase: ver, err = onRebaseAutoRandomType(d.store, t, job) case model.ActionRenameTable: diff --git a/ddl/table.go b/ddl/table.go index 37c123cdb1cb3..46a16300aa42b 100644 --- a/ddl/table.go +++ b/ddl/table.go @@ -216,10 +216,10 @@ const ( func (w *worker) onRecoverTable(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, err error) { schemaID := job.SchemaID tblInfo := &model.TableInfo{} - var autoIncID, autoRandID, dropJobID, recoverTableCheckFlag int64 + var rowID, autoIncID, autoRandID, dropJobID, recoverTableCheckFlag int64 var snapshotTS uint64 const checkFlagIndexInJobArgs = 4 // The index of `recoverTableCheckFlag` in job arg list. - if err = job.DecodeArgs(tblInfo, &autoIncID, &dropJobID, &snapshotTS, &recoverTableCheckFlag, &autoRandID); err != nil { + if err = job.DecodeArgs(tblInfo, &rowID, &dropJobID, &snapshotTS, &recoverTableCheckFlag, &autoRandID, &autoIncID); err != nil { // Invalid arguments, cancel this job. job.State = model.JobStateCancelled return ver, errors.Trace(err) @@ -310,7 +310,7 @@ func (w *worker) onRecoverTable(d *ddlCtx, t *meta.Meta, job *model.Job) (ver in tblInfo.State = model.StatePublic tblInfo.UpdateTS = t.StartTS - err = t.CreateTableAndSetAutoID(schemaID, tblInfo, autoIncID, autoRandID) + err = t.CreateTableAndSetAutoID(schemaID, tblInfo, rowID, autoIncID, autoRandID) if err != nil { return ver, errors.Trace(err) } @@ -504,8 +504,8 @@ func onTruncateTable(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, _ erro return ver, nil } -func onRebaseRowIDType(store kv.Storage, t *meta.Meta, job *model.Job) (ver int64, _ error) { - return onRebaseAutoID(store, t, job, autoid.RowIDAllocType) +func onRebaseAutoIncrementIDType(store kv.Storage, t *meta.Meta, job *model.Job) (ver int64, _ error) { + return onRebaseAutoID(store, t, job, autoid.AutoIncrementType) } func onRebaseAutoRandomType(store kv.Storage, t *meta.Meta, job *model.Job) (ver int64, _ error) { @@ -526,7 +526,7 @@ func onRebaseAutoID(store kv.Storage, t *meta.Meta, job *model.Job, tp autoid.Al return ver, errors.Trace(err) } // No need to check `newBase` again, because `RebaseAutoID` will do this check. - if tp == autoid.RowIDAllocType { + if tp == autoid.AutoIncrementType { tblInfo.AutoIncID = newBase } else { tblInfo.AutoRandID = newBase @@ -619,7 +619,11 @@ func verifyNoOverflowShardBits(s *sessionPool, tbl table.Table, shardRowIDBits u defer s.put(ctx) // Check next global max auto ID first. - autoIncID, err := tbl.Allocators(ctx).Get(autoid.RowIDAllocType).NextGlobalAutoID(tbl.Meta().ID) + var alloc autoid.Allocator + if alloc = tbl.Allocators(ctx).Get(autoid.RowIDAllocType); alloc == nil { + return autoid.ErrAutoIDAllocatorNotFound.GenWithStackByArgs("_tidb_rowid") + } + autoIncID, err := alloc.NextGlobalAutoID(tbl.Meta().ID) if err != nil { return errors.Trace(err) } @@ -694,17 +698,10 @@ func checkAndRenameTables(t *meta.Meta, job *model.Job, oldSchemaID int64, newSc return ver, tblInfo, errors.Trace(err) } - var autoTableID int64 - var autoRandID int64 - shouldDelAutoID := false - if newSchemaID != oldSchemaID { - shouldDelAutoID = true - autoTableID, err = t.GetAutoTableID(tblInfo.GetDBID(oldSchemaID), tblInfo.ID) - if err != nil { - job.State = model.JobStateCancelled - return ver, tblInfo, errors.Trace(err) - } - autoRandID, err = t.GetAutoRandomID(tblInfo.GetDBID(oldSchemaID), tblInfo.ID) + var rowID, incID, randID int64 + shouldRecreateAutoID := newSchemaID != oldSchemaID + if shouldRecreateAutoID { + rowID, incID, randID, err = t.GetAllAutoIDs(tblInfo.GetDBID(oldSchemaID), tblInfo.ID) if err != nil { job.State = model.JobStateCancelled return ver, tblInfo, errors.Trace(err) @@ -714,7 +711,7 @@ func checkAndRenameTables(t *meta.Meta, job *model.Job, oldSchemaID int64, newSc tblInfo.OldSchemaID = 0 } - err = t.DropTableOrView(oldSchemaID, tblInfo.ID, shouldDelAutoID) + err = t.DropTableOrView(oldSchemaID, tblInfo.ID, shouldRecreateAutoID) if err != nil { job.State = model.JobStateCancelled return ver, tblInfo, errors.Trace(err) @@ -730,24 +727,11 @@ func checkAndRenameTables(t *meta.Meta, job *model.Job, oldSchemaID int64, newSc }) tblInfo.Name = *tableName - err = t.CreateTableOrView(newSchemaID, tblInfo) + err = t.CreateTableAndSetAutoID(newSchemaID, tblInfo, rowID, incID, randID) if err != nil { job.State = model.JobStateCancelled return ver, tblInfo, errors.Trace(err) } - // Update the table's auto-increment ID. - if newSchemaID != oldSchemaID { - _, err = t.GenAutoTableID(newSchemaID, tblInfo.ID, autoTableID) - if err != nil { - job.State = model.JobStateCancelled - return ver, tblInfo, errors.Trace(err) - } - _, err = t.GenAutoRandomID(newSchemaID, tblInfo.ID, autoRandID) - if err != nil { - job.State = model.JobStateCancelled - return ver, tblInfo, errors.Trace(err) - } - } return ver, tblInfo, nil } diff --git a/errno/errcode.go b/errno/errcode.go index a47b67668902f..fc992e941814e 100644 --- a/errno/errcode.go +++ b/errno/errcode.go @@ -1022,6 +1022,7 @@ const ( ErrTableOptionInsertMethodUnsupported = 8233 ErrInvalidPlacementSpec = 8234 ErrDDLReorgElementNotExist = 8235 + ErrAutoIDAllocatorNotFound = 8236 // TiKV/PD errors. ErrPDServerTimeout = 9001 diff --git a/errno/errname.go b/errno/errname.go index f64cad2abf123..ca839638e64ae 100644 --- a/errno/errname.go +++ b/errno/errname.go @@ -990,6 +990,7 @@ var MySQLErrName = map[uint16]*mysql.ErrMessage{ ErrUnknownAllocatorType: mysql.Message("Invalid allocator type", nil), ErrAutoRandReadFailed: mysql.Message("Failed to read auto-random value from storage engine", nil), ErrInvalidIncrementAndOffset: mysql.Message("Invalid auto_increment settings: auto_increment_increment: %d, auto_increment_offset: %d, both of them must be in range [1..65535]", nil), + ErrAutoIDAllocatorNotFound: mysql.Message("ID allocator for %s not found", nil), ErrWarnOptimizerHintInvalidInteger: mysql.Message("integer value is out of range in '%s'", nil), ErrWarnOptimizerHintUnsupportedHint: mysql.Message("Optimizer hint %s is not supported by TiDB and is ignored", nil), diff --git a/errors.toml b/errors.toml index 077ade3fc9606..538aa04b83f15 100644 --- a/errors.toml +++ b/errors.toml @@ -46,6 +46,11 @@ error = ''' Failed to read auto-random value from storage engine ''' +["autoid:8236"] +error = ''' +ID allocator for %s not found +''' + ["ddl:1025"] error = ''' Error on rename of '%-.210s' to '%-.210s' (errno: %d - %s) diff --git a/executor/ddl.go b/executor/ddl.go index 81e02f3d366c4..4aa6fbfbc5476 100644 --- a/executor/ddl.go +++ b/executor/ddl.go @@ -399,7 +399,7 @@ func (e *DDLExec) executeRecoverTable(s *ast.RecoverTableStmt) error { return infoschema.ErrTableExists.GenWithStack("Table '%-.192s' already been recover to '%-.192s', can't be recover repeatedly", s.Table.Name.O, tbl.Meta().Name.O) } - autoIncID, autoRandID, err := e.getTableAutoIDsFromSnapshot(job) + rowID, autoIncID, autoRandID, err := e.getTableAutoIDsFromSnapshot(job) if err != nil { return err } @@ -411,28 +411,26 @@ func (e *DDLExec) executeRecoverTable(s *ast.RecoverTableStmt) error { SnapshotTS: job.StartTS, CurAutoIncID: autoIncID, CurAutoRandID: autoRandID, + CurRowID: rowID, } // Call DDL RecoverTable. err = domain.GetDomain(e.ctx).DDL().RecoverTable(e.ctx, recoverInfo) return err } -func (e *DDLExec) getTableAutoIDsFromSnapshot(job *model.Job) (autoIncID, autoRandID int64, err error) { +func (e *DDLExec) getTableAutoIDsFromSnapshot(job *model.Job) (rowID, autoIncID, autoRandID int64, err error) { // Get table original autoIDs before table drop. dom := domain.GetDomain(e.ctx) m, err := dom.GetSnapshotMeta(job.StartTS) if err != nil { - return 0, 0, err + return 0, 0, 0, err } - autoIncID, err = m.GetAutoTableID(job.SchemaID, job.TableID) + rowID, autoIncID, autoRandID, err = m.GetAllAutoIDs(job.SchemaID, job.TableID) if err != nil { - return 0, 0, errors.Errorf("recover table_id: %d, get original autoIncID from snapshot meta err: %s", job.TableID, err.Error()) + err = errors.Errorf("recover table_id: %d, get original autoIncID from snapshot meta err: %s", job.TableID, err.Error()) + return } - autoRandID, err = m.GetAutoRandomID(job.SchemaID, job.TableID) - if err != nil { - return 0, 0, errors.Errorf("recover table_id: %d, get original autoRandID from snapshot meta err: %s", job.TableID, err.Error()) - } - return autoIncID, autoRandID, nil + return } func (e *DDLExec) getRecoverTableByJobID(s *ast.RecoverTableStmt, t *meta.Meta, dom *domain.Domain) (*model.Job, *model.TableInfo, error) { @@ -575,7 +573,7 @@ func (e *DDLExec) executeFlashbackTable(s *ast.FlashBackTableStmt) error { return infoschema.ErrTableExists.GenWithStack("Table '%-.192s' already been flashback to '%-.192s', can't be flashback repeatedly", s.Table.Name.O, tbl.Meta().Name.O) } - autoIncID, autoRandID, err := e.getTableAutoIDsFromSnapshot(job) + rowID, autoIncID, autoRandID, err := e.getTableAutoIDsFromSnapshot(job) if err != nil { return err } @@ -586,6 +584,7 @@ func (e *DDLExec) executeFlashbackTable(s *ast.FlashBackTableStmt) error { SnapshotTS: job.StartTS, CurAutoIncID: autoIncID, CurAutoRandID: autoRandID, + CurRowID: rowID, } // Call DDL RecoverTable. err = domain.GetDomain(e.ctx).DDL().RecoverTable(e.ctx, recoverInfo) diff --git a/executor/executor.go b/executor/executor.go index 2102f3ca1fabf..dc2e651091606 100644 --- a/executor/executor.go +++ b/executor/executor.go @@ -334,13 +334,15 @@ func (e *ShowNextRowIDExec) Next(ctx context.Context, req *chunk.Chunk) error { var colName, idType string switch alloc.GetType() { - case autoid.RowIDAllocType, autoid.AutoIncrementType: + case autoid.RowIDAllocType: + colName = model.ExtraHandleName.O + case autoid.AutoIncrementType: idType = "AUTO_INCREMENT" - if col := tblMeta.GetAutoIncrementColInfo(); col != nil { - colName = col.Name.O - } else { - colName = model.ExtraHandleName.O + col := tblMeta.GetAutoIncrementColInfo() + if col == nil { + return errors.Errorf("auto_increment column not found") } + colName = col.Name.O case autoid.AutoRandomType: idType = "AUTO_RANDOM" colName = tblMeta.GetPkName().O diff --git a/executor/executor_test.go b/executor/executor_test.go index d04d7157dd63b..88eaec8c45752 100644 --- a/executor/executor_test.go +++ b/executor/executor_test.go @@ -3516,15 +3516,12 @@ func (s *testSuite) TestCheckIndex(c *C) { c.Assert(err, IsNil) is := s.domain.InfoSchema() db := model.NewCIStr("test_admin") - dbInfo, ok := is.SchemaByName(db) - c.Assert(ok, IsTrue) tblName := model.NewCIStr("t") tbl, err := is.TableByName(db, tblName) c.Assert(err, IsNil) tbInfo := tbl.Meta() - alloc := autoid.NewAllocator(s.store, dbInfo.ID, false, autoid.RowIDAllocType) - tb, err := tables.TableFromMeta(autoid.NewAllocators(alloc), tbInfo) + tb, err := tables.TableFromMeta(nil, tbInfo) c.Assert(err, IsNil) _, err = se.Execute(context.Background(), "admin check index t c") diff --git a/executor/infoschema_reader.go b/executor/infoschema_reader.go index 1ee6c713b2ed1..e3a2045cadb39 100644 --- a/executor/infoschema_reader.go +++ b/executor/infoschema_reader.go @@ -281,7 +281,7 @@ func getAutoIncrementID(ctx sessionctx.Context, schema *model.DBInfo, tblInfo *m if err != nil { return 0, err } - return tbl.Allocators(ctx).Get(autoid.RowIDAllocType).Base() + 1, nil + return tbl.Allocators(ctx).Get(autoid.AutoIncrementType).Base() + 1, nil } func (e *memtableRetriever) setDataFromSchemata(ctx sessionctx.Context, schemas []*model.DBInfo) { diff --git a/executor/insert_common.go b/executor/insert_common.go index 6e0c944172ec5..189c6290c9d9e 100644 --- a/executor/insert_common.go +++ b/executor/insert_common.go @@ -690,7 +690,7 @@ func (e *InsertValues) lazyAdjustAutoIncrementDatum(ctx context.Context, rows [] } // Use the value if it's not null and not 0. if recordID != 0 { - err = e.Table.RebaseAutoID(e.ctx, recordID, true, autoid.RowIDAllocType) + err = e.Table.RebaseAutoID(e.ctx, recordID, true, autoid.AutoIncrementType) if err != nil { return nil, err } @@ -777,7 +777,7 @@ func (e *InsertValues) adjustAutoIncrementDatum(ctx context.Context, d types.Dat } // Use the value if it's not null and not 0. if recordID != 0 { - err = e.Table.RebaseAutoID(e.ctx, recordID, true, autoid.RowIDAllocType) + err = e.Table.RebaseAutoID(e.ctx, recordID, true, autoid.AutoIncrementType) if err != nil { return types.Datum{}, err } diff --git a/executor/insert_test.go b/executor/insert_test.go index 6ac541a42672e..bf52dc5943962 100644 --- a/executor/insert_test.go +++ b/executor/insert_test.go @@ -1087,13 +1087,13 @@ func (s *testSuite3) TestAutoIDIncrementAndOffset(c *C) { // AutoID allocation will take increment and offset into consideration. tk.MustQuery(`select b from io`).Check(testkit.Rows("10", "12", "14")) // HandleID allocation will ignore the increment and offset. - tk.MustQuery(`select _tidb_rowid from io`).Check(testkit.Rows("15", "16", "17")) + tk.MustQuery(`select _tidb_rowid from io`).Check(testkit.Rows("1", "2", "3")) tk.MustExec(`delete from io`) tk.Se.GetSessionVars().AutoIncrementIncrement = 10 tk.MustExec(`insert into io(b) values (null),(null),(null)`) tk.MustQuery(`select b from io`).Check(testkit.Rows("20", "30", "40")) - tk.MustQuery(`select _tidb_rowid from io`).Check(testkit.Rows("41", "42", "43")) + tk.MustQuery(`select _tidb_rowid from io`).Check(testkit.Rows("4", "5", "6")) // Test invalid value. tk.Se.GetSessionVars().AutoIncrementIncrement = -1 diff --git a/executor/seqtest/seq_executor_test.go b/executor/seqtest/seq_executor_test.go index 80a106173d4fa..5a277a85ff6a1 100644 --- a/executor/seqtest/seq_executor_test.go +++ b/executor/seqtest/seq_executor_test.go @@ -799,17 +799,17 @@ func HelperTestAdminShowNextID(c *C, s *seqTestSuite, str string) { tk.MustExec("create table t(id int, c int)") // Start handle is 1. r := tk.MustQuery(str + " t next_row_id") - r.Check(testkit.Rows("test t _tidb_rowid 1 AUTO_INCREMENT")) + 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(str + " t next_row_id") - r.Check(testkit.Rows("test t _tidb_rowid 11 AUTO_INCREMENT")) + 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(str + " t next_row_id") - r.Check(testkit.Rows("test t _tidb_rowid 21 AUTO_INCREMENT")) + r.Check(testkit.Rows("test t _tidb_rowid 21 ")) tk.MustExec("drop table t") // test for a table with the primary key @@ -854,19 +854,19 @@ func HelperTestAdminShowNextID(c *C, s *seqTestSuite, str string) { // Test for a sequence. tk.MustExec("create sequence seq1 start 15 cache 57") r = tk.MustQuery(str + " seq1 next_row_id") - r.Check(testkit.Rows("test1 seq1 _tidb_rowid 1 AUTO_INCREMENT", "test1 seq1 15 SEQUENCE")) + r.Check(testkit.Rows("test1 seq1 _tidb_rowid 1 ", "test1 seq1 15 SEQUENCE")) r = tk.MustQuery("select nextval(seq1)") r.Check(testkit.Rows("15")) r = tk.MustQuery(str + " seq1 next_row_id") - r.Check(testkit.Rows("test1 seq1 _tidb_rowid 1 AUTO_INCREMENT", "test1 seq1 72 SEQUENCE")) + r.Check(testkit.Rows("test1 seq1 _tidb_rowid 1 ", "test1 seq1 72 SEQUENCE")) r = tk.MustQuery("select nextval(seq1)") r.Check(testkit.Rows("16")) r = tk.MustQuery(str + " seq1 next_row_id") - r.Check(testkit.Rows("test1 seq1 _tidb_rowid 1 AUTO_INCREMENT", "test1 seq1 72 SEQUENCE")) + r.Check(testkit.Rows("test1 seq1 _tidb_rowid 1 ", "test1 seq1 72 SEQUENCE")) r = tk.MustQuery("select setval(seq1, 96)") r.Check(testkit.Rows("96")) r = tk.MustQuery(str + " seq1 next_row_id") - r.Check(testkit.Rows("test1 seq1 _tidb_rowid 1 AUTO_INCREMENT", "test1 seq1 97 SEQUENCE")) + r.Check(testkit.Rows("test1 seq1 _tidb_rowid 1 ", "test1 seq1 97 SEQUENCE")) } func (s *seqTestSuite) TestNoHistoryWhenDisableRetry(c *C) { diff --git a/executor/show.go b/executor/show.go index 8858407cd3a55..714ea9cc1c747 100644 --- a/executor/show.go +++ b/executor/show.go @@ -933,7 +933,7 @@ func ConstructResultOfShowCreateTable(ctx sessionctx.Context, tableInfo *model.T fmt.Fprintf(buf, " COMPRESSION='%s'", tableInfo.Compression) } - incrementAllocator := allocators.Get(autoid.RowIDAllocType) + incrementAllocator := allocators.Get(autoid.AutoIncrementType) if hasAutoIncID && incrementAllocator != nil { autoIncID, err := incrementAllocator.NextGlobalAutoID(tableInfo.ID) if err != nil { diff --git a/executor/write.go b/executor/write.go index abb6c1318d0d2..48ecf78d7af00 100644 --- a/executor/write.go +++ b/executor/write.go @@ -105,7 +105,7 @@ func updateRecord(ctx context.Context, sctx sessionctx.Context, h kv.Handle, old if err != nil { return false, err } - if err = t.RebaseAutoID(sctx, recordID, true, autoid.RowIDAllocType); err != nil { + if err = t.RebaseAutoID(sctx, recordID, true, autoid.AutoIncrementType); err != nil { return false, err } } diff --git a/infoschema/builder.go b/infoschema/builder.go index cc3fd22093ac1..a9ccd37d97c5b 100644 --- a/infoschema/builder.go +++ b/infoschema/builder.go @@ -320,8 +320,15 @@ func (b *Builder) applyCreateTable(m *meta.Meta, dbInfo *model.DBInfo, tableID i } else { switch tp { case model.ActionRebaseAutoID, model.ActionModifyTableAutoIdCache: - newAlloc := autoid.NewAllocator(b.handle.store, dbInfo.ID, tblInfo.IsAutoIncColUnsigned(), autoid.RowIDAllocType) - allocs = append(allocs, newAlloc) + hasRowID := !tblInfo.PKIsHandle && !tblInfo.IsCommonHandle + if hasRowID { + newAlloc := autoid.NewAllocator(b.handle.store, dbInfo.ID, tblInfo.IsAutoIncColUnsigned(), autoid.RowIDAllocType) + allocs = append(allocs, newAlloc) + } + if ok, _ := HasAutoIncrementColumn(tblInfo); ok { + newAlloc := autoid.NewAllocator(b.handle.store, dbInfo.ID, tblInfo.IsAutoIncColUnsigned(), autoid.AutoIncrementType) + allocs = append(allocs, newAlloc) + } case model.ActionRebaseAutoRandomBase: newAlloc := autoid.NewAllocator(b.handle.store, dbInfo.ID, tblInfo.IsAutoRandomBitColUnsigned(), autoid.AutoRandomType) allocs = append(allocs, newAlloc) diff --git a/meta/autoid/autoid.go b/meta/autoid/autoid.go index 3444537bda050..8a1b6341e3d68 100644 --- a/meta/autoid/autoid.go +++ b/meta/autoid/autoid.go @@ -428,14 +428,23 @@ func NewAllocatorsFromTblInfo(store kv.Storage, schemaID int64, tblInfo *model.T var allocs []Allocator dbID := tblInfo.GetDBID(schemaID) hasRowID := !tblInfo.PKIsHandle && !tblInfo.IsCommonHandle - hasAutoIncID := tblInfo.GetAutoIncrementColInfo() != nil - if hasRowID || hasAutoIncID { + if hasRowID { if tblInfo.AutoIdCache > 0 { - allocs = append(allocs, NewAllocator(store, dbID, tblInfo.IsAutoIncColUnsigned(), RowIDAllocType, CustomAutoIncCacheOption(tblInfo.AutoIdCache))) + opt := CustomAutoIncCacheOption(tblInfo.AutoIdCache) + allocs = append(allocs, NewAllocator(store, dbID, tblInfo.IsAutoIncColUnsigned(), RowIDAllocType, opt)) } else { allocs = append(allocs, NewAllocator(store, dbID, tblInfo.IsAutoIncColUnsigned(), RowIDAllocType)) } } + hasAutoIncID := tblInfo.GetAutoIncrementColInfo() != nil + if hasAutoIncID { + if tblInfo.AutoIdCache > 0 { + opt := CustomAutoIncCacheOption(tblInfo.AutoIdCache) + allocs = append(allocs, NewAllocator(store, dbID, tblInfo.IsAutoIncColUnsigned(), AutoIncrementType, opt)) + } else { + allocs = append(allocs, NewAllocator(store, dbID, tblInfo.IsAutoIncColUnsigned(), AutoIncrementType)) + } + } if tblInfo.ContainsAutoRandomBits() { allocs = append(allocs, NewAllocator(store, dbID, tblInfo.IsAutoRandomBitColUnsigned(), AutoRandomType)) } @@ -465,7 +474,7 @@ func (alloc *allocator) Alloc(tableID int64, n uint64, increment, offset int64) if n == 0 { return 0, 0, nil } - if alloc.allocType == AutoIncrementType || alloc.allocType == RowIDAllocType { + if alloc.allocType == AutoIncrementType { if !validIncrementAndOffset(increment, offset) { return 0, 0, errInvalidIncrementAndOffset.GenWithStackByArgs(increment, offset) } @@ -872,9 +881,10 @@ func (alloc *allocator) alloc4Sequence(tableID int64) (min int64, max int64, rou func getAutoIDByAllocType(m *meta.Meta, dbID, tableID int64, allocType AllocatorType) (int64, error) { switch allocType { - // Currently, row id allocator and auto-increment value allocator shares the same key-value pair. - case RowIDAllocType, AutoIncrementType: + case RowIDAllocType: return m.GetAutoTableID(dbID, tableID) + case AutoIncrementType: + return m.GetAutoIncrementID(dbID, tableID) case AutoRandomType: return m.GetAutoRandomID(dbID, tableID) case SequenceType: @@ -886,8 +896,10 @@ func getAutoIDByAllocType(m *meta.Meta, dbID, tableID int64, allocType Allocator func generateAutoIDByAllocType(m *meta.Meta, dbID, tableID, step int64, allocType AllocatorType) (int64, error) { switch allocType { - case RowIDAllocType, AutoIncrementType: + case RowIDAllocType: return m.GenAutoTableID(dbID, tableID, step) + case AutoIncrementType: + return m.GenAutoIncrementID(dbID, tableID, step) case AutoRandomType: return m.GenAutoRandomID(dbID, tableID, step) case SequenceType: diff --git a/meta/autoid/errors.go b/meta/autoid/errors.go index ad0b1bcf5d12b..5d8c0dbd68e8e 100644 --- a/meta/autoid/errors.go +++ b/meta/autoid/errors.go @@ -26,6 +26,7 @@ var ( ErrWrongAutoKey = dbterror.ClassAutoid.NewStd(mysql.ErrWrongAutoKey) ErrInvalidAllocatorType = dbterror.ClassAutoid.NewStd(mysql.ErrUnknownAllocatorType) ErrAutoRandReadFailed = dbterror.ClassAutoid.NewStd(mysql.ErrAutoRandReadFailed) + ErrAutoIDAllocatorNotFound = dbterror.ClassAutoid.NewStd(mysql.ErrAutoIDAllocatorNotFound) ) const ( diff --git a/meta/meta.go b/meta/meta.go index 849835b5574c2..d763009966c76 100644 --- a/meta/meta.go +++ b/meta/meta.go @@ -66,6 +66,7 @@ var ( mSeqCyclePrefix = "SequenceCycle" mTableIDPrefix = "TID" mRandomIDPrefix = "TARID" + mAutoIncIDPrefix = "TINCID" mBootstrapKey = []byte("BootstrapKey") mSchemaDiffPrefix = "Diff" ) @@ -154,6 +155,10 @@ func (m *Meta) autoRandomTableIDKey(tableID int64) []byte { return []byte(fmt.Sprintf("%s:%d", mRandomIDPrefix, tableID)) } +func (m *Meta) autoIncrementIDKey(tableID int64) []byte { + return []byte(fmt.Sprintf("%s:%d", mAutoIncIDPrefix, tableID)) +} + func (m *Meta) tableKey(tableID int64) []byte { return []byte(fmt.Sprintf("%s:%d", mTablePrefix, tableID)) } @@ -178,8 +183,7 @@ func (m *Meta) GenAutoTableIDKeyValue(dbID, tableID, autoID int64) (key, value [ return m.txn.EncodeHashAutoIDKeyValue(dbKey, autoTableIDKey, autoID) } -// GenAutoTableID adds step to the auto ID of the table and returns the sum. -func (m *Meta) GenAutoTableID(dbID, tableID, step int64) (int64, error) { +func (m *Meta) generateID(dbID, tableID, step int64, idKey []byte) (int64, error) { // Check if DB exists. dbKey := m.dbKey(dbID) if err := m.checkDBExists(dbKey); err != nil { @@ -190,24 +194,22 @@ func (m *Meta) GenAutoTableID(dbID, tableID, step int64) (int64, error) { if err := m.checkTableExists(dbKey, tableKey); err != nil { return 0, errors.Trace(err) } + return m.txn.HInc(dbKey, idKey, step) +} - return m.txn.HInc(dbKey, m.autoTableIDKey(tableID), step) +// GenAutoTableID adds step to the auto ID of the table and returns the sum. +func (m *Meta) GenAutoTableID(dbID, tableID, step int64) (int64, error) { + return m.generateID(dbID, tableID, step, m.autoTableIDKey(tableID)) } -// GenAutoRandomID adds step to the auto shard ID of the table and returns the sum. +// GenAutoRandomID adds step to the auto random ID of the table and returns the sum. func (m *Meta) GenAutoRandomID(dbID, tableID, step int64) (int64, error) { - // Check if DB exists. - dbKey := m.dbKey(dbID) - if err := m.checkDBExists(dbKey); err != nil { - return 0, errors.Trace(err) - } - // Check if table exists. - tableKey := m.tableKey(tableID) - if err := m.checkTableExists(dbKey, tableKey); err != nil { - return 0, errors.Trace(err) - } + return m.generateID(dbID, tableID, step, m.autoRandomTableIDKey(tableID)) +} - return m.txn.HInc(dbKey, m.autoRandomTableIDKey(tableID), step) +// GenAutoIncrementID adds step to the auto increment ID of the table and returns the sum. +func (m *Meta) GenAutoIncrementID(dbID, tableID, step int64) (int64, error) { + return m.generateID(dbID, tableID, step, m.autoIncrementIDKey(tableID)) } // GetAutoTableID gets current auto id with table id. @@ -220,6 +222,25 @@ func (m *Meta) GetAutoRandomID(dbID int64, tableID int64) (int64, error) { return m.txn.HGetInt64(m.dbKey(dbID), m.autoRandomTableIDKey(tableID)) } +// GetAutoIncrementID gets current auto increment id with table id. +func (m *Meta) GetAutoIncrementID(dbID int64, tableID int64) (int64, error) { + return m.txn.HGetInt64(m.dbKey(dbID), m.autoIncrementIDKey(tableID)) +} + +// GetAllAutoIDs gets all the auto IDs from a table, including _tidb_rowid, auto_increment ID and auto_random ID. +func (m *Meta) GetAllAutoIDs(dbID int64, tableID int64) (rowID, autoIncID, autoRandID int64, err error) { + rowID, err = m.GetAutoTableID(dbID, tableID) + if err != nil { + return + } + autoIncID, err = m.GetAutoIncrementID(dbID, tableID) + if err != nil { + return + } + autoRandID, err = m.GetAutoRandomID(dbID, tableID) + return +} + // GenSequenceValue adds step to the sequence value and returns the sum. func (m *Meta) GenSequenceValue(dbID, sequenceID, step int64) (int64, error) { // Check if DB exists. @@ -352,15 +373,18 @@ func (m *Meta) CreateTableOrView(dbID int64, tableInfo *model.TableInfo) error { } // CreateTableAndSetAutoID creates a table with tableInfo in database, -// and rebases the table autoID. -func (m *Meta) CreateTableAndSetAutoID(dbID int64, tableInfo *model.TableInfo, autoIncID, autoRandID int64) error { +// and rebase the table autoIDs. +func (m *Meta) CreateTableAndSetAutoID(dbID int64, tableInfo *model.TableInfo, rowID, autoIncID, autoRandID int64) error { err := m.CreateTableOrView(dbID, tableInfo) if err != nil { return errors.Trace(err) } - _, err = m.txn.HInc(m.dbKey(dbID), m.autoTableIDKey(tableInfo.ID), autoIncID) - if err != nil { - return errors.Trace(err) + hasRowID := !tableInfo.PKIsHandle && !tableInfo.IsCommonHandle + if hasRowID { + _, err = m.txn.HInc(m.dbKey(dbID), m.autoTableIDKey(tableInfo.ID), rowID) + if err != nil { + return errors.Trace(err) + } } if tableInfo.AutoRandomBits > 0 { _, err = m.txn.HInc(m.dbKey(dbID), m.autoRandomTableIDKey(tableInfo.ID), autoRandID) @@ -368,6 +392,12 @@ func (m *Meta) CreateTableAndSetAutoID(dbID int64, tableInfo *model.TableInfo, a return errors.Trace(err) } } + if tableInfo.GetAutoIncrementColInfo() != nil { + _, err = m.txn.HInc(m.dbKey(dbID), m.autoIncrementIDKey(tableInfo.ID), autoIncID) + if err != nil { + return errors.Trace(err) + } + } return nil } @@ -449,6 +479,9 @@ func (m *Meta) DropTableOrView(dbID int64, tblID int64, delAutoID bool) error { if err := m.txn.HDel(dbKey, m.autoRandomTableIDKey(tblID)); err != nil { return errors.Trace(err) } + if err := m.txn.HDel(dbKey, m.autoIncrementIDKey(tblID)); err != nil { + return errors.Trace(err) + } } return nil } diff --git a/meta/meta_test.go b/meta/meta_test.go index 4ea9db0e88a12..21fd64663b6e3 100644 --- a/meta/meta_test.go +++ b/meta/meta_test.go @@ -214,7 +214,7 @@ func (s *testSuite) TestMeta(c *C) { ID: 3, Name: model.NewCIStr("tbl3"), } - err = t.CreateTableAndSetAutoID(1, tbInfo3, 123, 0) + err = t.CreateTableAndSetAutoID(1, tbInfo3, 123, 0, 0) c.Assert(err, IsNil) id, err := t.GetAutoTableID(1, tbInfo3.ID) c.Assert(err, IsNil) diff --git a/session/session_test.go b/session/session_test.go index ea806445c7bb2..1f0d9323abe65 100644 --- a/session/session_test.go +++ b/session/session_test.go @@ -1092,16 +1092,13 @@ func (s *testSessionSuite) TestAutoIncrementID(c *C) { tk.MustExec("insert into autoid values(9223372036854775808);") tk.MustExec("insert into autoid values();") tk.MustExec("insert into autoid values();") - tk.MustQuery("select * from autoid").Check(testkit.Rows("9223372036854775808", "9223372036854775810", "9223372036854775812")) + tk.MustQuery("select * from autoid").Check(testkit.Rows("9223372036854775808", "9223372036854775809", "9223372036854775810")) // In TiDB : _tidb_rowid will also consume the autoID when the auto_increment column is not the primary key. // Using the MaxUint64 and MaxInt64 as the autoID upper limit like MySQL will cause _tidb_rowid allocation fail here. - _, err := tk.Exec("insert into autoid values(18446744073709551614)") - c.Assert(terror.ErrorEqual(err, autoid.ErrAutoincReadFailed), IsTrue) - _, err = tk.Exec("insert into autoid values()") - c.Assert(terror.ErrorEqual(err, autoid.ErrAutoincReadFailed), IsTrue) - // FixMe: MySQL works fine with the this sql. - _, err = tk.Exec("insert into autoid values(18446744073709551615)") + tk.MustExec("insert into autoid values(18446744073709551614)") + _, err := tk.Exec("insert into autoid values()") c.Assert(terror.ErrorEqual(err, autoid.ErrAutoincReadFailed), IsTrue) + tk.MustExec("insert into autoid values(18446744073709551615)") tk.MustExec("drop table if exists autoid") tk.MustExec("create table autoid(`auto_inc_id` bigint(20) UNSIGNED NOT NULL AUTO_INCREMENT,UNIQUE KEY `auto_inc_id` (`auto_inc_id`))") @@ -1120,15 +1117,12 @@ func (s *testSessionSuite) TestAutoIncrementID(c *C) { // Corner cases for signed bigint auto_increment Columns. tk.MustExec("drop table if exists autoid") tk.MustExec("create table autoid(`auto_inc_id` bigint(20) NOT NULL AUTO_INCREMENT,UNIQUE KEY `auto_inc_id` (`auto_inc_id`))") - // In TiDB : _tidb_rowid will also consume the autoID when the auto_increment column is not the primary key. - // Using the MaxUint64 and MaxInt64 as autoID upper limit like MySQL will cause insert fail if the values is - // 9223372036854775806. Because _tidb_rowid will be allocated 9223372036854775807 at same time. - tk.MustExec("insert into autoid values(9223372036854775805);") - tk.MustQuery("select auto_inc_id, _tidb_rowid from autoid use index()").Check(testkit.Rows("9223372036854775805 9223372036854775806")) + tk.MustExec("insert into autoid values(9223372036854775806);") + tk.MustQuery("select auto_inc_id, _tidb_rowid from autoid use index()").Check(testkit.Rows("9223372036854775806 1")) _, err = tk.Exec("insert into autoid values();") c.Assert(terror.ErrorEqual(err, autoid.ErrAutoincReadFailed), IsTrue) - tk.MustQuery("select auto_inc_id, _tidb_rowid from autoid use index()").Check(testkit.Rows("9223372036854775805 9223372036854775806")) - tk.MustQuery("select auto_inc_id, _tidb_rowid from autoid use index(auto_inc_id)").Check(testkit.Rows("9223372036854775805 9223372036854775806")) + tk.MustQuery("select auto_inc_id, _tidb_rowid from autoid use index()").Check(testkit.Rows("9223372036854775806 1")) + tk.MustQuery("select auto_inc_id, _tidb_rowid from autoid use index(auto_inc_id)").Check(testkit.Rows("9223372036854775806 1")) tk.MustExec("drop table if exists autoid") tk.MustExec("create table autoid(`auto_inc_id` bigint(20) NOT NULL AUTO_INCREMENT,UNIQUE KEY `auto_inc_id` (`auto_inc_id`))") diff --git a/table/table.go b/table/table.go index a43fe104bdb6f..f7358b3bcdfb3 100644 --- a/table/table.go +++ b/table/table.go @@ -220,7 +220,7 @@ func AllocAutoIncrementValue(ctx context.Context, t Table, sctx sessionctx.Conte } increment := sctx.GetSessionVars().AutoIncrementIncrement offset := sctx.GetSessionVars().AutoIncrementOffset - _, max, err := t.Allocators(sctx).Get(autoid.RowIDAllocType).Alloc(t.Meta().ID, uint64(1), int64(increment), int64(offset)) + _, max, err := t.Allocators(sctx).Get(autoid.AutoIncrementType).Alloc(t.Meta().ID, uint64(1), int64(increment), int64(offset)) if err != nil { return 0, err } @@ -236,7 +236,7 @@ func AllocBatchAutoIncrementValue(ctx context.Context, t Table, sctx sessionctx. } increment = int64(sctx.GetSessionVars().AutoIncrementIncrement) offset := int64(sctx.GetSessionVars().AutoIncrementOffset) - min, max, err := t.Allocators(sctx).Get(autoid.RowIDAllocType).Alloc(t.Meta().ID, uint64(N), increment, offset) + min, max, err := t.Allocators(sctx).Get(autoid.AutoIncrementType).Alloc(t.Meta().ID, uint64(N), increment, offset) if err != nil { return min, max, err } diff --git a/table/tables/tables_test.go b/table/tables/tables_test.go index b9fb958e7ad11..edee198bab8d7 100644 --- a/table/tables/tables_test.go +++ b/table/tables/tables_test.go @@ -102,10 +102,6 @@ func (ts *testSuite) TestBasic(c *C) { c.Assert(err, IsNil) c.Assert(autoID, Greater, int64(0)) - handle, err := tables.AllocHandle(nil, tb) - c.Assert(err, IsNil) - c.Assert(handle.IntValue(), Greater, int64(0)) - ctx := ts.se rid, err := tb.AddRecord(ctx, types.MakeDatums(1, "abc")) c.Assert(err, IsNil) @@ -159,10 +155,10 @@ func (ts *testSuite) TestBasic(c *C) { c.Assert(err, IsNil) table.MockTableFromMeta(tb.Meta()) - alc := tb.Allocators(nil).Get(autoid.RowIDAllocType) + alc := tb.Allocators(nil).Get(autoid.AutoIncrementType) c.Assert(alc, NotNil) - err = tb.RebaseAutoID(nil, 0, false, autoid.RowIDAllocType) + err = tb.RebaseAutoID(nil, 0, false, autoid.AutoIncrementType) c.Assert(err, IsNil) } @@ -246,10 +242,6 @@ func (ts *testSuite) TestUniqueIndexMultipleNullEntries(c *C) { c.Assert(string(tb.RecordPrefix()), Not(Equals), "") c.Assert(tables.FindIndexByColName(tb, "b"), NotNil) - handle, err := tables.AllocHandle(nil, tb) - c.Assert(err, IsNil) - c.Assert(handle.IntValue(), Greater, int64(0)) - autoid, err := table.AllocAutoIncrementValue(context.Background(), tb, ts.se) c.Assert(err, IsNil) c.Assert(autoid, Greater, int64(0)) From 8c76e4c4aeca82079d0fb203f6577eff7766deb3 Mon Sep 17 00:00:00 2001 From: tangenta Date: Sat, 20 Feb 2021 16:46:32 +0800 Subject: [PATCH 2/7] only separate the allocators with table info version >= 4 --- executor/executor.go | 1 + go.mod | 2 ++ go.sum | 2 ++ infoschema/builder.go | 8 ++++++-- meta/autoid/autoid.go | 47 ++++++++++++++++++++++++++----------------- 5 files changed, 39 insertions(+), 21 deletions(-) diff --git a/executor/executor.go b/executor/executor.go index 7b5c69953a1bd..adb3669d881d1 100644 --- a/executor/executor.go +++ b/executor/executor.go @@ -346,6 +346,7 @@ func (e *ShowNextRowIDExec) Next(ctx context.Context, req *chunk.Chunk) error { var colName, idType string switch alloc.GetType() { case autoid.RowIDAllocType: + idType = "_tidb_rowid" colName = model.ExtraHandleName.O case autoid.AutoIncrementType: idType = "AUTO_INCREMENT" diff --git a/go.mod b/go.mod index 3e773b641b1fa..d7633e18549bb 100644 --- a/go.mod +++ b/go.mod @@ -85,3 +85,5 @@ require ( ) go 1.13 + +replace github.com/pingcap/parser => github.com/tangenta/parser v0.0.0-20210219085229-04201433f5dc diff --git a/go.sum b/go.sum index e3d255cb879f6..dd592e702f9af 100644 --- a/go.sum +++ b/go.sum @@ -512,6 +512,8 @@ github.com/swaggo/swag v1.6.3/go.mod h1:wcc83tB4Mb2aNiL/HP4MFeQdpHUrca+Rp/DRNgWA github.com/swaggo/swag v1.6.6-0.20200529100950-7c765ddd0476/go.mod h1:xDhTyuFIujYiN3DKWC/H/83xcfHp+UE/IzWWampG7Zc= github.com/syndtr/goleveldb v1.0.1-0.20190318030020-c3a204f8e965 h1:1oFLiOyVl+W7bnBzGhf7BbIv9loSFQcieWWYIjLqcAw= github.com/syndtr/goleveldb v1.0.1-0.20190318030020-c3a204f8e965/go.mod h1:9OrXJhf154huy1nPWmuSrkgjPUtUNhA+Zmy+6AESzuA= +github.com/tangenta/parser v0.0.0-20210219085229-04201433f5dc h1:/H/cU2264RaFU7UtXGJArl2q+tFTaFoAWVvJlZgJXI8= +github.com/tangenta/parser v0.0.0-20210219085229-04201433f5dc/go.mod h1:GbEr2PgY72/4XqPZzmzstlOU/+il/wrjeTNFs6ihsSE= github.com/thoas/go-funk v0.7.0/go.mod h1:+IWnUfUmFO1+WVYQWQtIJHeRRdaIyyYglZN7xzUPe4Q= github.com/tiancaiamao/appdash v0.0.0-20181126055449-889f96f722a2 h1:mbAskLJ0oJfDRtkanvQPiooDH8HvJ2FBh+iKT/OmiQQ= github.com/tiancaiamao/appdash v0.0.0-20181126055449-889f96f722a2/go.mod h1:2PfKggNGDuadAa0LElHrByyrz4JPZ9fFx6Gs7nx7ZZU= diff --git a/infoschema/builder.go b/infoschema/builder.go index e1a15f086a01e..eceae6ff3132c 100644 --- a/infoschema/builder.go +++ b/infoschema/builder.go @@ -361,12 +361,16 @@ func (b *Builder) applyCreateTable(m *meta.Meta, dbInfo *model.DBInfo, tableID i switch tp { case model.ActionRebaseAutoID, model.ActionModifyTableAutoIdCache: hasRowID := !tblInfo.PKIsHandle && !tblInfo.IsCommonHandle + var opts []autoid.AllocOption + if tblInfo.Version >= model.TableInfoVersion4 { + opts = []autoid.AllocOption{autoid.V4TableInfoAllocOption{}} + } if hasRowID { - newAlloc := autoid.NewAllocator(b.handle.store, dbInfo.ID, tblInfo.IsAutoIncColUnsigned(), autoid.RowIDAllocType) + newAlloc := autoid.NewAllocator(b.handle.store, dbInfo.ID, tblInfo.IsAutoIncColUnsigned(), autoid.RowIDAllocType, opts...) allocs = append(allocs, newAlloc) } if ok, _ := HasAutoIncrementColumn(tblInfo); ok { - newAlloc := autoid.NewAllocator(b.handle.store, dbInfo.ID, tblInfo.IsAutoIncColUnsigned(), autoid.AutoIncrementType) + newAlloc := autoid.NewAllocator(b.handle.store, dbInfo.ID, tblInfo.IsAutoIncColUnsigned(), autoid.AutoIncrementType, opts...) allocs = append(allocs, newAlloc) } case model.ActionRebaseAutoRandomBase: diff --git a/meta/autoid/autoid.go b/meta/autoid/autoid.go index b5d1ca7079d1f..8fe5a80c3b6bd 100644 --- a/meta/autoid/autoid.go +++ b/meta/autoid/autoid.go @@ -91,6 +91,14 @@ func (step CustomAutoIncCacheOption) ApplyOn(alloc *allocator) { alloc.customStep = true } +// V4TableInfoAllocOption is one kind of AllocOption to represent the related `TableInfo` is in version 4. +type V4TableInfoAllocOption struct{} + +// ApplyOn is implement the AllocOption interface. +func (step V4TableInfoAllocOption) ApplyOn(alloc *allocator) { + alloc.greaterThanV4TableInfo = true +} + // AllocOption is a interface to define allocator custom options coming in future. type AllocOption interface { ApplyOn(*allocator) @@ -154,13 +162,14 @@ type allocator struct { end int64 store kv.Storage // dbID is current database's ID. - dbID int64 - isUnsigned bool - lastAllocTime time.Time - step int64 - customStep bool - allocType AllocatorType - sequence *model.SequenceInfo + dbID int64 + isUnsigned bool + lastAllocTime time.Time + step int64 + customStep bool + allocType AllocatorType // Please use GetType() to access this field. + sequence *model.SequenceInfo + greaterThanV4TableInfo bool } // GetStep is only used by tests @@ -408,6 +417,9 @@ func NewAllocator(store kv.Storage, dbID int64, isUnsigned bool, allocType Alloc for _, fn := range opts { fn.ApplyOn(alloc) } + if !alloc.greaterThanV4TableInfo && alloc.allocType == AutoIncrementType { + alloc.allocType = RowIDAllocType + } return alloc } @@ -426,25 +438,22 @@ func NewSequenceAllocator(store kv.Storage, dbID int64, info *model.SequenceInfo // NewAllocatorsFromTblInfo creates an array of allocators of different types with the information of model.TableInfo. func NewAllocatorsFromTblInfo(store kv.Storage, schemaID int64, tblInfo *model.TableInfo) Allocators { + var opts []AllocOption + if tblInfo.AutoIdCache > 0 { + opts = []AllocOption{CustomAutoIncCacheOption(tblInfo.AutoIdCache)} + } + if tblInfo.Version >= model.TableInfoVersion4 { + opts = append(opts, V4TableInfoAllocOption{}) + } var allocs []Allocator dbID := tblInfo.GetDBID(schemaID) hasRowID := !tblInfo.PKIsHandle && !tblInfo.IsCommonHandle if hasRowID { - if tblInfo.AutoIdCache > 0 { - opt := CustomAutoIncCacheOption(tblInfo.AutoIdCache) - allocs = append(allocs, NewAllocator(store, dbID, tblInfo.IsAutoIncColUnsigned(), RowIDAllocType, opt)) - } else { - allocs = append(allocs, NewAllocator(store, dbID, tblInfo.IsAutoIncColUnsigned(), RowIDAllocType)) - } + allocs = append(allocs, NewAllocator(store, dbID, tblInfo.IsAutoIncColUnsigned(), RowIDAllocType, opts...)) } hasAutoIncID := tblInfo.GetAutoIncrementColInfo() != nil if hasAutoIncID { - if tblInfo.AutoIdCache > 0 { - opt := CustomAutoIncCacheOption(tblInfo.AutoIdCache) - allocs = append(allocs, NewAllocator(store, dbID, tblInfo.IsAutoIncColUnsigned(), AutoIncrementType, opt)) - } else { - allocs = append(allocs, NewAllocator(store, dbID, tblInfo.IsAutoIncColUnsigned(), AutoIncrementType)) - } + allocs = append(allocs, NewAllocator(store, dbID, tblInfo.IsAutoIncColUnsigned(), AutoIncrementType, opts...)) } if tblInfo.ContainsAutoRandomBits() { allocs = append(allocs, NewAllocator(store, dbID, tblInfo.IsAutoRandomBitColUnsigned(), AutoRandomType)) From 2f6daead26842c2e6d734b5e16a895f5c9fde2b5 Mon Sep 17 00:00:00 2001 From: tangenta Date: Sat, 20 Feb 2021 17:17:03 +0800 Subject: [PATCH 3/7] remove unnecessary alloc option --- infoschema/builder.go | 14 ++++++-------- meta/autoid/autoid.go | 41 ++++++++++++++++------------------------- 2 files changed, 22 insertions(+), 33 deletions(-) diff --git a/infoschema/builder.go b/infoschema/builder.go index eceae6ff3132c..094d712f940b8 100644 --- a/infoschema/builder.go +++ b/infoschema/builder.go @@ -361,17 +361,15 @@ func (b *Builder) applyCreateTable(m *meta.Meta, dbInfo *model.DBInfo, tableID i switch tp { case model.ActionRebaseAutoID, model.ActionModifyTableAutoIdCache: hasRowID := !tblInfo.PKIsHandle && !tblInfo.IsCommonHandle - var opts []autoid.AllocOption - if tblInfo.Version >= model.TableInfoVersion4 { - opts = []autoid.AllocOption{autoid.V4TableInfoAllocOption{}} - } if hasRowID { - newAlloc := autoid.NewAllocator(b.handle.store, dbInfo.ID, tblInfo.IsAutoIncColUnsigned(), autoid.RowIDAllocType, opts...) + newAlloc := autoid.NewAllocator(b.handle.store, dbInfo.ID, tblInfo.IsAutoIncColUnsigned(), autoid.RowIDAllocType) allocs = append(allocs, newAlloc) } - if ok, _ := HasAutoIncrementColumn(tblInfo); ok { - newAlloc := autoid.NewAllocator(b.handle.store, dbInfo.ID, tblInfo.IsAutoIncColUnsigned(), autoid.AutoIncrementType, opts...) - allocs = append(allocs, newAlloc) + if tblInfo.Version >= model.TableInfoVersion4 { + if ok, _ := HasAutoIncrementColumn(tblInfo); ok { + newAlloc := autoid.NewAllocator(b.handle.store, dbInfo.ID, tblInfo.IsAutoIncColUnsigned(), autoid.AutoIncrementType) + allocs = append(allocs, newAlloc) + } } case model.ActionRebaseAutoRandomBase: newAlloc := autoid.NewAllocator(b.handle.store, dbInfo.ID, tblInfo.IsAutoRandomBitColUnsigned(), autoid.AutoRandomType) diff --git a/meta/autoid/autoid.go b/meta/autoid/autoid.go index 8fe5a80c3b6bd..0fd29deaf3cf2 100644 --- a/meta/autoid/autoid.go +++ b/meta/autoid/autoid.go @@ -91,14 +91,6 @@ func (step CustomAutoIncCacheOption) ApplyOn(alloc *allocator) { alloc.customStep = true } -// V4TableInfoAllocOption is one kind of AllocOption to represent the related `TableInfo` is in version 4. -type V4TableInfoAllocOption struct{} - -// ApplyOn is implement the AllocOption interface. -func (step V4TableInfoAllocOption) ApplyOn(alloc *allocator) { - alloc.greaterThanV4TableInfo = true -} - // AllocOption is a interface to define allocator custom options coming in future. type AllocOption interface { ApplyOn(*allocator) @@ -153,6 +145,10 @@ func (all Allocators) Get(allocType AllocatorType) Allocator { return a } } + // fallback to row id allocator. + if allocType == AutoIncrementType { + return all.Get(RowIDAllocType) + } return nil } @@ -162,14 +158,13 @@ type allocator struct { end int64 store kv.Storage // dbID is current database's ID. - dbID int64 - isUnsigned bool - lastAllocTime time.Time - step int64 - customStep bool - allocType AllocatorType // Please use GetType() to access this field. - sequence *model.SequenceInfo - greaterThanV4TableInfo bool + dbID int64 + isUnsigned bool + lastAllocTime time.Time + step int64 + customStep bool + allocType AllocatorType + sequence *model.SequenceInfo } // GetStep is only used by tests @@ -417,9 +412,6 @@ func NewAllocator(store kv.Storage, dbID int64, isUnsigned bool, allocType Alloc for _, fn := range opts { fn.ApplyOn(alloc) } - if !alloc.greaterThanV4TableInfo && alloc.allocType == AutoIncrementType { - alloc.allocType = RowIDAllocType - } return alloc } @@ -442,18 +434,17 @@ func NewAllocatorsFromTblInfo(store kv.Storage, schemaID int64, tblInfo *model.T if tblInfo.AutoIdCache > 0 { opts = []AllocOption{CustomAutoIncCacheOption(tblInfo.AutoIdCache)} } - if tblInfo.Version >= model.TableInfoVersion4 { - opts = append(opts, V4TableInfoAllocOption{}) - } var allocs []Allocator dbID := tblInfo.GetDBID(schemaID) hasRowID := !tblInfo.PKIsHandle && !tblInfo.IsCommonHandle if hasRowID { allocs = append(allocs, NewAllocator(store, dbID, tblInfo.IsAutoIncColUnsigned(), RowIDAllocType, opts...)) } - hasAutoIncID := tblInfo.GetAutoIncrementColInfo() != nil - if hasAutoIncID { - allocs = append(allocs, NewAllocator(store, dbID, tblInfo.IsAutoIncColUnsigned(), AutoIncrementType, opts...)) + if tblInfo.Version >= model.TableInfoVersion4 { + hasAutoIncID := tblInfo.GetAutoIncrementColInfo() != nil + if hasAutoIncID { + allocs = append(allocs, NewAllocator(store, dbID, tblInfo.IsAutoIncColUnsigned(), AutoIncrementType, opts...)) + } } if tblInfo.ContainsAutoRandomBits() { allocs = append(allocs, NewAllocator(store, dbID, tblInfo.IsAutoRandomBitColUnsigned(), AutoRandomType)) From afc985f933a52001d92a1a8aed1dc2d0eeb824c9 Mon Sep 17 00:00:00 2001 From: tangenta Date: Mon, 22 Feb 2021 10:24:05 +0800 Subject: [PATCH 4/7] update parser dependency --- go.mod | 4 +--- go.sum | 6 ++---- 2 files changed, 3 insertions(+), 7 deletions(-) diff --git a/go.mod b/go.mod index d7633e18549bb..7ab0b4091499a 100644 --- a/go.mod +++ b/go.mod @@ -46,7 +46,7 @@ require ( github.com/pingcap/goleveldb v0.0.0-20191226122134-f82aafb29989 github.com/pingcap/kvproto v0.0.0-20210204054616-1c1ed89bb167 github.com/pingcap/log v0.0.0-20201112100606-8f1e84a3abc8 - github.com/pingcap/parser v0.0.0-20210203141130-32ef3e0e2e13 + github.com/pingcap/parser v0.0.0-20210220173529-f85308956f9a github.com/pingcap/sysutil v0.0.0-20201130064824-f0c8aa6a6966 github.com/pingcap/tidb-tools v4.0.9-0.20201127090955-2707c97b3853+incompatible github.com/pingcap/tipb v0.0.0-20201229060814-148bc717ce4c @@ -85,5 +85,3 @@ require ( ) go 1.13 - -replace github.com/pingcap/parser => github.com/tangenta/parser v0.0.0-20210219085229-04201433f5dc diff --git a/go.sum b/go.sum index dd592e702f9af..cc8dc7747bc28 100644 --- a/go.sum +++ b/go.sum @@ -411,8 +411,8 @@ github.com/pingcap/log v0.0.0-20200117041106-d28c14d3b1cd/go.mod h1:4rbK1p9ILyIf github.com/pingcap/log v0.0.0-20200511115504-543df19646ad/go.mod h1:4rbK1p9ILyIfb6hU7OG2CiWSqMXnp3JMbiaVJ6mvoY8= github.com/pingcap/log v0.0.0-20201112100606-8f1e84a3abc8 h1:M+DNpOu/I3uDmwee6vcnoPd6GgSMqND4gxvDQ/W584U= github.com/pingcap/log v0.0.0-20201112100606-8f1e84a3abc8/go.mod h1:4rbK1p9ILyIfb6hU7OG2CiWSqMXnp3JMbiaVJ6mvoY8= -github.com/pingcap/parser v0.0.0-20210203141130-32ef3e0e2e13 h1:hIbRR2siRrJniSRrMU+XgZl3reOWiPZk0P+7PwKXuEQ= -github.com/pingcap/parser v0.0.0-20210203141130-32ef3e0e2e13/go.mod h1:GbEr2PgY72/4XqPZzmzstlOU/+il/wrjeTNFs6ihsSE= +github.com/pingcap/parser v0.0.0-20210220173529-f85308956f9a h1:PyyRS4ULbTY30b+Pd/SjqwUX3H47mcHe/4zbFW4uCgQ= +github.com/pingcap/parser v0.0.0-20210220173529-f85308956f9a/go.mod h1:GbEr2PgY72/4XqPZzmzstlOU/+il/wrjeTNFs6ihsSE= github.com/pingcap/sysutil v0.0.0-20200206130906-2bfa6dc40bcd/go.mod h1:EB/852NMQ+aRKioCpToQ94Wl7fktV+FNnxf3CX/TTXI= github.com/pingcap/sysutil v0.0.0-20201130064824-f0c8aa6a6966 h1:JI0wOAb8aQML0vAVLHcxTEEC0VIwrk6gtw3WjbHvJLA= github.com/pingcap/sysutil v0.0.0-20201130064824-f0c8aa6a6966/go.mod h1:EB/852NMQ+aRKioCpToQ94Wl7fktV+FNnxf3CX/TTXI= @@ -512,8 +512,6 @@ github.com/swaggo/swag v1.6.3/go.mod h1:wcc83tB4Mb2aNiL/HP4MFeQdpHUrca+Rp/DRNgWA github.com/swaggo/swag v1.6.6-0.20200529100950-7c765ddd0476/go.mod h1:xDhTyuFIujYiN3DKWC/H/83xcfHp+UE/IzWWampG7Zc= github.com/syndtr/goleveldb v1.0.1-0.20190318030020-c3a204f8e965 h1:1oFLiOyVl+W7bnBzGhf7BbIv9loSFQcieWWYIjLqcAw= github.com/syndtr/goleveldb v1.0.1-0.20190318030020-c3a204f8e965/go.mod h1:9OrXJhf154huy1nPWmuSrkgjPUtUNhA+Zmy+6AESzuA= -github.com/tangenta/parser v0.0.0-20210219085229-04201433f5dc h1:/H/cU2264RaFU7UtXGJArl2q+tFTaFoAWVvJlZgJXI8= -github.com/tangenta/parser v0.0.0-20210219085229-04201433f5dc/go.mod h1:GbEr2PgY72/4XqPZzmzstlOU/+il/wrjeTNFs6ihsSE= github.com/thoas/go-funk v0.7.0/go.mod h1:+IWnUfUmFO1+WVYQWQtIJHeRRdaIyyYglZN7xzUPe4Q= github.com/tiancaiamao/appdash v0.0.0-20181126055449-889f96f722a2 h1:mbAskLJ0oJfDRtkanvQPiooDH8HvJ2FBh+iKT/OmiQQ= github.com/tiancaiamao/appdash v0.0.0-20181126055449-889f96f722a2/go.mod h1:2PfKggNGDuadAa0LElHrByyrz4JPZ9fFx6Gs7nx7ZZU= From 4adb489a191f77903d6cdbbac7ab69162b7dc635 Mon Sep 17 00:00:00 2001 From: tangenta Date: Mon, 22 Feb 2021 10:54:09 +0800 Subject: [PATCH 5/7] fix error.toml --- errors.toml | 2 +- go.mod | 2 +- go.sum | 4 ++-- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/errors.toml b/errors.toml index 865833c870d43..b94c245d24b54 100644 --- a/errors.toml +++ b/errors.toml @@ -46,7 +46,7 @@ error = ''' Failed to read auto-random value from storage engine ''' -["autoid:8236"] +["autoid:8237"] error = ''' ID allocator for %s not found ''' diff --git a/go.mod b/go.mod index 7ab0b4091499a..0065950e18902 100644 --- a/go.mod +++ b/go.mod @@ -79,7 +79,7 @@ require ( gopkg.in/check.v1 v1.0.0-20200902074654-038fdea0a05b // indirect gopkg.in/natefinch/lumberjack.v2 v2.0.0 gopkg.in/yaml.v2 v2.3.0 // indirect - honnef.co/go/tools v0.1.1 // indirect + honnef.co/go/tools v0.1.2 // indirect sourcegraph.com/sourcegraph/appdash v0.0.0-20190731080439-ebfcffb1b5c0 sourcegraph.com/sourcegraph/appdash-data v0.0.0-20151005221446-73f23eafcf67 ) diff --git a/go.sum b/go.sum index cc8dc7747bc28..ea186cc5c648a 100644 --- a/go.sum +++ b/go.sum @@ -835,8 +835,8 @@ honnef.co/go/tools v0.0.0-20190106161140-3f1c8253044a/go.mod h1:rf3lG4BRIbNafJWh honnef.co/go/tools v0.0.0-20190418001031-e561f6794a2a/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= honnef.co/go/tools v0.0.0-20190523083050-ea95bdfd59fc/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= honnef.co/go/tools v0.0.1-2019.2.3/go.mod h1:a3bituU0lyd329TUQxRnasdCoJDkEUEAqEt0JzvZhAg= -honnef.co/go/tools v0.1.1 h1:EVDuO03OCZwpV2t/tLLxPmPiomagMoBOgfPt0FM+4IY= -honnef.co/go/tools v0.1.1/go.mod h1:NgwopIslSNH47DimFoV78dnkksY2EFtX0ajyb3K/las= +honnef.co/go/tools v0.1.2 h1:SMdYLJl312RXuxXziCCHhRsp/tvct9cGKey0yv95tZM= +honnef.co/go/tools v0.1.2/go.mod h1:NgwopIslSNH47DimFoV78dnkksY2EFtX0ajyb3K/las= k8s.io/klog v1.0.0/go.mod h1:4Bi6QPql/J/LkTDqv7R/cd3hPo4k2DG6Ptcz060Ez5I= rsc.io/binaryregexp v0.2.0/go.mod h1:qTv7/COck+e2FymRvadv62gMdZztPaShugOCi3I+8D8= sigs.k8s.io/yaml v1.1.0 h1:4A07+ZFc2wgJwo8YNlQpr1rVlgUDlxXHhPJciaPY5gs= From 8337016a297a0575bc8948cd0c58446e911d8dbc Mon Sep 17 00:00:00 2001 From: tangenta Date: Mon, 22 Feb 2021 11:15:52 +0800 Subject: [PATCH 6/7] fix integration test --- table/tables/tables_test.go | 4 ---- 1 file changed, 4 deletions(-) diff --git a/table/tables/tables_test.go b/table/tables/tables_test.go index 3b26dd5292b87..101a9e3d65dd6 100644 --- a/table/tables/tables_test.go +++ b/table/tables/tables_test.go @@ -249,10 +249,6 @@ func (ts *testSuite) TestUniqueIndexMultipleNullEntries(c *C) { c.Assert(string(tb.RecordPrefix()), Not(Equals), "") c.Assert(tables.FindIndexByColName(tb, "b"), NotNil) - handle, err := tables.AllocHandle(context.Background(), nil, tb) - c.Assert(err, IsNil) - c.Assert(handle.IntValue(), Greater, int64(0)) - autoid, err := table.AllocAutoIncrementValue(context.Background(), tb, ts.se) c.Assert(err, IsNil) c.Assert(autoid, Greater, int64(0)) From e340180bd43af94b3a128e15da937f0f89a79c1d Mon Sep 17 00:00:00 2001 From: tangenta Date: Wed, 3 Mar 2021 14:06:55 +0800 Subject: [PATCH 7/7] fix integration test --- executor/seqtest/seq_executor_test.go | 14 +++++++------- go.sum | 1 + meta/autoid/autoid.go | 4 ---- table/tables/tables_test.go | 4 ---- 4 files changed, 8 insertions(+), 15 deletions(-) diff --git a/executor/seqtest/seq_executor_test.go b/executor/seqtest/seq_executor_test.go index 56c03272f0536..66bcd4e43b5a4 100644 --- a/executor/seqtest/seq_executor_test.go +++ b/executor/seqtest/seq_executor_test.go @@ -803,17 +803,17 @@ func HelperTestAdminShowNextID(c *C, s *seqTestSuite, str string) { tk.MustExec("create table t(id int, c int)") // Start handle is 1. r := tk.MustQuery(str + " t next_row_id") - r.Check(testkit.Rows("test t _tidb_rowid 1 ")) + r.Check(testkit.Rows("test t _tidb_rowid 1 _tidb_rowid")) // Row ID is step + 1. tk.MustExec("insert into t values(1, 1)") r = tk.MustQuery(str + " t next_row_id") - r.Check(testkit.Rows("test t _tidb_rowid 11 ")) + r.Check(testkit.Rows("test t _tidb_rowid 11 _tidb_rowid")) // Row ID is original + step. for i := 0; i < int(step); i++ { tk.MustExec("insert into t values(10000, 1)") } r = tk.MustQuery(str + " t next_row_id") - r.Check(testkit.Rows("test t _tidb_rowid 21 ")) + r.Check(testkit.Rows("test t _tidb_rowid 21 _tidb_rowid")) tk.MustExec("drop table t") // test for a table with the primary key @@ -858,19 +858,19 @@ func HelperTestAdminShowNextID(c *C, s *seqTestSuite, str string) { // Test for a sequence. tk.MustExec("create sequence seq1 start 15 cache 57") r = tk.MustQuery(str + " seq1 next_row_id") - r.Check(testkit.Rows("test1 seq1 _tidb_rowid 1 ", "test1 seq1 15 SEQUENCE")) + r.Check(testkit.Rows("test1 seq1 _tidb_rowid 1 _tidb_rowid", "test1 seq1 15 SEQUENCE")) r = tk.MustQuery("select nextval(seq1)") r.Check(testkit.Rows("15")) r = tk.MustQuery(str + " seq1 next_row_id") - r.Check(testkit.Rows("test1 seq1 _tidb_rowid 1 ", "test1 seq1 72 SEQUENCE")) + r.Check(testkit.Rows("test1 seq1 _tidb_rowid 1 _tidb_rowid", "test1 seq1 72 SEQUENCE")) r = tk.MustQuery("select nextval(seq1)") r.Check(testkit.Rows("16")) r = tk.MustQuery(str + " seq1 next_row_id") - r.Check(testkit.Rows("test1 seq1 _tidb_rowid 1 ", "test1 seq1 72 SEQUENCE")) + r.Check(testkit.Rows("test1 seq1 _tidb_rowid 1 _tidb_rowid", "test1 seq1 72 SEQUENCE")) r = tk.MustQuery("select setval(seq1, 96)") r.Check(testkit.Rows("96")) r = tk.MustQuery(str + " seq1 next_row_id") - r.Check(testkit.Rows("test1 seq1 _tidb_rowid 1 ", "test1 seq1 97 SEQUENCE")) + r.Check(testkit.Rows("test1 seq1 _tidb_rowid 1 _tidb_rowid", "test1 seq1 97 SEQUENCE")) } func (s *seqTestSuite) TestNoHistoryWhenDisableRetry(c *C) { diff --git a/go.sum b/go.sum index bb2cb68c04133..8200fb5c2835d 100644 --- a/go.sum +++ b/go.sum @@ -465,6 +465,7 @@ github.com/russross/blackfriday v1.5.2/go.mod h1:JO/DiYxRf+HjHt06OyowR9PTA263kcR github.com/russross/blackfriday/v2 v2.0.1/go.mod h1:+Rmxgy9KzJVeS9/2gXHxylqXiyQDYRxCVz55jmeOWTM= github.com/sasha-s/go-deadlock v0.2.0/go.mod h1:StQn567HiB1fF2yJ44N9au7wOhrPS3iZqiDbRupzT10= github.com/satori/go.uuid v1.2.0/go.mod h1:dA0hQrYB0VpLJoorglMZABFdXlWrHn1NEOzdhQKdks0= +github.com/sergi/go-diff v1.0.1-0.20180205163309-da645544ed44 h1:tB9NOR21++IjLyVx3/PCPhWMwqGNCMQEH96A6dMZ/gc= github.com/sergi/go-diff v1.0.1-0.20180205163309-da645544ed44/go.mod h1:0CfEIISq7TuYL3j771MWULgwwjU+GofnZX9QAmXWZgo= github.com/shirou/gopsutil v2.19.10+incompatible/go.mod h1:5b4v6he4MtMOwMlS0TUMTu2PcXUg8+E1lC7eC3UO/RA= github.com/shirou/gopsutil v3.20.12+incompatible h1:6VEGkOXP/eP4o2Ilk8cSsX0PhOEfX6leqAnD+urrp9M= diff --git a/meta/autoid/autoid.go b/meta/autoid/autoid.go index 0fd29deaf3cf2..e9213222735e9 100644 --- a/meta/autoid/autoid.go +++ b/meta/autoid/autoid.go @@ -145,10 +145,6 @@ func (all Allocators) Get(allocType AllocatorType) Allocator { return a } } - // fallback to row id allocator. - if allocType == AutoIncrementType { - return all.Get(RowIDAllocType) - } return nil } diff --git a/table/tables/tables_test.go b/table/tables/tables_test.go index bfe824df0c18b..d5429762345f5 100644 --- a/table/tables/tables_test.go +++ b/table/tables/tables_test.go @@ -136,10 +136,6 @@ func (ts *testSuite) TestBasic(c *C) { c.Assert(err, IsNil) c.Assert(autoID, Greater, int64(0)) - handle, err := tables.AllocHandle(context.Background(), nil, tb) - c.Assert(err, IsNil) - c.Assert(handle.IntValue(), Greater, int64(0)) - ctx := ts.se rid, err := tb.AddRecord(ctx, types.MakeDatums(1, "abc")) c.Assert(err, IsNil)