From 451a3c16540e9efc405f43445d80b8515abb5430 Mon Sep 17 00:00:00 2001 From: tangenta Date: Tue, 28 Sep 2021 11:16:12 +0800 Subject: [PATCH 1/4] *: separate auto_increment ID from tidb row id --- br/pkg/backup/client.go | 85 +++++++++------ br/pkg/kv/kv.go | 3 +- br/pkg/lightning/backend/kv/sql2kv.go | 2 +- br/pkg/lightning/restore/restore.go | 17 ++- br/pkg/lightning/restore/table_restore.go | 5 +- br/pkg/utils/schema.go | 13 ++- cmd/explaintest/r/separate_auto_id.result | 8 ++ cmd/explaintest/t/separate_auto_id.test | 6 ++ ddl/column.go | 36 ++++--- ddl/ddl_api.go | 47 +++++--- ddl/ddl_worker.go | 6 +- ddl/table.go | 24 +++-- executor/infoschema_reader.go | 3 +- executor/insert_common.go | 6 +- executor/show.go | 2 +- executor/write.go | 2 +- go.mod | 2 + go.sum | 7 +- infoschema/builder.go | 124 +++++++++++----------- meta/autoid/autoid.go | 66 ++++++------ meta/autoid/errors.go | 15 ++- meta/autoid/memid.go | 5 - meta/meta_autoid.go | 20 ++-- table/table.go | 6 +- table/tables/tables_test.go | 2 +- 25 files changed, 290 insertions(+), 222 deletions(-) create mode 100644 cmd/explaintest/r/separate_auto_id.result create mode 100644 cmd/explaintest/t/separate_auto_id.test diff --git a/br/pkg/backup/client.go b/br/pkg/backup/client.go index 2a4db4ca2afbc..65bea75399550 100644 --- a/br/pkg/backup/client.go +++ b/br/pkg/backup/client.go @@ -289,39 +289,10 @@ func BuildBackupRangeAndSchema( zap.String("db", dbInfo.Name.O), zap.String("table", tableInfo.Name.O), ) - - tblVer := autoid.AllocOptionTableInfoVersion(tableInfo.Version) - idAlloc := autoid.NewAllocator(storage, dbInfo.ID, tableInfo.ID, false, autoid.RowIDAllocType, tblVer) - seqAlloc := autoid.NewAllocator(storage, dbInfo.ID, tableInfo.ID, false, autoid.SequenceType, tblVer) - randAlloc := autoid.NewAllocator(storage, dbInfo.ID, tableInfo.ID, false, autoid.AutoRandomType, tblVer) - - var globalAutoID int64 - switch { - case tableInfo.IsSequence(): - globalAutoID, err = seqAlloc.NextGlobalAutoID() - case tableInfo.IsView() || !utils.NeedAutoID(tableInfo): - // no auto ID for views or table without either rowID nor auto_increment ID. - default: - globalAutoID, err = idAlloc.NextGlobalAutoID() - } + err := setTableInfoAutoIDs(storage, dbInfo, tableInfo, logger) if err != nil { return nil, nil, errors.Trace(err) } - tableInfo.AutoIncID = globalAutoID - - if tableInfo.PKIsHandle && tableInfo.ContainsAutoRandomBits() { - // this table has auto_random id, we need backup and rebase in restoration - var globalAutoRandID int64 - globalAutoRandID, err = randAlloc.NextGlobalAutoID() - if err != nil { - return nil, nil, errors.Trace(err) - } - tableInfo.AutoRandID = globalAutoRandID - logger.Debug("change table AutoRandID", - zap.Int64("AutoRandID", globalAutoRandID)) - } - logger.Debug("change table AutoIncID", - zap.Int64("AutoIncID", globalAutoID)) // remove all non-public indices n := 0 @@ -355,6 +326,60 @@ func BuildBackupRangeAndSchema( return ranges, backupSchemas, nil } +func setTableInfoAutoIDs(storage kv.Storage, dbInfo *model.DBInfo, tableInfo *model.TableInfo, logger *zap.Logger) error { + if tableInfo.IsView() { + return nil + } + if tableInfo.IsSequence() { + seqAlloc := autoid.NewAllocator(storage, dbInfo.ID, tableInfo.ID, false, autoid.SequenceType) + nextSeqID, err := seqAlloc.NextGlobalAutoID() + if err != nil { + return errors.Trace(err) + } + tableInfo.AutoIncID = nextSeqID + logger.Debug("change sequence AutoIncID", + zap.Int64("sequence nextID", nextSeqID)) + return nil + } + + var autoIDs meta.AutoIDGroup + err := kv.RunInNewTxn(context.Background(), storage, true, func(ctx context.Context, txn kv.Transaction) error { + var err1 error + acc := meta.NewMeta(txn).GetAutoIDAccessors(dbInfo.ID, tableInfo.ID) + autoIDs, err1 = acc.Get() + if err1 != nil { + return errors.Trace(err1) + } + return nil + }) + if err != nil { + return errors.Trace(err) + } + autoIncIDsSeparated := tableInfo.Version >= model.TableInfoVersion5 + if !autoIncIDsSeparated { + tableInfo.AutoIncID = autoIDs.RowID + 1 + logger.Debug("change table AutoIncID", + zap.Int64("AutoIncID", tableInfo.AutoIncID)) + } else { + if utils.HasRowID(tableInfo) { + tableInfo.AutoRowID = autoIDs.RowID + 1 + logger.Debug("change table AutoRowID", + zap.Int64("AutoRowID", tableInfo.AutoRowID)) + } + if utils.HasAutoIncID(tableInfo) { + tableInfo.AutoIncID = autoIDs.IncrementID + 1 + logger.Debug("change table AutoIncID", + zap.Int64("AutoIncID", tableInfo.AutoIncID)) + } + } + if tableInfo.PKIsHandle && tableInfo.ContainsAutoRandomBits() { + tableInfo.AutoRandID = autoIDs.RandomID + 1 + logger.Debug("change table AutoRandID", + zap.Int64("AutoRandID", tableInfo.AutoRandID)) + } + return nil +} + // WriteBackupDDLJobs sends the ddl jobs are done in (lastBackupTS, backupTS] to metaWriter. func WriteBackupDDLJobs(metaWriter *metautil.MetaWriter, store kv.Storage, lastBackupTS, backupTS uint64) error { snapshot := store.GetSnapshot(kv.NewVersion(backupTS)) diff --git a/br/pkg/kv/kv.go b/br/pkg/kv/kv.go index 229255b0d5867..736957e64540b 100644 --- a/br/pkg/kv/kv.go +++ b/br/pkg/kv/kv.go @@ -353,7 +353,8 @@ func (kvcodec *tableKVEncoder) AddRecord( _ = alloc.Rebase(value.GetInt64()&((1< newLayout.IncrementalBits { overflowCnt := usedBits - newLayout.IncrementalBits errMsg := fmt.Sprintf(autoid.AutoRandomOverflowErrMsg, newAutoRandBits-overflowCnt, newAutoRandBits, oldCol.Name.O) @@ -1637,28 +1640,29 @@ func checkNewAutoRandomBits(idAccessors meta.AutoIDAccessors, oldCol *model.Colu // applyNewAutoRandomBits set auto_random bits to TableInfo and // migrate auto_increment ID to auto_random ID if possible. -func applyNewAutoRandomBits(d *ddlCtx, m *meta.Meta, dbInfo *model.DBInfo, +func applyNewAutoRandomBits(m *meta.Meta, dbInfo *model.DBInfo, tblInfo *model.TableInfo, oldCol *model.ColumnInfo, newAutoRandBits uint64) error { tblInfo.AutoRandomBits = newAutoRandBits needMigrateFromAutoIncToAutoRand := mysql.HasAutoIncrementFlag(oldCol.Flag) if !needMigrateFromAutoIncToAutoRand { return nil } - autoRandAlloc := autoid.NewAllocatorsFromTblInfo(d.store, dbInfo.ID, tblInfo).Get(autoid.AutoRandomType) - if autoRandAlloc == nil { - errMsg := fmt.Sprintf(autoid.AutoRandomAllocatorNotFound, dbInfo.Name.O, tblInfo.Name.O) - return ErrInvalidAutoRandom.GenWithStackByArgs(errMsg) + idAcc := m.GetAutoIDAccessors(dbInfo.ID, tblInfo.ID) + var autoIncAcc meta.AutoIDAccessor + if model.AutoIncrementIDIsSeparated(tblInfo.Version) { + autoIncAcc = idAcc.IncrementID() + } else { + autoIncAcc = idAcc.RowID() } - idAcc := m.GetAutoIDAccessors(dbInfo.ID, tblInfo.ID).RowID() - nextAutoIncID, err := idAcc.Get() + nextAutoIncID, err := autoIncAcc.Get() if err != nil { return errors.Trace(err) } - err = autoRandAlloc.Rebase(nextAutoIncID, false) - if err != nil { + if err := autoIncAcc.Del(); err != nil { return errors.Trace(err) } - if err := idAcc.Del(); err != nil { + err = idAcc.RandomID().Put(nextAutoIncID) + if err != nil { return errors.Trace(err) } return nil diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index 6715fd1a3b666..80ce5027b1d9e 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -2003,17 +2003,24 @@ func (d *ddl) CreateTableWithInfo( } } else if actionType == model.ActionCreateTable { d.preSplitAndScatter(ctx, tbInfo, tbInfo.GetPartitionInfo()) - if tbInfo.AutoIncID > 1 { - // Default tableAutoIncID base is 0. - // If the first ID is expected to greater than 1, we need to do rebase. + if tbInfo.AutoRowID > 1 { newEnd := tbInfo.AutoIncID - 1 if err = d.handleAutoIncID(tbInfo, schema.ID, newEnd, autoid.RowIDAllocType); err != nil { return errors.Trace(err) } } + if tbInfo.AutoIncID > 1 { + newEnd := tbInfo.AutoIncID - 1 + if model.AutoIncrementIDIsSeparated(tbInfo.Version) { + err = d.handleAutoIncID(tbInfo, schema.ID, newEnd, autoid.AutoIncrementType) + } else { + err = d.handleAutoIncID(tbInfo, schema.ID, newEnd, autoid.RowIDAllocType) + } + if err != nil { + return errors.Trace(err) + } + } if tbInfo.AutoRandID > 1 { - // Default tableAutoRandID base is 0. - // If the first ID is expected to greater than 1, we need to do rebase. newEnd := tbInfo.AutoRandID - 1 err = d.handleAutoIncID(tbInfo, schema.ID, newEnd, autoid.AutoRandomType) } @@ -2674,16 +2681,14 @@ func (d *ddl) AlterTable(ctx context.Context, sctx sessionctx.Context, ident ast opt.UintValue = shardRowIDBitsMax } err = d.ShardRowID(sctx, ident, opt.UintValue) - case ast.TableOptionAutoIncrement: - err = d.RebaseAutoID(sctx, ident, int64(opt.UintValue), autoid.RowIDAllocType, opt.BoolValue) + case ast.TableOptionRowID, ast.TableOptionAutoIncrement, ast.TableOptionAutoRandomBase: + err = d.RebaseAutoID(sctx, ident, int64(opt.UintValue), opt.Tp, opt.BoolValue) case ast.TableOptionAutoIdCache: if opt.UintValue > uint64(math.MaxInt64) { // TODO: Refine this error. return errors.New("table option auto_id_cache overflows int64") } err = d.AlterTableAutoIDCache(sctx, ident, int64(opt.UintValue)) - case ast.TableOptionAutoRandomBase: - err = d.RebaseAutoID(sctx, ident, int64(opt.UintValue), autoid.AutoRandomType, opt.BoolValue) case ast.TableOptionComment: spec.Comment = opt.StrValue err = d.AlterTableComment(sctx, ident, spec) @@ -2744,14 +2749,17 @@ func (d *ddl) AlterTable(ctx context.Context, sctx sessionctx.Context, ident ast return nil } -func (d *ddl) RebaseAutoID(ctx sessionctx.Context, ident ast.Ident, newBase int64, tp autoid.AllocatorType, force bool) error { +func (d *ddl) RebaseAutoID(ctx sessionctx.Context, ident ast.Ident, newBase int64, tp ast.TableOptionType, force bool) error { schema, t, err := d.getSchemaAndTableByIdent(ctx, ident) if err != nil { return errors.Trace(err) } - var actionType model.ActionType + var ( + actionType model.ActionType + allocType autoid.AllocatorType + ) switch tp { - case autoid.AutoRandomType: + case ast.TableOptionAutoRandomBase: tbInfo := t.Meta() if tbInfo.AutoRandomBits == 0 { return errors.Trace(ErrInvalidAutoRandom.GenWithStackByArgs(autoid.AutoRandomRebaseNotApplicable)) @@ -2769,12 +2777,21 @@ func (d *ddl) RebaseAutoID(ctx sessionctx.Context, ident ast.Ident, newBase int6 return errors.Trace(ErrInvalidAutoRandom.GenWithStackByArgs(errMsg)) } actionType = model.ActionRebaseAutoRandomBase - case autoid.RowIDAllocType: + allocType = autoid.AutoRandomType + case ast.TableOptionAutoIncrement: actionType = model.ActionRebaseAutoID + tbInfo := t.Meta() + if model.AutoIncrementIDIsSeparated(tbInfo.Version) { + allocType = autoid.AutoIncrementType + } else { + allocType = autoid.RowIDAllocType + } + case ast.TableOptionRowID: + actionType = model.ActionRebaseRowID + allocType = autoid.RowIDAllocType } - if !force { - newBase, err = adjustNewBaseToNextGlobalID(ctx, t, tp, newBase) + newBase, err = adjustNewBaseToNextGlobalID(ctx, t, allocType, newBase) if err != nil { return err } diff --git a/ddl/ddl_worker.go b/ddl/ddl_worker.go index a7c253b5ddbdf..637d4dba839d5 100644 --- a/ddl/ddl_worker.go +++ b/ddl/ddl_worker.go @@ -786,10 +786,8 @@ func (w *worker) runDDLJob(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, ver, err = onDropForeignKey(t, job) case model.ActionTruncateTable: ver, err = onTruncateTable(d, t, job) - case model.ActionRebaseAutoID: - ver, err = onRebaseRowIDType(d.store, t, job) - case model.ActionRebaseAutoRandomBase: - ver, err = onRebaseAutoRandomType(d.store, t, job) + case model.ActionRebaseAutoID, model.ActionRebaseAutoRandomBase, model.ActionRebaseRowID: + ver, err = onRebaseAutoID(d.store, t, job) case model.ActionRenameTable: ver, err = onRenameTable(d, t, job) case model.ActionShardRowID: diff --git a/ddl/table.go b/ddl/table.go index 507ca5b404451..5b430d037c241 100644 --- a/ddl/table.go +++ b/ddl/table.go @@ -654,15 +654,7 @@ 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 onRebaseAutoRandomType(store kv.Storage, t *meta.Meta, job *model.Job) (ver int64, _ error) { - return onRebaseAutoID(store, t, job, autoid.AutoRandomType) -} - -func onRebaseAutoID(store kv.Storage, t *meta.Meta, job *model.Job, tp autoid.AllocatorType) (ver int64, _ error) { +func onRebaseAutoID(store kv.Storage, t *meta.Meta, job *model.Job) (ver int64, _ error) { schemaID := job.SchemaID var ( newBase int64 @@ -679,10 +671,20 @@ 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 { + var tp autoid.AllocatorType + switch job.Type { + case model.ActionRebaseAutoID: tblInfo.AutoIncID = newBase - } else { + if model.AutoIncrementIDIsSeparated(tblInfo.Version) { + tp = autoid.AutoIncrementType + } else { + tp = autoid.RowIDAllocType + } + case model.ActionRebaseAutoRandomBase: tblInfo.AutoRandID = newBase + case model.ActionRebaseRowID: + tblInfo.AutoRowID = newBase + tp = autoid.RowIDAllocType } tbl, err := getTable(store, schemaID, tblInfo) diff --git a/executor/infoschema_reader.go b/executor/infoschema_reader.go index 072404fa3437e..303dd5a67e08d 100644 --- a/executor/infoschema_reader.go +++ b/executor/infoschema_reader.go @@ -45,7 +45,6 @@ import ( "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/kv" - "github.com/pingcap/tidb/meta/autoid" plannercore "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/privilege" "github.com/pingcap/tidb/session/txninfo" @@ -317,7 +316,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).GetAutoIncrement(tblInfo.Version).Base() + 1, nil } func hasPriv(ctx sessionctx.Context, priv mysql.PrivilegeType) bool { diff --git a/executor/insert_common.go b/executor/insert_common.go index 72ed1f51b584f..06a2f6ddd342a 100644 --- a/executor/insert_common.go +++ b/executor/insert_common.go @@ -711,7 +711,8 @@ 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.Allocators(e.ctx).Get(autoid.RowIDAllocType).Rebase(recordID, true) + ver := e.Table.Meta().Version + err = e.Table.Allocators(e.ctx).GetAutoIncrement(ver).Rebase(recordID, true) if err != nil { return nil, err } @@ -801,7 +802,8 @@ 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.Allocators(e.ctx).Get(autoid.RowIDAllocType).Rebase(recordID, true) + ver := e.Table.Meta().Version + err = e.Table.Allocators(e.ctx).GetAutoIncrement(ver).Rebase(recordID, true) if err != nil { return types.Datum{}, err } diff --git a/executor/show.go b/executor/show.go index b22c7bf4543aa..8cd097e315695 100644 --- a/executor/show.go +++ b/executor/show.go @@ -1025,7 +1025,7 @@ func ConstructResultOfShowCreateTable(ctx sessionctx.Context, tableInfo *model.T fmt.Fprintf(buf, " COMPRESSION='%s'", tableInfo.Compression) } - incrementAllocator := allocators.Get(autoid.RowIDAllocType) + incrementAllocator := allocators.GetAutoIncrement(tableInfo.Version) if hasAutoIncID && incrementAllocator != nil { autoIncID, err := incrementAllocator.NextGlobalAutoID() if err != nil { diff --git a/executor/write.go b/executor/write.go index 28925957bd764..ffaf74d63ea61 100644 --- a/executor/write.go +++ b/executor/write.go @@ -111,7 +111,7 @@ func updateRecord(ctx context.Context, sctx sessionctx.Context, h kv.Handle, old if err != nil { return false, err } - if err = t.Allocators(sctx).Get(autoid.RowIDAllocType).Rebase(recordID, true); err != nil { + if err = t.Allocators(sctx).GetAutoIncrement(t.Meta().Version).Rebase(recordID, true); err != nil { return false, err } } diff --git a/go.mod b/go.mod index 7787e47638e2a..6cb3c6b0809d4 100644 --- a/go.mod +++ b/go.mod @@ -96,3 +96,5 @@ require ( // cloud.google.com/go/storage will upgrade grpc to v1.40.0 // we need keep the replacement until go.etcd.io supports the higher version of grpc. replace google.golang.org/grpc => google.golang.org/grpc v1.29.1 + +replace github.com/pingcap/parser => github.com/tangenta/parser v0.0.0-20210926095652-e5410e656f1f diff --git a/go.sum b/go.sum index 145d1d5ecbdf5..93519808b6e93 100644 --- a/go.sum +++ b/go.sum @@ -577,7 +577,6 @@ github.com/pingcap/errors v0.11.0/go.mod h1:Oi8TUi2kEtXXLMJk9l1cGmz20kV3TaQ0usTw github.com/pingcap/errors v0.11.4/go.mod h1:Oi8TUi2kEtXXLMJk9l1cGmz20kV3TaQ0usTwv5KuLY8= github.com/pingcap/errors v0.11.5-0.20190809092503-95897b64e011/go.mod h1:Oi8TUi2kEtXXLMJk9l1cGmz20kV3TaQ0usTwv5KuLY8= github.com/pingcap/errors v0.11.5-0.20200917111840-a15ef68f753d/go.mod h1:g4vx//d6VakjJ0mk7iLBlKA8LFavV/sAVINT/1PFxeQ= -github.com/pingcap/errors v0.11.5-0.20201029093017-5a7df2af2ac7/go.mod h1:G7x87le1poQzLB/TqvTJI2ILrSgobnq4Ut7luOwvfvI= github.com/pingcap/errors v0.11.5-0.20201126102027-b0a155152ca3/go.mod h1:G7x87le1poQzLB/TqvTJI2ILrSgobnq4Ut7luOwvfvI= github.com/pingcap/errors v0.11.5-0.20210425183316-da1aaba5fb63 h1:+FZIDR/D97YOPik4N4lPDaUcLDF/EQPogxtlHB2ZZRM= github.com/pingcap/errors v0.11.5-0.20210425183316-da1aaba5fb63/go.mod h1:X2r9ueLEUZgtx2cIogM0v4Zj5uvvzhuuiu7Pn8HzMPg= @@ -602,9 +601,6 @@ github.com/pingcap/log v0.0.0-20210317133921-96f4fcab92a4/go.mod h1:4rbK1p9ILyIf github.com/pingcap/log v0.0.0-20210625125904-98ed8e2eb1c7/go.mod h1:8AanEdAHATuRurdGxZXBz0At+9avep+ub7U1AGYLIMM= github.com/pingcap/log v0.0.0-20210906054005-afc726e70354 h1:SvWCbCPh1YeHd9yQLksvJYAgft6wLTY1aNG81tpyscQ= github.com/pingcap/log v0.0.0-20210906054005-afc726e70354/go.mod h1:DWQW5jICDR7UJh4HtxXSM20Churx4CQL0fwL/SoOSA4= -github.com/pingcap/parser v0.0.0-20210525032559-c37778aff307/go.mod h1:xZC8I7bug4GJ5KtHhgAikjTfU4kBv1Sbo3Pf1MZ6lVw= -github.com/pingcap/parser v0.0.0-20210917114242-ac711116bdff h1:LiwvvutmyeSkFkdVM09mH6KK+OeDVJzX7WKy9Lf0ri0= -github.com/pingcap/parser v0.0.0-20210917114242-ac711116bdff/go.mod h1:+xcMiiZzdIktT/Nqdfm81dkECJ2EPuoAYywd57py4Pk= github.com/pingcap/sysutil v0.0.0-20200206130906-2bfa6dc40bcd/go.mod h1:EB/852NMQ+aRKioCpToQ94Wl7fktV+FNnxf3CX/TTXI= github.com/pingcap/sysutil v0.0.0-20210315073920-cc0985d983a3/go.mod h1:tckvA041UWP+NqYzrJ3fMgC/Hw9wnmQ/tUkp/JaHly8= github.com/pingcap/sysutil v0.0.0-20210730114356-fcd8a63f68c5 h1:7rvAtZe/ZUzOKzgriNPQoBNvleJXBk4z7L3Z47+tS98= @@ -651,7 +647,6 @@ github.com/prometheus/procfs v0.0.5/go.mod h1:4A/X28fw3Fc593LaREMrKMqOKvUAntwMDa github.com/prometheus/procfs v0.0.8 h1:+fpWZdT24pJBiqJdAwYBjPSk+5YmQzYNPYzQsdzLkt8= github.com/prometheus/procfs v0.0.8/go.mod h1:7Qr8sr6344vo1JqZ6HhLceV9o3AJ1Ff+GxbHq6oeK9A= github.com/prometheus/tsdb v0.7.1/go.mod h1:qhTCs0VvXwvX/y3TZrWD7rabWM+ijKTux40TwIPHuXU= -github.com/remyoudompheng/bigfft v0.0.0-20190728182440-6a916e37a237/go.mod h1:qqbHyh8v60DhA7CoWK5oRCqLrMHRGoxYCSS9EjAz6Eo= github.com/remyoudompheng/bigfft v0.0.0-20200410134404-eec4a21b6bb0 h1:OdAsTTz6OkFY5QxjkYwrChwuRruF69c169dPK26NUlk= github.com/remyoudompheng/bigfft v0.0.0-20200410134404-eec4a21b6bb0/go.mod h1:qqbHyh8v60DhA7CoWK5oRCqLrMHRGoxYCSS9EjAz6Eo= github.com/rivo/uniseg v0.1.0/go.mod h1:J6wj4VEh+S6ZtnVlnTBMWIodfgj8LQOQFoIToxlJtxc= @@ -724,6 +719,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-20210926095652-e5410e656f1f h1:tpUeckXAum2tzVqlp8CS1+2pFuOd7BnQk+9HkgVmxZI= +github.com/tangenta/parser v0.0.0-20210926095652-e5410e656f1f/go.mod h1:+xcMiiZzdIktT/Nqdfm81dkECJ2EPuoAYywd57py4Pk= github.com/thoas/go-funk v0.7.0/go.mod h1:+IWnUfUmFO1+WVYQWQtIJHeRRdaIyyYglZN7xzUPe4Q= github.com/thoas/go-funk v0.8.0/go.mod h1:+IWnUfUmFO1+WVYQWQtIJHeRRdaIyyYglZN7xzUPe4Q= github.com/tiancaiamao/appdash v0.0.0-20181126055449-889f96f722a2 h1:mbAskLJ0oJfDRtkanvQPiooDH8HvJ2FBh+iKT/OmiQQ= diff --git a/infoschema/builder.go b/infoschema/builder.go index f3c69e3e46761..0bee05bdfdb6d 100644 --- a/infoschema/builder.go +++ b/infoschema/builder.go @@ -106,22 +106,12 @@ func (b *Builder) ApplyDiff(m *meta.Meta, diff *model.SchemaDiff) ([]int64, erro b.copySortedTables(oldTableID, newTableID) tblIDs := make([]int64, 0, 2) - // We try to reuse the old allocator, so the cached auto ID can be reused. - var allocs autoid.Allocators + + var oldTbl table.Table if tableIDIsValid(oldTableID) { - if oldTableID == newTableID && diff.Type != model.ActionRenameTable && - diff.Type != model.ActionExchangeTablePartition && - // For repairing table in TiDB cluster, given 2 normal node and 1 repair node. - // For normal node's information schema, repaired table is existed. - // For repair node's information schema, repaired table is filtered (couldn't find it in `is`). - // So here skip to reserve the allocators when repairing table. - diff.Type != model.ActionRepairTable && - // Alter sequence will change the sequence info in the allocator, so the old allocator is not valid any more. - diff.Type != model.ActionAlterSequence { - oldAllocs, _ := b.is.AllocByID(oldTableID) - allocs = filterAllocators(diff, oldAllocs) + if oldTableID == newTableID { + oldTbl, _ = b.is.TableByID(oldTableID) } - tmpIDs := tblIDs if (diff.Type == model.ActionRenameTable || diff.Type == model.ActionRenameTables) && diff.OldSchemaID != diff.SchemaID { oldRoDBInfo, ok := b.is.SchemaByID(diff.OldSchemaID) @@ -144,7 +134,7 @@ func (b *Builder) ApplyDiff(m *meta.Meta, diff *model.SchemaDiff) ([]int64, erro if tableIDIsValid(newTableID) { // All types except DropTableOrView. var err error - tblIDs, err = b.applyCreateTable(m, dbInfo, newTableID, allocs, diff.Type, tblIDs) + tblIDs, err = b.applyCreateTable(m, dbInfo, newTableID, oldTbl, diff.Type, tblIDs) if err != nil { return nil, errors.Trace(err) } @@ -209,28 +199,6 @@ func (b *Builder) ApplyDiff(m *meta.Meta, diff *model.SchemaDiff) ([]int64, erro return tblIDs, nil } -func filterAllocators(diff *model.SchemaDiff, oldAllocs autoid.Allocators) autoid.Allocators { - var newAllocs autoid.Allocators - switch diff.Type { - case model.ActionRebaseAutoID, model.ActionModifyTableAutoIdCache: - // Only drop auto-increment allocator. - newAllocs = oldAllocs.Filter(func(a autoid.Allocator) bool { - tp := a.GetType() - return tp != autoid.RowIDAllocType && tp != autoid.AutoIncrementType - }) - case model.ActionRebaseAutoRandomBase: - // Only drop auto-random allocator. - newAllocs = oldAllocs.Filter(func(a autoid.Allocator) bool { - tp := a.GetType() - return tp != autoid.AutoRandomType - }) - default: - // Keep all allocators. - newAllocs = oldAllocs - } - return newAllocs -} - func appendAffectedIDs(affected []int64, tblInfo *model.TableInfo) []int64 { affected = append(affected, tblInfo.ID) if pi := tblInfo.GetPartitionInfo(); pi != nil { @@ -360,7 +328,7 @@ func (b *Builder) copySortedTablesBucket(bucketIdx int) { b.is.sortedTablesBuckets[bucketIdx] = newSortedTables } -func (b *Builder) applyCreateTable(m *meta.Meta, dbInfo *model.DBInfo, tableID int64, allocs autoid.Allocators, tp model.ActionType, affected []int64) ([]int64, error) { +func (b *Builder) applyCreateTable(m *meta.Meta, dbInfo *model.DBInfo, tableID int64, oldTbl table.Table, tp model.ActionType, affected []int64) ([]int64, error) { tblInfo, err := m.GetTable(dbInfo.ID, tableID) if err != nil { return nil, errors.Trace(err) @@ -406,29 +374,7 @@ func (b *Builder) applyCreateTable(m *meta.Meta, dbInfo *model.DBInfo, tableID i ConvertCharsetCollateToLowerCaseIfNeed(tblInfo) ConvertOldVersionUTF8ToUTF8MB4IfNeed(tblInfo) - if len(allocs) == 0 { - allocs = autoid.NewAllocatorsFromTblInfo(b.store, dbInfo.ID, tblInfo) - } else { - tblVer := autoid.AllocOptionTableInfoVersion(tblInfo.Version) - switch tp { - case model.ActionRebaseAutoID, model.ActionModifyTableAutoIdCache: - newAlloc := autoid.NewAllocator(b.store, dbInfo.ID, tblInfo.ID, tblInfo.IsAutoIncColUnsigned(), autoid.RowIDAllocType, tblVer) - allocs = append(allocs, newAlloc) - case model.ActionRebaseAutoRandomBase: - newAlloc := autoid.NewAllocator(b.store, dbInfo.ID, tblInfo.ID, tblInfo.IsAutoRandomBitColUnsigned(), autoid.AutoRandomType, tblVer) - allocs = append(allocs, newAlloc) - case model.ActionModifyColumn: - // Change column attribute from auto_increment to auto_random. - if tblInfo.ContainsAutoRandomBits() && allocs.Get(autoid.AutoRandomType) == nil { - // Remove auto_increment allocator. - allocs = allocs.Filter(func(a autoid.Allocator) bool { - return a.GetType() != autoid.AutoIncrementType && a.GetType() != autoid.RowIDAllocType - }) - newAlloc := autoid.NewAllocator(b.store, dbInfo.ID, tblInfo.ID, tblInfo.IsAutoRandomBitColUnsigned(), autoid.AutoRandomType, tblVer) - allocs = append(allocs, newAlloc) - } - } - } + allocs := reconstructAllocators(b.store, dbInfo.ID, oldTbl, tblInfo, tp) tbl, err := tables.TableFromMeta(allocs, tblInfo) if err != nil { return nil, errors.Trace(err) @@ -448,6 +394,62 @@ func (b *Builder) applyCreateTable(m *meta.Meta, dbInfo *model.DBInfo, tableID i return affected, nil } +// reconstructAllocators tries to reuse the old allocator so that the cached auto ID can be reused. +func reconstructAllocators(s kv.Storage, dbID int64, oldTbl table.Table, newTbl *model.TableInfo, + tp model.ActionType) autoid.Allocators { + if oldTbl == nil { + return autoid.NewAllocatorsFromTblInfo(s, dbID, newTbl) + } + switch tp { + case model.ActionRenameTable, model.ActionExchangeTablePartition, model.ActionRepairTable, + model.ActionAlterSequence: + return autoid.NewAllocatorsFromTblInfo(s, dbID, newTbl) + } + // Try to reuse the old allocators. + as := oldTbl.Allocators(nil) + switch tp { + case model.ActionRebaseRowID: + as = as.Remove(autoid.RowIDAllocType) + rowID := autoid.NewAllocator(s, dbID, newTbl.ID, false, autoid.RowIDAllocType) + as = append(as, rowID) + return as + case model.ActionRebaseAutoID, model.ActionModifyTableAutoIdCache: + unsigned := newTbl.IsAutoIncColUnsigned() + if model.AutoIncrementIDIsSeparated(newTbl.Version) { + as = as.Remove(autoid.AutoIncrementType) + incID := autoid.NewAllocator(s, dbID, newTbl.ID, unsigned, autoid.AutoIncrementType) + as = append(as, incID) + return as + } + as = as.Remove(autoid.RowIDAllocType).Remove(autoid.AutoIncrementType) + rowID := autoid.NewAllocator(s, dbID, newTbl.ID, unsigned, autoid.RowIDAllocType) + as = append(as, rowID) + return as + case model.ActionRebaseAutoRandomBase: + unsigned := newTbl.IsAutoRandomBitColUnsigned() + as = as.Remove(autoid.AutoRandomType) + randID := autoid.NewAllocator(s, dbID, newTbl.ID, unsigned, autoid.AutoRandomType) + as = append(as, randID) + return as + case model.ActionModifyColumn: + // Change column attribute from auto_increment to auto_random. + if !oldTbl.Meta().ContainsAutoRandomBits() && newTbl.ContainsAutoRandomBits() { + unsigned := newTbl.IsAutoRandomBitColUnsigned() + if model.AutoIncrementIDIsSeparated(newTbl.Version) { + as = as.Remove(autoid.AutoIncrementType) + randID := autoid.NewAllocator(s, dbID, newTbl.ID, unsigned, autoid.AutoRandomType) + as = append(as, randID) + return as + } + as = as.Remove(autoid.RowIDAllocType).Remove(autoid.AutoIncrementType) + randID := autoid.NewAllocator(s, dbID, newTbl.ID, unsigned, autoid.AutoRandomType) + as = append(as, randID) + return as + } + } + return as +} + // ConvertCharsetCollateToLowerCaseIfNeed convert the charset / collation of table and its columns to lower case, // if the table's version is prior to TableInfoVersion3. func ConvertCharsetCollateToLowerCaseIfNeed(tbInfo *model.TableInfo) { diff --git a/meta/autoid/autoid.go b/meta/autoid/autoid.go index 0700b8ef35a38..d02f6348cc446 100644 --- a/meta/autoid/autoid.go +++ b/meta/autoid/autoid.go @@ -109,14 +109,6 @@ func (step CustomAutoIncCacheOption) ApplyOn(alloc *allocator) { alloc.customStep = true } -// AllocOptionTableInfoVersion is used to pass the TableInfo.Version to the allocator. -type AllocOptionTableInfoVersion uint16 - -// ApplyOn implements the AllocOption interface. -func (v AllocOptionTableInfoVersion) ApplyOn(alloc *allocator) { - alloc.tbVersion = uint16(v) -} - // AllocOption is a interface to define allocator custom options coming in future. type AllocOption interface { ApplyOn(*allocator) @@ -177,15 +169,28 @@ func (all Allocators) Get(allocType AllocatorType) Allocator { return nil } -// Filter filters all the allocators that match pred. -func (all Allocators) Filter(pred func(Allocator) bool) Allocators { - var ret Allocators - for _, a := range all { - if pred(a) { - ret = append(ret, a) +// GetAutoIncrement gets the auto_increment ID allocator. +func (all Allocators) GetAutoIncrement(tableVersion uint16) Allocator { + if model.AutoIncrementIDIsSeparated(tableVersion) { + return all.Get(AutoIncrementType) + } + return all.Get(RowIDAllocType) +} + +// Remove remove all the allocators that match tp. +func (all Allocators) Remove(tp AllocatorType) Allocators { + total := all + cur := 0 + for cur < len(total) { + last := len(total) - 1 + if total[cur].GetType() == tp { + total[cur], total[last] = total[last], total[cur] + total = total[:len(total)-1] + } else { + cur++ } } - return ret + return total } type allocator struct { @@ -196,7 +201,6 @@ type allocator struct { // dbID is current database's ID. dbID int64 tbID int64 - tbVersion uint16 isUnsigned bool lastAllocTime time.Time step int64 @@ -493,24 +497,27 @@ func NewSequenceAllocator(store kv.Storage, dbID, tbID int64, info *model.Sequen // 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 allocs []Allocator dbID := tblInfo.GetDBID(schemaID) + var allocs []Allocator + if tblInfo.IsSequence() { + allocs = append(allocs, NewSequenceAllocator(store, dbID, tblInfo.ID, tblInfo.Sequence)) + return allocs + } idCacheOpt := CustomAutoIncCacheOption(tblInfo.AutoIdCache) - tblVer := AllocOptionTableInfoVersion(tblInfo.Version) - hasRowID := !tblInfo.PKIsHandle && !tblInfo.IsCommonHandle hasAutoIncID := tblInfo.GetAutoIncrementColInfo() != nil - if hasRowID || hasAutoIncID { - alloc := NewAllocator(store, dbID, tblInfo.ID, tblInfo.IsAutoIncColUnsigned(), RowIDAllocType, idCacheOpt, tblVer) + separated := model.AutoIncrementIDIsSeparated(tblInfo.Version) + if hasRowID || (hasAutoIncID && !separated) { + alloc := NewAllocator(store, dbID, tblInfo.ID, false, RowIDAllocType, idCacheOpt) allocs = append(allocs, alloc) } - hasAutoRandID := tblInfo.ContainsAutoRandomBits() - if hasAutoRandID { - alloc := NewAllocator(store, dbID, tblInfo.ID, tblInfo.IsAutoRandomBitColUnsigned(), AutoRandomType, idCacheOpt, tblVer) + if hasAutoIncID && separated { + alloc := NewAllocator(store, dbID, tblInfo.ID, tblInfo.IsAutoIncColUnsigned(), AutoIncrementType, idCacheOpt) allocs = append(allocs, alloc) } - if tblInfo.IsSequence() { - allocs = append(allocs, NewSequenceAllocator(store, dbID, tblInfo.ID, tblInfo.Sequence)) + if tblInfo.ContainsAutoRandomBits() { + alloc := NewAllocator(store, dbID, tblInfo.ID, tblInfo.IsAutoRandomBitColUnsigned(), AutoRandomType, idCacheOpt) + allocs = append(allocs, alloc) } return NewAllocators(allocs...) } @@ -535,11 +542,6 @@ func (alloc *allocator) Alloc(ctx context.Context, n uint64, increment, offset i if n == 0 { return 0, 0, nil } - if alloc.allocType == AutoIncrementType || alloc.allocType == RowIDAllocType { - if !validIncrementAndOffset(increment, offset) { - return 0, 0, errInvalidIncrementAndOffset.GenWithStackByArgs(increment, offset) - } - } alloc.mu.Lock() defer alloc.mu.Unlock() if alloc.isUnsigned { @@ -954,7 +956,7 @@ func (alloc *allocator) getIDAccessor(txn kv.Transaction) meta.AutoIDAccessor { case RowIDAllocType: return acc.RowID() case AutoIncrementType: - return acc.IncrementID(alloc.tbVersion) + return acc.IncrementID() case AutoRandomType: return acc.RandomID() case SequenceType: diff --git a/meta/autoid/errors.go b/meta/autoid/errors.go index 90d9fcc6a310e..11d6e670d9f97 100644 --- a/meta/autoid/errors.go +++ b/meta/autoid/errors.go @@ -21,13 +21,12 @@ import ( // Error instances. var ( - errInvalidTableID = dbterror.ClassAutoid.NewStd(mysql.ErrInvalidTableID) - errInvalidIncrementAndOffset = dbterror.ClassAutoid.NewStd(mysql.ErrInvalidIncrementAndOffset) - errNotImplemented = dbterror.ClassAutoid.NewStd(mysql.ErrNotImplemented) - ErrAutoincReadFailed = dbterror.ClassAutoid.NewStd(mysql.ErrAutoincReadFailed) - ErrWrongAutoKey = dbterror.ClassAutoid.NewStd(mysql.ErrWrongAutoKey) - ErrInvalidAllocatorType = dbterror.ClassAutoid.NewStd(mysql.ErrUnknownAllocatorType) - ErrAutoRandReadFailed = dbterror.ClassAutoid.NewStd(mysql.ErrAutoRandReadFailed) + errInvalidTableID = dbterror.ClassAutoid.NewStd(mysql.ErrInvalidTableID) + errNotImplemented = dbterror.ClassAutoid.NewStd(mysql.ErrNotImplemented) + ErrAutoincReadFailed = dbterror.ClassAutoid.NewStd(mysql.ErrAutoincReadFailed) + ErrWrongAutoKey = dbterror.ClassAutoid.NewStd(mysql.ErrWrongAutoKey) + ErrInvalidAllocatorType = dbterror.ClassAutoid.NewStd(mysql.ErrUnknownAllocatorType) + ErrAutoRandReadFailed = dbterror.ClassAutoid.NewStd(mysql.ErrAutoRandReadFailed) ) const ( @@ -61,6 +60,4 @@ const ( AutoRandomAlterAddColumn = "unsupported add column '%s' constraint AUTO_RANDOM when altering '%s.%s'" // AutoRandomAlterChangeFromAutoInc is reported when the column is changing from a non-auto_increment or a non-primary key. AutoRandomAlterChangeFromAutoInc = "auto_random can only be converted from auto_increment clustered primary key" - // AutoRandomAllocatorNotFound is reported when auto_random ID allocator not found during changing from auto_inc to auto_random. - AutoRandomAllocatorNotFound = "auto_random ID allocator not found in table '%s.%s'" ) diff --git a/meta/autoid/memid.go b/meta/autoid/memid.go index 1a9af524959b2..a499f34e8a5da 100644 --- a/meta/autoid/memid.go +++ b/meta/autoid/memid.go @@ -72,11 +72,6 @@ func (alloc *inMemoryAllocator) Alloc(ctx context.Context, n uint64, increment, if n == 0 { return 0, 0, nil } - if alloc.allocType == AutoIncrementType || alloc.allocType == RowIDAllocType { - if !validIncrementAndOffset(increment, offset) { - return 0, 0, errInvalidIncrementAndOffset.GenWithStackByArgs(increment, offset) - } - } if alloc.isUnsigned { return alloc.alloc4Unsigned(n, increment, offset) } diff --git a/meta/meta_autoid.go b/meta/meta_autoid.go index 8df7c1bdb51a3..bdb6dc504dd06 100644 --- a/meta/meta_autoid.go +++ b/meta/meta_autoid.go @@ -18,7 +18,6 @@ import ( "strconv" "github.com/pingcap/errors" - "github.com/pingcap/parser/model" ) var _ AutoIDAccessor = &autoIDAccessor{} @@ -92,7 +91,7 @@ type AutoIDAccessors interface { type AccessorPicker interface { RowID() AutoIDAccessor RandomID() AutoIDAccessor - IncrementID(tableVersion uint16) AutoIDAccessor + IncrementID() AutoIDAccessor SequenceValue() AutoIDAccessor SequenceCycle() AutoIDAccessor @@ -102,14 +101,12 @@ type autoIDAccessors struct { access autoIDAccessor } -const sepAutoIncVer = model.TableInfoVersion4 + 1 - // Get implements the interface AutoIDAccessors. func (a *autoIDAccessors) Get() (autoIDs AutoIDGroup, err error) { if autoIDs.RowID, err = a.RowID().Get(); err != nil { return autoIDs, err } - if autoIDs.IncrementID, err = a.IncrementID(sepAutoIncVer).Get(); err != nil { + if autoIDs.IncrementID, err = a.IncrementID().Get(); err != nil { return autoIDs, err } if autoIDs.RandomID, err = a.RandomID().Get(); err != nil { @@ -123,7 +120,7 @@ func (a *autoIDAccessors) Put(autoIDs AutoIDGroup) error { if err := a.RowID().Put(autoIDs.RowID); err != nil { return err } - if err := a.IncrementID(sepAutoIncVer).Put(autoIDs.IncrementID); err != nil { + if err := a.IncrementID().Put(autoIDs.IncrementID); err != nil { return err } return a.RandomID().Put(autoIDs.RandomID) @@ -134,7 +131,7 @@ func (a *autoIDAccessors) Del() error { if err := a.RowID().Del(); err != nil { return err } - if err := a.IncrementID(sepAutoIncVer).Del(); err != nil { + if err := a.IncrementID().Del(); err != nil { return err } return a.RandomID().Del() @@ -147,13 +144,8 @@ func (a *autoIDAccessors) RowID() AutoIDAccessor { } // IncrementID is used to get the auto_increment ID meta key-value accessor. -func (a *autoIDAccessors) IncrementID(tableVersion uint16) AutoIDAccessor { - // _tidb_rowid and auto_increment ID in old version TiDB share the same meta key-value. - if tableVersion < sepAutoIncVer { - a.access.idEncodeFn = a.access.m.autoTableIDKey - } else { - a.access.idEncodeFn = a.access.m.autoIncrementIDKey - } +func (a *autoIDAccessors) IncrementID() AutoIDAccessor { + a.access.idEncodeFn = a.access.m.autoIncrementIDKey return &a.access } diff --git a/table/table.go b/table/table.go index aeba32d8dc1fd..cecf10c4a9f5c 100644 --- a/table/table.go +++ b/table/table.go @@ -200,7 +200,8 @@ 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(ctx, uint64(1), int64(increment), int64(offset)) + ver := t.Meta().Version + _, max, err := t.Allocators(sctx).GetAutoIncrement(ver).Alloc(ctx, uint64(1), int64(increment), int64(offset)) if err != nil { return 0, err } @@ -212,7 +213,8 @@ func AllocAutoIncrementValue(ctx context.Context, t Table, sctx sessionctx.Conte func AllocBatchAutoIncrementValue(ctx context.Context, t Table, sctx sessionctx.Context, N int) (firstID int64, increment int64, err error) { increment = int64(sctx.GetSessionVars().AutoIncrementIncrement) offset := int64(sctx.GetSessionVars().AutoIncrementOffset) - min, max, err := t.Allocators(sctx).Get(autoid.RowIDAllocType).Alloc(ctx, uint64(N), increment, offset) + ver := t.Meta().Version + min, max, err := t.Allocators(sctx).GetAutoIncrement(ver).Alloc(ctx, 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 9d63e72e309e5..880fa486e2701 100644 --- a/table/tables/tables_test.go +++ b/table/tables/tables_test.go @@ -162,7 +162,7 @@ func TestBasic(t *testing.T) { require.NoError(t, err) table.MockTableFromMeta(tb.Meta()) - alc := tb.Allocators(nil).Get(autoid.RowIDAllocType) + alc := tb.Allocators(nil).Get(autoid.AutoIncrementType) require.NotNil(t, alc) err = alc.Rebase(0, false) From fa70e498265fc80ba6255bab01c99df4c39f1ff4 Mon Sep 17 00:00:00 2001 From: tangenta Date: Tue, 28 Sep 2021 14:50:33 +0800 Subject: [PATCH 2/4] *: refactor autoid.Allocators --- br/pkg/cdclog/buffer.go | 2 +- br/pkg/kv/kv.go | 3 +- br/pkg/lightning/backend/kv/allocator.go | 4 +-- br/pkg/lightning/backend/kv/sql2kv.go | 2 +- br/pkg/lightning/backend/kv/sql2kv_test.go | 18 +++++----- br/pkg/lightning/backend/tidb/tidb_test.go | 4 +-- br/pkg/lightning/restore/check_info.go | 2 +- br/pkg/lightning/restore/restore_test.go | 2 +- br/pkg/lightning/restore/table_restore.go | 2 +- br/pkg/restore/log_client.go | 2 +- ddl/column_change_test.go | 2 +- ddl/ddl_api.go | 2 +- ddl/table_test.go | 2 +- executor/executor.go | 10 +++--- executor/executor_test.go | 2 +- executor/infoschema_reader.go | 3 +- executor/insert_common.go | 6 ++-- executor/show.go | 2 +- executor/write.go | 2 +- infoschema/builder.go | 24 ++++--------- infoschema/infoschema.go | 9 ----- infoschema/tables.go | 4 +-- meta/autoid/autoid.go | 42 ++++++++++++---------- table/table.go | 6 ++-- table/tables/tables.go | 37 ++++++------------- table/temptable/ddl.go | 5 +-- 26 files changed, 81 insertions(+), 118 deletions(-) diff --git a/br/pkg/cdclog/buffer.go b/br/pkg/cdclog/buffer.go index 7cfe2ea638d94..c8b88bcf7d3ee 100644 --- a/br/pkg/cdclog/buffer.go +++ b/br/pkg/cdclog/buffer.go @@ -107,7 +107,7 @@ func (t *TableBuffer) ReloadMeta(tbl table.Table, allocator autoid.Allocators) { if kv.TableHasAutoRowID(tbl.Meta()) { colPerm = append(colPerm, -1) } - if t.allocator == nil { + if t.allocator.Items == nil { t.allocator = allocator } t.tableInfo = tbl diff --git a/br/pkg/kv/kv.go b/br/pkg/kv/kv.go index 736957e64540b..87b6ec71fbb7e 100644 --- a/br/pkg/kv/kv.go +++ b/br/pkg/kv/kv.go @@ -353,8 +353,7 @@ func (kvcodec *tableKVEncoder) AddRecord( _ = alloc.Rebase(value.GetInt64()&((1< Date: Tue, 28 Sep 2021 20:53:46 +0800 Subject: [PATCH 3/4] *: fix integration tests and refine some code --- br/pkg/backup/client.go | 8 +++--- br/pkg/lightning/restore/restore.go | 19 +++++--------- br/pkg/lightning/restore/table_restore.go | 21 +++++++++++----- br/pkg/lightning/restore/tidb.go | 15 ++++++++++++ br/pkg/utils/schema.go | 10 -------- ddl/db_integration_test.go | 30 ++++++----------------- ddl/db_test.go | 4 +-- ddl/ddl_api.go | 17 ++++--------- ddl/table.go | 7 ++---- executor/infoschema_reader.go | 2 +- executor/insert_test.go | 28 ++++++++++----------- executor/seqtest/seq_executor_test.go | 14 +++++------ infoschema/builder.go | 5 ++-- meta/autoid/autoid.go | 28 +++++++++++---------- meta/autoid/memid.go | 21 ++++++++++------ meta/autoid/memid_test.go | 4 +-- table/tables/tables.go | 24 ++++++++---------- table/temptable/ddl.go | 3 +-- util/tableutil/tableutil.go | 4 +-- 19 files changed, 125 insertions(+), 139 deletions(-) diff --git a/br/pkg/backup/client.go b/br/pkg/backup/client.go index 65bea75399550..ec7a5d51144c3 100644 --- a/br/pkg/backup/client.go +++ b/br/pkg/backup/client.go @@ -23,6 +23,7 @@ import ( filter "github.com/pingcap/tidb-tools/pkg/table-filter" "github.com/pingcap/tidb/br/pkg/conn" berrors "github.com/pingcap/tidb/br/pkg/errors" + "github.com/pingcap/tidb/br/pkg/lightning/common" "github.com/pingcap/tidb/br/pkg/logutil" "github.com/pingcap/tidb/br/pkg/metautil" "github.com/pingcap/tidb/br/pkg/redact" @@ -355,18 +356,17 @@ func setTableInfoAutoIDs(storage kv.Storage, dbInfo *model.DBInfo, tableInfo *mo if err != nil { return errors.Trace(err) } - autoIncIDsSeparated := tableInfo.Version >= model.TableInfoVersion5 - if !autoIncIDsSeparated { + if !model.AutoIncrementIDIsSeparated(tableInfo.Version) { tableInfo.AutoIncID = autoIDs.RowID + 1 logger.Debug("change table AutoIncID", zap.Int64("AutoIncID", tableInfo.AutoIncID)) } else { - if utils.HasRowID(tableInfo) { + if common.TableHasAutoRowID(tableInfo) { tableInfo.AutoRowID = autoIDs.RowID + 1 logger.Debug("change table AutoRowID", zap.Int64("AutoRowID", tableInfo.AutoRowID)) } - if utils.HasAutoIncID(tableInfo) { + if tableInfo.GetAutoIncrementColInfo() != nil { tableInfo.AutoIncID = autoIDs.IncrementID + 1 logger.Debug("change table AutoIncID", zap.Int64("AutoIncID", tableInfo.AutoIncID)) diff --git a/br/pkg/lightning/restore/restore.go b/br/pkg/lightning/restore/restore.go index 5bd10492e2002..a18b59c032226 100644 --- a/br/pkg/lightning/restore/restore.go +++ b/br/pkg/lightning/restore/restore.go @@ -1508,20 +1508,13 @@ func (tr *TableRestore) restoreTable( return false, err } } else { + maxRowID := tr.tableInfo.Core.AutoIncID if model.AutoIncrementIDIsSeparated(tr.tableInfo.Core.Version) { - cp.AllocBase = mathutil.MaxInt64(cp.AllocBase, tr.tableInfo.Core.AutoRowID) - if err := tr.alloc.Get(autoid.RowIDAllocType).Rebase(cp.AllocBase, false); err != nil { - return false, err - } - autoIncID := tr.tableInfo.Core.AutoIncID - if err := tr.alloc.Get(autoid.AutoIncrementType).Rebase(autoIncID, false); err != nil { - return false, err - } - } else { - cp.AllocBase = mathutil.MaxInt64(cp.AllocBase, tr.tableInfo.Core.AutoIncID) - if err := tr.alloc.Get(autoid.RowIDAllocType).Rebase(cp.AllocBase, false); err != nil { - return false, err - } + maxRowID = tr.tableInfo.Core.AutoRowID + } + cp.AllocBase = mathutil.MaxInt64(cp.AllocBase, maxRowID) + if err := tr.alloc.Get(autoid.RowIDAllocType).Rebase(cp.AllocBase, false); err != nil { + return false, err } } rc.saveCpCh <- saveCp{ diff --git a/br/pkg/lightning/restore/table_restore.go b/br/pkg/lightning/restore/table_restore.go index 0bc1225497aca..be8185635abd4 100644 --- a/br/pkg/lightning/restore/table_restore.go +++ b/br/pkg/lightning/restore/table_restore.go @@ -678,15 +678,24 @@ func (tr *TableRestore) postProcess( if cp.Status < checkpoints.CheckpointStatusAlteredAutoInc { rc.alterTableLock.Lock() tblInfo := tr.tableInfo.Core - separated := model.AutoIncrementIDIsSeparated(tblInfo.Version) + hasRowID := common.TableHasAutoRowID(tblInfo) + hasIncID := tblInfo.GetAutoIncrementColInfo() != nil var err error if tblInfo.PKIsHandle && tblInfo.ContainsAutoRandomBits() { err = AlterAutoRandom(ctx, rc.tidbGlue.GetSQLExecutor(), tr.tableName, tr.alloc.Get(autoid.AutoRandomType).Base()+1) - } else if common.TableHasAutoRowID(tblInfo) || (tblInfo.GetAutoIncrementColInfo() != nil && !separated) { - // only alter auto increment id iff table contains auto-increment column or generated handle - err = AlterAutoIncrement(ctx, rc.tidbGlue.GetSQLExecutor(), tr.tableName, tr.alloc.Get(autoid.RowIDAllocType).Base()+1) - } else if tblInfo.GetAutoIncrementColInfo() != nil && separated { - err = AlterAutoIncrement(ctx, rc.tidbGlue.GetSQLExecutor(), tr.tableName, tr.alloc.Get(autoid.AutoIncrementType).Base()+1) + } + if model.AutoIncrementIDIsSeparated(tblInfo.Version) { + if hasRowID && err == nil { + err = AlterRowID(ctx, rc.tidbGlue.GetSQLExecutor(), tr.tableName, tr.alloc.Get(autoid.RowIDAllocType).Base()+1) + } + if hasIncID && err == nil { + err = AlterAutoIncrement(ctx, rc.tidbGlue.GetSQLExecutor(), tr.tableName, tr.alloc.Get(autoid.AutoIncrementType).Base()+1) + } + } else { + if (hasRowID || hasIncID) && err == nil { + // only alter auto increment id iff table contains auto-increment column or generated handle + err = AlterAutoIncrement(ctx, rc.tidbGlue.GetSQLExecutor(), tr.tableName, tr.alloc.Get(autoid.RowIDAllocType).Base()+1) + } } rc.alterTableLock.Unlock() saveCpErr := rc.saveStatusCheckpoint(ctx, tr.tableName, checkpoints.WholeTableEngineID, err, checkpoints.CheckpointStatusAlteredAutoInc) diff --git a/br/pkg/lightning/restore/tidb.go b/br/pkg/lightning/restore/tidb.go index 44277e24433a1..2dbf2e89316ba 100644 --- a/br/pkg/lightning/restore/tidb.go +++ b/br/pkg/lightning/restore/tidb.go @@ -373,6 +373,21 @@ func AlterAutoIncrement(ctx context.Context, g glue.SQLExecutor, tableName strin return errors.Annotatef(err, "%s", query) } +func AlterRowID(ctx context.Context, g glue.SQLExecutor, tableName string, rowIDBase int64) error { + logger := log.With(zap.String("table", tableName), zap.Int64("row_id", rowIDBase)) + query := fmt.Sprintf("ALTER TABLE %s ROW_ID=%d", tableName, rowIDBase) + task := logger.Begin(zap.InfoLevel, "alter table row_id") + err := g.ExecuteWithLog(ctx, query, "alter table row_id", logger) + task.End(zap.ErrorLevel, err) + if err != nil { + task.Error( + "alter table row_id failed, please perform the query manually", + zap.String("query", query), + ) + } + return errors.Annotatef(err, "%s", query) +} + func AlterAutoRandom(ctx context.Context, g glue.SQLExecutor, tableName string, randomBase int64) error { logger := log.With(zap.String("table", tableName), zap.Int64("auto_random", randomBase)) query := fmt.Sprintf("ALTER TABLE %s AUTO_RANDOM_BASE=%d", tableName, randomBase) diff --git a/br/pkg/utils/schema.go b/br/pkg/utils/schema.go index 056c68f8b86bc..a68ce6e9d78dd 100644 --- a/br/pkg/utils/schema.go +++ b/br/pkg/utils/schema.go @@ -17,16 +17,6 @@ import ( // temporaryDBNamePrefix is the prefix name of system db, e.g. mysql system db will be rename to __TiDB_BR_Temporary_mysql const temporaryDBNamePrefix = "__TiDB_BR_Temporary_" -// HasRowID checks whether the tables has _tidb_rowid. -func HasRowID(tblInfo *model.TableInfo) bool { - return !tblInfo.PKIsHandle && !tblInfo.IsCommonHandle -} - -// HasAutoIncID checks whether the tables has auto_increment ID. -func HasAutoIncID(tblInfo *model.TableInfo) bool { - return tblInfo.GetAutoIncrementColInfo() != nil -} - // Database wraps the schema and tables of a database. type Database struct { Info *model.DBInfo diff --git a/ddl/db_integration_test.go b/ddl/db_integration_test.go index f31f1ef9dc08f..9dc8a17a517bd 100644 --- a/ddl/db_integration_test.go +++ b/ddl/db_integration_test.go @@ -2422,22 +2422,6 @@ func (s *testIntegrationSuite3) TestCreateTableWithAutoIdCache(c *C) { tk.MustExec("insert into t1 values()") tk.MustQuery("select * from t1").Check(testkit.Rows("101")) - // Test primary key is not handle. - tk.MustExec("drop table if exists t;") - tk.MustExec("drop table if exists t1;") - tk.MustExec("create table t(a int) auto_id_cache 100") - _, err = s.dom.InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("t")) - c.Assert(err, IsNil) - - tk.MustExec("insert into t values()") - tk.MustQuery("select _tidb_rowid from t").Check(testkit.Rows("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 values()") - tk.MustQuery("select _tidb_rowid from t1").Check(testkit.Rows("101")) - // Test both auto_increment and rowid exist. tk.MustExec("drop table if exists t;") tk.MustExec("drop table if exists t1;") @@ -2446,13 +2430,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 from t1").Check(testkit.Rows("101")) tk.MustExec("delete from t1") // Test alter auto_id_cache. @@ -2462,13 +2446,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 from t1").Check(testkit.Rows("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 from t").Check(testkit.Rows("401")) tk.MustExec("delete from t") tk.MustExec("drop table if exists t;") @@ -2649,12 +2633,12 @@ func (s *testIntegrationSuite3) TestAutoIncrementForce(c *C) { } // Rebase _tidb_row_id. tk.MustExec("create table t (a int);") - tk.MustExec("alter table t force auto_increment = 2;") + tk.MustExec("alter table t force row_id = 2;") tk.MustExec("insert into t values (1),(2);") tk.MustQuery("select a, _tidb_rowid from t;").Check(testkit.Rows("1 2", "2 3")) // Cannot set next global ID to 0. - tk.MustGetErrCode("alter table t force auto_increment = 0;", errno.ErrAutoincReadFailed) - tk.MustExec("alter table t force auto_increment = 1;") + tk.MustGetErrCode("alter table t force row_id = 0;", errno.ErrAutoincReadFailed) + tk.MustExec("alter table t force row_id = 1;") c.Assert(getNextGlobalID(), Equals, uint64(1)) // inserting new rows can overwrite the existing data. tk.MustExec("insert into t values (3);") diff --git a/ddl/db_test.go b/ddl/db_test.go index 2507a56f19549..ccdce1f206067 100644 --- a/ddl/db_test.go +++ b/ddl/db_test.go @@ -5808,7 +5808,7 @@ func (s *testSerialDBSuite) TestAlterShardRowIDBits(c *C) { tk.MustExec("drop table if exists t1") defer tk.MustExec("drop table if exists t1") tk.MustExec("create table t1 (a int) shard_row_id_bits = 5") - tk.MustExec(fmt.Sprintf("alter table t1 auto_increment = %d;", 1<<56)) + tk.MustExec(fmt.Sprintf("alter table t1 row_id = %d;", 1<<56)) tk.MustExec("insert into t1 set a=1;") // Test increase shard_row_id_bits failed by overflow global auto ID. @@ -5830,7 +5830,7 @@ func (s *testSerialDBSuite) TestAlterShardRowIDBits(c *C) { tk.MustExec("create table t1 (a int) shard_row_id_bits = 10") tk.MustExec("alter table t1 SHARD_ROW_ID_BITS = 5;") checkShardRowID(10, 5) - tk.MustExec(fmt.Sprintf("alter table t1 auto_increment = %d;", 1<<56)) + tk.MustExec(fmt.Sprintf("alter table t1 row_id = %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") diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index a1d6bdd18ada1..04f8eade9ad7a 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -1710,6 +1710,8 @@ func buildTableInfoWithLike(ctx sessionctx.Context, ident ast.Ident, referTblInf tblInfo.Indices = newIndices tblInfo.Name = ident.Name tblInfo.AutoIncID = 0 + tblInfo.AutoRandID = 0 + tblInfo.AutoRowID = 0 tblInfo.ForeignKeys = nil // Ignore TiFlash replicas for temporary tables. if s.TemporaryKeyword != ast.TemporaryNone { @@ -2004,18 +2006,14 @@ func (d *ddl) CreateTableWithInfo( } else if actionType == model.ActionCreateTable { d.preSplitAndScatter(ctx, tbInfo, tbInfo.GetPartitionInfo()) if tbInfo.AutoRowID > 1 { - newEnd := tbInfo.AutoIncID - 1 + newEnd := tbInfo.AutoRowID - 1 if err = d.handleAutoIncID(tbInfo, schema.ID, newEnd, autoid.RowIDAllocType); err != nil { return errors.Trace(err) } } if tbInfo.AutoIncID > 1 { newEnd := tbInfo.AutoIncID - 1 - if model.AutoIncrementIDIsSeparated(tbInfo.Version) { - err = d.handleAutoIncID(tbInfo, schema.ID, newEnd, autoid.AutoIncrementType) - } else { - err = d.handleAutoIncID(tbInfo, schema.ID, newEnd, autoid.RowIDAllocType) - } + err = d.handleAutoIncID(tbInfo, schema.ID, newEnd, autoid.AutoIncrementType) if err != nil { return errors.Trace(err) } @@ -2780,12 +2778,7 @@ func (d *ddl) RebaseAutoID(ctx sessionctx.Context, ident ast.Ident, newBase int6 allocType = autoid.AutoRandomType case ast.TableOptionAutoIncrement: actionType = model.ActionRebaseAutoID - tbInfo := t.Meta() - if model.AutoIncrementIDIsSeparated(tbInfo.Version) { - allocType = autoid.AutoIncrementType - } else { - allocType = autoid.RowIDAllocType - } + allocType = autoid.AutoIncrementType case ast.TableOptionRowID: actionType = model.ActionRebaseRowID allocType = autoid.RowIDAllocType diff --git a/ddl/table.go b/ddl/table.go index 5b430d037c241..a8964ba0fa693 100644 --- a/ddl/table.go +++ b/ddl/table.go @@ -675,13 +675,10 @@ func onRebaseAutoID(store kv.Storage, t *meta.Meta, job *model.Job) (ver int64, switch job.Type { case model.ActionRebaseAutoID: tblInfo.AutoIncID = newBase - if model.AutoIncrementIDIsSeparated(tblInfo.Version) { - tp = autoid.AutoIncrementType - } else { - tp = autoid.RowIDAllocType - } + tp = autoid.AutoIncrementType case model.ActionRebaseAutoRandomBase: tblInfo.AutoRandID = newBase + tp = autoid.AutoRandomType case model.ActionRebaseRowID: tblInfo.AutoRowID = newBase tp = autoid.RowIDAllocType diff --git a/executor/infoschema_reader.go b/executor/infoschema_reader.go index 7df924ecffe86..2abd0eaee9510 100644 --- a/executor/infoschema_reader.go +++ b/executor/infoschema_reader.go @@ -20,7 +20,6 @@ import ( "encoding/hex" "encoding/json" "fmt" - "github.com/pingcap/tidb/meta/autoid" "io" "net/http" "sort" @@ -46,6 +45,7 @@ import ( "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/kv" + "github.com/pingcap/tidb/meta/autoid" plannercore "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/privilege" "github.com/pingcap/tidb/session/txninfo" diff --git a/executor/insert_test.go b/executor/insert_test.go index 23fd0ba99cba8..07fd1e33ac708 100644 --- a/executor/insert_test.go +++ b/executor/insert_test.go @@ -1131,27 +1131,27 @@ 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 - tk.Se.GetSessionVars().AutoIncrementOffset = -2 - _, err := tk.Exec(`insert into io(b) values (null),(null),(null)`) - c.Assert(err, NotNil) - c.Assert(err.Error(), Equals, "[autoid:8060]Invalid auto_increment settings: auto_increment_increment: -1, auto_increment_offset: -2, both of them must be in range [1..65535]") - tk.MustExec(`delete from io`) - - tk.Se.GetSessionVars().AutoIncrementIncrement = 65536 - tk.Se.GetSessionVars().AutoIncrementOffset = 65536 - _, err = tk.Exec(`insert into io(b) values (null),(null),(null)`) - c.Assert(err, NotNil) - c.Assert(err.Error(), Equals, "[autoid:8060]Invalid auto_increment settings: auto_increment_increment: 65536, auto_increment_offset: 65536, both of them must be in range [1..65535]") + tk.MustExec("set @@auto_increment_increment = -1") + tk.MustQuery("show warnings;").Check( + testkit.Rows("Warning 1292 Truncated incorrect auto_increment_increment value: '-1'")) + tk.MustExec("set @@auto_increment_offset = -2") + tk.MustQuery("show warnings;").Check( + testkit.Rows("Warning 1292 Truncated incorrect auto_increment_offset value: '-2'")) + tk.MustExec("set @@auto_increment_increment = 65536") + tk.MustQuery("show warnings;").Check( + testkit.Rows("Warning 1292 Truncated incorrect auto_increment_increment value: '65536'")) + tk.MustExec("set @@auto_increment_offset = 65536") + tk.MustQuery("show warnings;").Check( + testkit.Rows("Warning 1292 Truncated incorrect auto_increment_offset value: '65536'")) } var _ = Suite(&testSuite9{&baseTestSuite{}}) diff --git a/executor/seqtest/seq_executor_test.go b/executor/seqtest/seq_executor_test.go index bb81ab9c9d270..39201aa033342 100644 --- a/executor/seqtest/seq_executor_test.go +++ b/executor/seqtest/seq_executor_test.go @@ -814,17 +814,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 _TIDB_ROW_ID")) // 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 _TIDB_ROW_ID")) // 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 _TIDB_ROW_ID")) tk.MustExec("drop table t") // test for a table with the primary key @@ -867,19 +867,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 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 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 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 97 SEQUENCE")) } func (s *seqTestSuite) TestNoHistoryWhenDisableRetry(c *C) { diff --git a/infoschema/builder.go b/infoschema/builder.go index 056e13df563fc..4d856f76980b5 100644 --- a/infoschema/builder.go +++ b/infoschema/builder.go @@ -413,11 +413,12 @@ func reconstructAllocators(s kv.Storage, dbID int64, oldTbl table.Table, newTbl return as.Remove(autoid.RowIDAllocType).Add(rowID) case model.ActionRebaseAutoID, model.ActionModifyTableAutoIdCache: unsigned := newTbl.IsAutoIncColUnsigned() + cacheOpt := autoid.CustomAutoIncCacheOption(newTbl.AutoIdCache) if model.AutoIncrementIDIsSeparated(newTbl.Version) { - incID := autoid.NewAllocator(s, dbID, newTbl.ID, unsigned, autoid.AutoIncrementType) + incID := autoid.NewAllocator(s, dbID, newTbl.ID, unsigned, autoid.AutoIncrementType, cacheOpt) return as.Remove(autoid.AutoIncrementType).Add(incID) } - rowID := autoid.NewAllocator(s, dbID, newTbl.ID, unsigned, autoid.RowIDAllocType) + rowID := autoid.NewAllocator(s, dbID, newTbl.ID, unsigned, autoid.RowIDAllocType, cacheOpt) return as.Remove(autoid.RowIDAllocType).Remove(autoid.AutoIncrementType).Add(rowID) case model.ActionRebaseAutoRandomBase: unsigned := newTbl.IsAutoRandomBitColUnsigned() diff --git a/meta/autoid/autoid.go b/meta/autoid/autoid.go index dc7dfd8157293..ffa3c27d198c6 100644 --- a/meta/autoid/autoid.go +++ b/meta/autoid/autoid.go @@ -511,18 +511,24 @@ func NewAllocatorsFromTblInfo(store kv.Storage, schemaID int64, tblInfo *model.T idCacheOpt := CustomAutoIncCacheOption(tblInfo.AutoIdCache) hasRowID := !tblInfo.PKIsHandle && !tblInfo.IsCommonHandle hasAutoIncID := tblInfo.GetAutoIncrementColInfo() != nil - separated := model.AutoIncrementIDIsSeparated(tblInfo.Version) var allocs []Allocator - if hasRowID || (hasAutoIncID && !separated) { - alloc := NewAllocator(store, dbID, tblInfo.ID, false, RowIDAllocType, idCacheOpt) - allocs = append(allocs, alloc) - } - if hasAutoIncID && separated { - alloc := NewAllocator(store, dbID, tblInfo.ID, tblInfo.IsAutoIncColUnsigned(), AutoIncrementType, idCacheOpt) - allocs = append(allocs, alloc) + if model.AutoIncrementIDIsSeparated(tblInfo.Version) { + if hasRowID { + alloc := NewAllocator(store, dbID, tblInfo.ID, false, RowIDAllocType) + allocs = append(allocs, alloc) + } + if hasAutoIncID { + alloc := NewAllocator(store, dbID, tblInfo.ID, tblInfo.IsAutoIncColUnsigned(), AutoIncrementType, idCacheOpt) + allocs = append(allocs, alloc) + } + } else { + if hasRowID || hasAutoIncID { + alloc := NewAllocator(store, dbID, tblInfo.ID, tblInfo.IsAutoIncColUnsigned(), RowIDAllocType, idCacheOpt) + allocs = append(allocs, alloc) + } } if tblInfo.ContainsAutoRandomBits() { - alloc := NewAllocator(store, dbID, tblInfo.ID, tblInfo.IsAutoRandomBitColUnsigned(), AutoRandomType, idCacheOpt) + alloc := NewAllocator(store, dbID, tblInfo.ID, tblInfo.IsAutoRandomBitColUnsigned(), AutoRandomType) allocs = append(allocs, alloc) } return NewAllocators(tblInfo.Version, allocs...) @@ -562,10 +568,6 @@ func (alloc *allocator) AllocSeqCache() (int64, int64, int64, error) { return alloc.alloc4Sequence() } -func validIncrementAndOffset(increment, offset int64) bool { - return (increment >= minIncrement && increment <= maxIncrement) && (offset >= minIncrement && offset <= maxIncrement) -} - // CalcNeededBatchSize is used to calculate batch size for autoID allocation. // It firstly seeks to the first valid position based on increment and offset, // then plus the length remained, which could be (n-1) * increment. diff --git a/meta/autoid/memid.go b/meta/autoid/memid.go index a499f34e8a5da..59ea725dee0eb 100644 --- a/meta/autoid/memid.go +++ b/meta/autoid/memid.go @@ -22,17 +22,24 @@ import ( ) // NewAllocatorFromTempTblInfo creates an in-memory allocator from a temporary table info. -func NewAllocatorFromTempTblInfo(tblInfo *model.TableInfo) Allocator { +func NewAllocatorFromTempTblInfo(tblInfo *model.TableInfo) Allocators { + var allocators []Allocator hasRowID := !tblInfo.PKIsHandle && !tblInfo.IsCommonHandle + if hasRowID { + allocators = append(allocators, &inMemoryAllocator{ + isUnsigned: false, + allocType: RowIDAllocType, + }) + } hasAutoIncID := tblInfo.GetAutoIncrementColInfo() != nil - // Temporary tables don't support auto_random and sequence. - if hasRowID || hasAutoIncID { - return &inMemoryAllocator{ + if hasAutoIncID { + allocators = append(allocators, &inMemoryAllocator{ isUnsigned: tblInfo.IsAutoIncColUnsigned(), - allocType: RowIDAllocType, - } + allocType: AutoIncrementType, + }) } - return nil + // Temporary tables don't support auto_random and sequence. + return NewAllocators(tblInfo.Version, allocators...) } // inMemoryAllocator is typically used for temporary tables. diff --git a/meta/autoid/memid_test.go b/meta/autoid/memid_test.go index 15dd436f98ed1..ddf58203c2260 100644 --- a/meta/autoid/memid_test.go +++ b/meta/autoid/memid_test.go @@ -44,7 +44,7 @@ func TestInMemoryAlloc(t *testing.T) { tblInfo := &model.TableInfo{ Columns: []*model.ColumnInfo{columnInfo}, } - alloc := autoid.NewAllocatorFromTempTblInfo(tblInfo) + alloc := autoid.NewAllocatorFromTempTblInfo(tblInfo).Get(autoid.RowIDAllocType) require.NotNil(t, alloc) // alloc 1 @@ -94,7 +94,7 @@ func TestInMemoryAlloc(t *testing.T) { // test unsigned columnInfo.FieldType.Flag |= mysql.UnsignedFlag - alloc = autoid.NewAllocatorFromTempTblInfo(tblInfo) + alloc = autoid.NewAllocatorFromTempTblInfo(tblInfo).Get(autoid.RowIDAllocType) require.NotNil(t, alloc) var n uint64 = math.MaxUint64 - 2 diff --git a/table/tables/tables.go b/table/tables/tables.go index a295625a950bc..ab6f186387908 100644 --- a/table/tables/tables.go +++ b/table/tables/tables.go @@ -1437,11 +1437,7 @@ func (t *TableCommon) Allocators(ctx sessionctx.Context) autoid.Allocators { } else if ctx.GetSessionVars().IDAllocator == nil { // Use an independent allocator for global temporary tables. if t.meta.TempTableType == model.TempTableGlobal { - if alloc := ctx.GetSessionVars().GetTemporaryTable(t.meta).GetAutoIDAllocator(); alloc != nil { - return autoid.NewAllocators(t.meta.Version, alloc) - } - // If the session is not in a txn, for example, in "show create table", use the original allocator. - // Otherwise the would be a nil pointer dereference. + return ctx.GetSessionVars().GetTemporaryTable(t.meta).GetAutoIDAllocators() } return t.allocs } @@ -1826,8 +1822,8 @@ type TemporaryTable struct { modified bool // The stats of this table. So far it's always pseudo stats. stats *statistics.Table - // The autoID allocator of this table. - autoIDAllocator autoid.Allocator + // The autoID allocators of this table. + autoIDAllocators autoid.Allocators // Table size. size int64 @@ -1837,16 +1833,16 @@ type TemporaryTable struct { // TempTableFromMeta builds a TempTable from model.TableInfo. func TempTableFromMeta(tblInfo *model.TableInfo) tableutil.TempTable { return &TemporaryTable{ - modified: false, - stats: statistics.PseudoTable(tblInfo), - autoIDAllocator: autoid.NewAllocatorFromTempTblInfo(tblInfo), - meta: tblInfo, + modified: false, + stats: statistics.PseudoTable(tblInfo), + autoIDAllocators: autoid.NewAllocatorFromTempTblInfo(tblInfo), + meta: tblInfo, } } -// GetAutoIDAllocator is implemented from TempTable.GetAutoIDAllocator. -func (t *TemporaryTable) GetAutoIDAllocator() autoid.Allocator { - return t.autoIDAllocator +// GetAutoIDAllocator is implemented from TempTable.GetAutoIDAllocators. +func (t *TemporaryTable) GetAutoIDAllocators() autoid.Allocators { + return t.autoIDAllocators } // SetModified is implemented from TempTable.SetModified. diff --git a/table/temptable/ddl.go b/table/temptable/ddl.go index 6d0a1c2f47bd3..fb56a91d5458e 100644 --- a/table/temptable/ddl.go +++ b/table/temptable/ddl.go @@ -175,8 +175,7 @@ func newTemporaryTableFromTableInfo(sctx sessionctx.Context, tbInfo *model.Table } // AutoID is allocated in mocked.. - alloc := autoid.NewAllocatorFromTempTblInfo(tbInfo) - allocs := autoid.NewAllocators(tbInfo.Version, alloc) + allocs := autoid.NewAllocatorFromTempTblInfo(tbInfo) return tables.TableFromMeta(allocs, tbInfo) } diff --git a/util/tableutil/tableutil.go b/util/tableutil/tableutil.go index 106c162a3915e..16b44c7b843cb 100644 --- a/util/tableutil/tableutil.go +++ b/util/tableutil/tableutil.go @@ -22,8 +22,8 @@ import ( // TempTable is used to store transaction-specific or session-specific information for global / local temporary tables. // For example, stats and autoID should have their own copies of data, instead of being shared by all sessions. type TempTable interface { - // GetAutoIDAllocator gets the autoID allocator of this table. - GetAutoIDAllocator() autoid.Allocator + // GetAutoIDAllocators gets the autoID allocators of this table. + GetAutoIDAllocators() autoid.Allocators // SetModified sets that the table is modified. SetModified(bool) From 0ad91cda17dc1756066fb6ed4b26afe4913abd4e Mon Sep 17 00:00:00 2001 From: tangenta Date: Wed, 29 Sep 2021 16:12:19 +0800 Subject: [PATCH 4/4] fix some integration tests --- ddl/column.go | 10 +++++----- ddl/table.go | 6 +++++- meta/autoid/autoid.go | 14 -------------- meta/autoid/memid_test.go | 3 ++- 4 files changed, 12 insertions(+), 21 deletions(-) diff --git a/ddl/column.go b/ddl/column.go index 9e86e578396b2..2f4c18779a84b 100644 --- a/ddl/column.go +++ b/ddl/column.go @@ -1609,12 +1609,12 @@ func checkAndApplyAutoRandomBits(m *meta.Meta, dbInfo *model.DBInfo, tblInfo *mo func checkNewAutoRandomBits(idAccessors meta.AutoIDAccessors, oldCol *model.ColumnInfo, newCol *model.ColumnInfo, newAutoRandBits uint64, tblInfoVer uint16) error { idAcc := idAccessors.RandomID() - convertedFromAutoInc := mysql.HasAutoIncrementFlag(oldCol.Flag) - if convertedFromAutoInc { - if !model.AutoIncrementIDIsSeparated(tblInfoVer) { - idAcc = idAccessors.RowID() - } else { + if mysql.HasAutoIncrementFlag(oldCol.Flag) { + // Converted from auto_increment ID meta key. + if model.AutoIncrementIDIsSeparated(tblInfoVer) { idAcc = idAccessors.IncrementID() + } else { + idAcc = idAccessors.RowID() } } // Generate a new auto ID first to prevent concurrent update in DML. diff --git a/ddl/table.go b/ddl/table.go index a8964ba0fa693..c3e7a1dce94e2 100644 --- a/ddl/table.go +++ b/ddl/table.go @@ -776,7 +776,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() + alloc := tbl.Allocators(ctx).Get(autoid.RowIDAllocType) + if alloc == nil { + return nil + } + autoIncID, err := alloc.NextGlobalAutoID() if err != nil { return errors.Trace(err) } diff --git a/meta/autoid/autoid.go b/meta/autoid/autoid.go index ffa3c27d198c6..6688288123e9d 100644 --- a/meta/autoid/autoid.go +++ b/meta/autoid/autoid.go @@ -83,20 +83,6 @@ const ( SequenceType ) -func (a AllocatorType) String() string { - switch a { - case RowIDAllocType: - return "_tidb_rowid" - case AutoIncrementType: - return "auto_increment" - case AutoRandomType: - return "auto_random" - case SequenceType: - return "sequence" - } - return "unknown" -} - // CustomAutoIncCacheOption is one kind of AllocOption to customize the allocator step length. type CustomAutoIncCacheOption int64 diff --git a/meta/autoid/memid_test.go b/meta/autoid/memid_test.go index ddf58203c2260..db89b31640efb 100644 --- a/meta/autoid/memid_test.go +++ b/meta/autoid/memid_test.go @@ -43,6 +43,7 @@ func TestInMemoryAlloc(t *testing.T) { } tblInfo := &model.TableInfo{ Columns: []*model.ColumnInfo{columnInfo}, + Version: model.CurrLatestTableInfoVersion, } alloc := autoid.NewAllocatorFromTempTblInfo(tblInfo).Get(autoid.RowIDAllocType) require.NotNil(t, alloc) @@ -94,7 +95,7 @@ func TestInMemoryAlloc(t *testing.T) { // test unsigned columnInfo.FieldType.Flag |= mysql.UnsignedFlag - alloc = autoid.NewAllocatorFromTempTblInfo(tblInfo).Get(autoid.RowIDAllocType) + alloc = autoid.NewAllocatorFromTempTblInfo(tblInfo).Get(autoid.AutoIncrementType) require.NotNil(t, alloc) var n uint64 = math.MaxUint64 - 2