diff --git a/ddl/column_change_test.go b/ddl/column_change_test.go index eb95d94c45d78..96a9dddb095f3 100644 --- a/ddl/column_change_test.go +++ b/ddl/column_change_test.go @@ -347,7 +347,7 @@ func getCurrentTable(d *ddl, schemaID, tableID int64) (table.Table, error) { if err != nil { return nil, errors.Trace(err) } - alloc := autoid.NewAllocator(d.store, schemaID) + alloc := autoid.NewAllocator(d.store, schemaID, false) tbl, err := table.TableFromMeta(alloc, tblInfo) if err != nil { return nil, errors.Trace(err) diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index 6d2e99c00183a..60e8c320f94ae 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -1132,7 +1132,7 @@ func checkCharsetAndCollation(cs string, co string) error { // handleAutoIncID handles auto_increment option in DDL. It creates a ID counter for the table and initiates the counter to a proper value. // For example if the option sets auto_increment to 10. The counter will be set to 9. So the next allocated ID will be 10. func (d *ddl) handleAutoIncID(tbInfo *model.TableInfo, schemaID int64) error { - alloc := autoid.NewAllocator(d.store, tbInfo.GetDBID(schemaID)) + alloc := autoid.NewAllocator(d.store, tbInfo.GetDBID(schemaID), tbInfo.IsAutoIncColUnsigned()) tbInfo.State = model.StatePublic tb, err := table.TableFromMeta(alloc, tbInfo) if err != nil { diff --git a/ddl/table.go b/ddl/table.go index 9eae32edc593d..d1d710454c7aa 100644 --- a/ddl/table.go +++ b/ddl/table.go @@ -183,7 +183,7 @@ func splitTableRegion(store kv.Storage, tableID int64) { } func getTable(store kv.Storage, schemaID int64, tblInfo *model.TableInfo) (table.Table, error) { - alloc := autoid.NewAllocator(store, tblInfo.GetDBID(schemaID)) + alloc := autoid.NewAllocator(store, tblInfo.GetDBID(schemaID), tblInfo.IsAutoIncColUnsigned()) tbl, err := table.TableFromMeta(alloc, tblInfo) return tbl, errors.Trace(err) } diff --git a/ddl/table_test.go b/ddl/table_test.go index 5e3703217a680..d4dd046164530 100644 --- a/ddl/table_test.go +++ b/ddl/table_test.go @@ -236,7 +236,7 @@ func testGetTableWithError(d *ddl, schemaID, tableID int64) (table.Table, error) if tblInfo == nil { return nil, errors.New("table not found") } - alloc := autoid.NewAllocator(d.store, schemaID) + alloc := autoid.NewAllocator(d.store, schemaID, false) tbl, err := table.TableFromMeta(alloc, tblInfo) if err != nil { return nil, errors.Trace(err) diff --git a/executor/executor_test.go b/executor/executor_test.go index 52430505bb534..33138489382bb 100644 --- a/executor/executor_test.go +++ b/executor/executor_test.go @@ -2848,7 +2848,7 @@ func (s *testSuite) TestCheckIndex(c *C) { c.Assert(err, IsNil) tbInfo := tbl.Meta() - alloc := autoid.NewAllocator(s.store, dbInfo.ID) + alloc := autoid.NewAllocator(s.store, dbInfo.ID, false) tb, err := tables.TableFromMeta(alloc, tbInfo) c.Assert(err, IsNil) diff --git a/executor/insert_common.go b/executor/insert_common.go index 9a39b5183f99c..f22ab941463b1 100644 --- a/executor/insert_common.go +++ b/executor/insert_common.go @@ -473,10 +473,12 @@ func (e *InsertValues) adjustAutoIncrementDatum(d types.Datum, hasValue bool, c d.SetNull() } if !d.IsNull() { - recordID, err = d.ToInt64(e.ctx.GetSessionVars().StmtCtx) - if e.filterErr(err) != nil { - return types.Datum{}, errors.Trace(err) + sc := e.ctx.GetSessionVars().StmtCtx + datum, err1 := d.ConvertTo(sc, &c.FieldType) + if e.filterErr(err1) != nil { + return types.Datum{}, err1 } + recordID = datum.GetInt64() } // Use the value if it's not null and not 0. if recordID != 0 { diff --git a/go.mod b/go.mod index da2ef76738f7c..8c9da6febe116 100644 --- a/go.mod +++ b/go.mod @@ -47,7 +47,7 @@ require ( github.com/pingcap/errors v0.11.0 github.com/pingcap/goleveldb v0.0.0-20171020122428-b9ff6c35079e github.com/pingcap/kvproto v0.0.0-20181105061835-1b5d69cd1d26 - github.com/pingcap/parser v0.0.0-20181214132045-732efe993f70 + github.com/pingcap/parser v0.0.0-20181218071912-deacf026787e github.com/pingcap/pd v2.1.0-rc.4+incompatible github.com/pingcap/tidb-tools v0.0.0-20181112132202-4860a0d5de03 github.com/pingcap/tipb v0.0.0-20181012112600-11e33c750323 diff --git a/go.sum b/go.sum index a777b280a1f47..59b5897e47cee 100644 --- a/go.sum +++ b/go.sum @@ -106,8 +106,8 @@ github.com/pingcap/goleveldb v0.0.0-20171020122428-b9ff6c35079e h1:P73/4dPCL96rG github.com/pingcap/goleveldb v0.0.0-20171020122428-b9ff6c35079e/go.mod h1:O17XtbryoCJhkKGbT62+L2OlrniwqiGLSqrmdHCMzZw= github.com/pingcap/kvproto v0.0.0-20181105061835-1b5d69cd1d26 h1:JK4VLNYbSn36QSbCnqALi2ySXdH0DfcMssT/zmLf4Ls= github.com/pingcap/kvproto v0.0.0-20181105061835-1b5d69cd1d26/go.mod h1:0gwbe1F2iBIjuQ9AH0DbQhL+Dpr5GofU8fgYyXk+ykk= -github.com/pingcap/parser v0.0.0-20181214132045-732efe993f70 h1:18cirMLfudQucdZM5keSuhUFculJ2xOQb9hqKd4O8wQ= -github.com/pingcap/parser v0.0.0-20181214132045-732efe993f70/go.mod h1:1FNvfp9+J0wvc4kl8eGNh7Rqrxveg15jJoWo/a0uHwA= +github.com/pingcap/parser v0.0.0-20181218071912-deacf026787e h1:jKibIs55HR7OMo62uhjA6Bfx3GK+rbHK4Gfd4/8aTzk= +github.com/pingcap/parser v0.0.0-20181218071912-deacf026787e/go.mod h1:1FNvfp9+J0wvc4kl8eGNh7Rqrxveg15jJoWo/a0uHwA= github.com/pingcap/pd v2.1.0-rc.4+incompatible h1:/buwGk04aHO5odk/+O8ZOXGs4qkUjYTJ2UpCJXna8NE= github.com/pingcap/pd v2.1.0-rc.4+incompatible/go.mod h1:nD3+EoYes4+aNNODO99ES59V83MZSI+dFbhyr667a0E= github.com/pingcap/tidb-tools v0.0.0-20181112132202-4860a0d5de03 h1:xVuo5U+l6XAWHsb+xhkZ8zz3jerIwDfCHAO6kR2Kaog= diff --git a/infoschema/builder.go b/infoschema/builder.go index eaa00ce953741..d06446e4a42ee 100644 --- a/infoschema/builder.go +++ b/infoschema/builder.go @@ -173,7 +173,7 @@ func (b *Builder) applyCreateTable(m *meta.Meta, dbInfo *model.DBInfo, tableID i } if alloc == nil { schemaID := dbInfo.ID - alloc = autoid.NewAllocator(b.handle.store, tblInfo.GetDBID(schemaID)) + alloc = autoid.NewAllocator(b.handle.store, tblInfo.GetDBID(schemaID), tblInfo.IsAutoIncColUnsigned()) } tbl, err := tables.TableFromMeta(alloc, tblInfo) if err != nil { @@ -276,7 +276,7 @@ func (b *Builder) createSchemaTablesForDB(di *model.DBInfo) error { b.is.schemaMap[di.Name.L] = schTbls for _, t := range di.Tables { schemaID := di.ID - alloc := autoid.NewAllocator(b.handle.store, t.GetDBID(schemaID)) + alloc := autoid.NewAllocator(b.handle.store, t.GetDBID(schemaID), t.IsAutoIncColUnsigned()) var tbl table.Table tbl, err := tables.TableFromMeta(alloc, t) if err != nil { diff --git a/meta/autoid/autoid.go b/meta/autoid/autoid.go index 9cd045226eaf2..93e354ede3dd8 100644 --- a/meta/autoid/autoid.go +++ b/meta/autoid/autoid.go @@ -57,7 +57,8 @@ type allocator struct { end int64 store kv.Storage // dbID is current database's ID. - dbID int64 + dbID int64 + isUnsigned bool } // GetStep is only used by tests @@ -91,25 +92,65 @@ func (alloc *allocator) NextGlobalAutoID(tableID int64) (int64, error) { return errors.Trace(err1) }) metrics.AutoIDHistogram.WithLabelValues(metrics.GlobalAutoID, metrics.RetLabel(err)).Observe(time.Since(startTime).Seconds()) - return autoID + 1, errors.Trace(err) + if alloc.isUnsigned { + return int64(uint64(autoID) + 1), err + } + return autoID + 1, err } -// Rebase implements autoid.Allocator Rebase interface. -// The requiredBase is the minimum base value after Rebase. -// The real base may be greater than the required base. -func (alloc *allocator) Rebase(tableID, requiredBase int64, allocIDs bool) error { - if tableID == 0 { - return errInvalidTableID.GenWithStack("Invalid tableID") +func (alloc *allocator) rebase4Unsigned(tableID int64, requiredBase uint64, allocIDs bool) error { + // Satisfied by alloc.base, nothing to do. + if requiredBase <= uint64(alloc.base) { + return nil + } + // Satisfied by alloc.end, need to update alloc.base. + if requiredBase <= uint64(alloc.end) { + alloc.base = int64(requiredBase) + return nil + } + var newBase, newEnd uint64 + startTime := time.Now() + err := kv.RunInNewTxn(alloc.store, true, func(txn kv.Transaction) error { + m := meta.NewMeta(txn) + currentEnd, err1 := m.GetAutoTableID(alloc.dbID, tableID) + if err1 != nil { + return errors.Trace(err1) + } + uCurrentEnd := uint64(currentEnd) + if allocIDs { + newBase = mathutil.MaxUint64(uCurrentEnd, requiredBase) + newEnd = mathutil.MinUint64(math.MaxUint64-uint64(step), newBase) + uint64(step) + } else { + if uCurrentEnd >= requiredBase { + newBase = uCurrentEnd + newEnd = uCurrentEnd + // Required base satisfied, we don't need to update KV. + return nil + } + // If we don't want to allocate IDs, for example when creating a table with a given base value, + // We need to make sure when other TiDB server allocates ID for the first time, requiredBase + 1 + // will be allocated, so we need to increase the end to exactly the requiredBase. + newBase = requiredBase + newEnd = requiredBase + } + _, err1 = m.GenAutoTableID(alloc.dbID, tableID, int64(newEnd-uCurrentEnd)) + return err1 + }) + metrics.AutoIDHistogram.WithLabelValues(metrics.TableAutoIDRebase, metrics.RetLabel(err)).Observe(time.Since(startTime).Seconds()) + if err != nil { + return err } + alloc.base, alloc.end = int64(newBase), int64(newEnd) + return nil +} - alloc.mu.Lock() - defer alloc.mu.Unlock() +func (alloc *allocator) rebase4Signed(tableID, requiredBase int64, allocIDs bool) error { + // Satisfied by alloc.base, nothing to do. if requiredBase <= alloc.base { - // Satisfied by alloc.base, nothing to do. return nil } + // Satisfied by alloc.end, need to update alloc.base. if requiredBase <= alloc.end { - // Satisfied by alloc.end, need to updata alloc.base. alloc.base = requiredBase return nil } @@ -123,7 +164,7 @@ func (alloc *allocator) Rebase(tableID, requiredBase int64, allocIDs bool) error } if allocIDs { newBase = mathutil.MaxInt64(currentEnd, requiredBase) - newEnd = newBase + step + newEnd = mathutil.MinInt64(math.MaxInt64-step, newBase) + step } else { if currentEnd >= requiredBase { newBase = currentEnd @@ -138,23 +179,34 @@ func (alloc *allocator) Rebase(tableID, requiredBase int64, allocIDs bool) error newEnd = requiredBase } _, err1 = m.GenAutoTableID(alloc.dbID, tableID, newEnd-currentEnd) - return errors.Trace(err1) + return err1 }) metrics.AutoIDHistogram.WithLabelValues(metrics.TableAutoIDRebase, metrics.RetLabel(err)).Observe(time.Since(startTime).Seconds()) if err != nil { - return errors.Trace(err) + return err } alloc.base, alloc.end = newBase, newEnd return nil } -// Alloc implements autoid.Allocator Alloc interface. -func (alloc *allocator) Alloc(tableID int64) (int64, error) { +// Rebase implements autoid.Allocator Rebase interface. +// The requiredBase is the minimum base value after Rebase. +// The real base may be greater than the required base. +func (alloc *allocator) Rebase(tableID, requiredBase int64, allocIDs bool) error { if tableID == 0 { - return 0, errInvalidTableID.GenWithStack("Invalid tableID") + return errInvalidTableID.GenWithStack("Invalid tableID") } + alloc.mu.Lock() defer alloc.mu.Unlock() + + if alloc.isUnsigned { + return alloc.rebase4Unsigned(tableID, uint64(requiredBase), allocIDs) + } + return alloc.rebase4Signed(tableID, requiredBase, allocIDs) +} + +func (alloc *allocator) alloc4Unsigned(tableID int64) (int64, error) { if alloc.base == alloc.end { // step var newBase, newEnd int64 startTime := time.Now() @@ -165,15 +217,46 @@ func (alloc *allocator) Alloc(tableID int64) (int64, error) { if err1 != nil { return errors.Trace(err1) } - newEnd, err1 = m.GenAutoTableID(alloc.dbID, tableID, step) + tmpStep := int64(mathutil.MinUint64(math.MaxUint64-uint64(newBase), uint64(step))) + newEnd, err1 = m.GenAutoTableID(alloc.dbID, tableID, tmpStep) + return err1 + }) + metrics.AutoIDHistogram.WithLabelValues(metrics.TableAutoIDAlloc, metrics.RetLabel(err)).Observe(time.Since(startTime).Seconds()) + if err != nil { + return 0, err + } + if uint64(newBase) == math.MaxUint64 { + return 0, ErrAutoincReadFailed + } + alloc.base, alloc.end = newBase, newEnd + } + + alloc.base = int64(uint64(alloc.base) + 1) + log.Debugf("[kv] Alloc id %d, table ID:%d, from %p, database ID:%d", uint64(alloc.base), tableID, alloc, alloc.dbID) + return alloc.base, nil +} + +func (alloc *allocator) alloc4Signed(tableID int64) (int64, error) { + if alloc.base == alloc.end { // step + var newBase, newEnd int64 + startTime := time.Now() + err := kv.RunInNewTxn(alloc.store, true, func(txn kv.Transaction) error { + m := meta.NewMeta(txn) + var err1 error + newBase, err1 = m.GetAutoTableID(alloc.dbID, tableID) if err1 != nil { return errors.Trace(err1) } - return nil + tmpStep := mathutil.MinInt64(math.MaxInt64-newBase, step) + newEnd, err1 = m.GenAutoTableID(alloc.dbID, tableID, tmpStep) + return err1 }) metrics.AutoIDHistogram.WithLabelValues(metrics.TableAutoIDAlloc, metrics.RetLabel(err)).Observe(time.Since(startTime).Seconds()) if err != nil { - return 0, errors.Trace(err) + return 0, err + } + if newBase == math.MaxInt64 { + return 0, ErrAutoincReadFailed } alloc.base, alloc.end = newBase, newEnd } @@ -183,6 +266,19 @@ func (alloc *allocator) Alloc(tableID int64) (int64, error) { return alloc.base, nil } +// Alloc implements autoid.Allocator Alloc interface. +func (alloc *allocator) Alloc(tableID int64) (int64, error) { + if tableID == 0 { + return 0, errInvalidTableID.GenWithStack("Invalid tableID") + } + alloc.mu.Lock() + defer alloc.mu.Unlock() + if alloc.isUnsigned { + return alloc.alloc4Unsigned(tableID) + } + return alloc.alloc4Signed(tableID) +} + var ( memID int64 memIDLock sync.Mutex @@ -237,10 +333,11 @@ func (alloc *memoryAllocator) Alloc(tableID int64) (int64, error) { } // NewAllocator returns a new auto increment id generator on the store. -func NewAllocator(store kv.Storage, dbID int64) Allocator { +func NewAllocator(store kv.Storage, dbID int64, isUnsigned bool) Allocator { return &allocator{ - store: store, - dbID: dbID, + store: store, + dbID: dbID, + isUnsigned: isUnsigned, } } diff --git a/meta/autoid/autoid_test.go b/meta/autoid/autoid_test.go index 5b56967b546f3..cc097dafb7750 100644 --- a/meta/autoid/autoid_test.go +++ b/meta/autoid/autoid_test.go @@ -57,7 +57,7 @@ func (*testSuite) TestT(c *C) { }) c.Assert(err, IsNil) - alloc := autoid.NewAllocator(store, 1) + alloc := autoid.NewAllocator(store, 1, false) c.Assert(alloc, NotNil) globalAutoId, err := alloc.NextGlobalAutoID(1) @@ -97,13 +97,13 @@ func (*testSuite) TestT(c *C) { c.Assert(err, IsNil) c.Assert(id, Equals, int64(3011)) - alloc = autoid.NewAllocator(store, 1) + alloc = autoid.NewAllocator(store, 1, false) c.Assert(alloc, NotNil) id, err = alloc.Alloc(1) c.Assert(err, IsNil) c.Assert(id, Equals, int64(autoid.GetStep()+1)) - alloc = autoid.NewAllocator(store, 1) + alloc = autoid.NewAllocator(store, 1, false) c.Assert(alloc, NotNil) err = alloc.Rebase(2, int64(1), false) c.Assert(err, IsNil) @@ -111,11 +111,102 @@ func (*testSuite) TestT(c *C) { c.Assert(err, IsNil) c.Assert(id, Equals, int64(2)) - alloc = autoid.NewAllocator(store, 1) + alloc = autoid.NewAllocator(store, 1, false) c.Assert(alloc, NotNil) err = alloc.Rebase(3, int64(3210), false) c.Assert(err, IsNil) - alloc = autoid.NewAllocator(store, 1) + alloc = autoid.NewAllocator(store, 1, false) + c.Assert(alloc, NotNil) + err = alloc.Rebase(3, int64(3000), false) + c.Assert(err, IsNil) + id, err = alloc.Alloc(3) + c.Assert(err, IsNil) + c.Assert(id, Equals, int64(3211)) + err = alloc.Rebase(3, int64(6543), false) + c.Assert(err, IsNil) + id, err = alloc.Alloc(3) + c.Assert(err, IsNil) + c.Assert(id, Equals, int64(6544)) +} + +func (*testSuite) TestUnsignedAutoid(c *C) { + store, err := mockstore.NewMockTikvStore() + c.Assert(err, IsNil) + defer store.Close() + + err = kv.RunInNewTxn(store, false, func(txn kv.Transaction) error { + m := meta.NewMeta(txn) + err = m.CreateDatabase(&model.DBInfo{ID: 1, Name: model.NewCIStr("a")}) + c.Assert(err, IsNil) + err = m.CreateTable(1, &model.TableInfo{ID: 1, Name: model.NewCIStr("t")}) + c.Assert(err, IsNil) + err = m.CreateTable(1, &model.TableInfo{ID: 2, Name: model.NewCIStr("t1")}) + c.Assert(err, IsNil) + err = m.CreateTable(1, &model.TableInfo{ID: 3, Name: model.NewCIStr("t1")}) + c.Assert(err, IsNil) + return nil + }) + c.Assert(err, IsNil) + + alloc := autoid.NewAllocator(store, 1, true) + c.Assert(alloc, NotNil) + + globalAutoId, err := alloc.NextGlobalAutoID(1) + c.Assert(err, IsNil) + c.Assert(globalAutoId, Equals, int64(1)) + id, err := alloc.Alloc(1) + c.Assert(err, IsNil) + c.Assert(id, Equals, int64(1)) + id, err = alloc.Alloc(1) + c.Assert(err, IsNil) + c.Assert(id, Equals, int64(2)) + _, err = alloc.Alloc(0) + c.Assert(err, NotNil) + globalAutoId, err = alloc.NextGlobalAutoID(1) + c.Assert(err, IsNil) + c.Assert(globalAutoId, Equals, int64(autoid.GetStep()+1)) + + // rebase + err = alloc.Rebase(1, int64(1), true) + c.Assert(err, IsNil) + id, err = alloc.Alloc(1) + c.Assert(err, IsNil) + c.Assert(id, Equals, int64(3)) + err = alloc.Rebase(1, int64(3), true) + c.Assert(err, IsNil) + id, err = alloc.Alloc(1) + c.Assert(err, IsNil) + c.Assert(id, Equals, int64(4)) + err = alloc.Rebase(1, int64(10), true) + c.Assert(err, IsNil) + id, err = alloc.Alloc(1) + c.Assert(err, IsNil) + c.Assert(id, Equals, int64(11)) + err = alloc.Rebase(1, int64(3010), true) + c.Assert(err, IsNil) + id, err = alloc.Alloc(1) + c.Assert(err, IsNil) + c.Assert(id, Equals, int64(3011)) + + alloc = autoid.NewAllocator(store, 1, true) + c.Assert(alloc, NotNil) + id, err = alloc.Alloc(1) + c.Assert(err, IsNil) + c.Assert(id, Equals, int64(autoid.GetStep()+1)) + + alloc = autoid.NewAllocator(store, 1, true) + c.Assert(alloc, NotNil) + err = alloc.Rebase(2, int64(1), false) + c.Assert(err, IsNil) + id, err = alloc.Alloc(2) + c.Assert(err, IsNil) + c.Assert(id, Equals, int64(2)) + + alloc = autoid.NewAllocator(store, 1, true) + c.Assert(alloc, NotNil) + err = alloc.Rebase(3, int64(3210), false) + c.Assert(err, IsNil) + alloc = autoid.NewAllocator(store, 1, true) c.Assert(alloc, NotNil) err = alloc.Rebase(3, int64(3000), false) c.Assert(err, IsNil) @@ -159,7 +250,7 @@ func (*testSuite) TestConcurrentAlloc(c *C) { errCh := make(chan error, count) allocIDs := func() { - alloc := autoid.NewAllocator(store, dbID) + alloc := autoid.NewAllocator(store, dbID, false) for j := 0; j < int(autoid.GetStep())+5; j++ { id, err1 := alloc.Alloc(tblID) if err1 != nil { @@ -213,7 +304,7 @@ func (*testSuite) TestRollbackAlloc(c *C) { injectConf := new(kv.InjectionConfig) injectConf.SetCommitError(errors.New("injected")) injectedStore := kv.NewInjectedStore(store, injectConf) - alloc := autoid.NewAllocator(injectedStore, 1) + alloc := autoid.NewAllocator(injectedStore, 1, false) _, err = alloc.Alloc(2) c.Assert(err, NotNil) c.Assert(alloc.Base(), Equals, int64(0)) diff --git a/meta/autoid/errors.go b/meta/autoid/errors.go new file mode 100644 index 0000000000000..44ef83650a202 --- /dev/null +++ b/meta/autoid/errors.go @@ -0,0 +1,32 @@ +// Copyright 2018 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package autoid + +import ( + "github.com/pingcap/parser/mysql" + "github.com/pingcap/parser/terror" +) + +// Error instances. +var ( + ErrAutoincReadFailed = terror.ClassAutoid.New(mysql.ErrAutoincReadFailed, mysql.MySQLErrName[mysql.ErrAutoincReadFailed]) +) + +func init() { + // Map error codes to mysql error codes. + tableMySQLErrCodes := map[terror.ErrCode]uint16{ + mysql.ErrAutoincReadFailed: mysql.ErrAutoincReadFailed, + } + terror.ErrClassToMySQLCodes[terror.ClassAutoid] = tableMySQLErrCodes +} diff --git a/meta/meta.go b/meta/meta.go index 901b69f450adc..41e93dda03d84 100644 --- a/meta/meta.go +++ b/meta/meta.go @@ -154,7 +154,7 @@ func (m *Meta) parseTableID(key string) (int64, error) { return n, errors.Trace(err) } -// GenAutoTableIDIDKeyValue generate meta key by dbID, tableID and coresponding value by autoID. +// GenAutoTableIDIDKeyValue generate meta key by dbID, tableID and corresponding value by autoID. func (m *Meta) GenAutoTableIDIDKeyValue(dbID, tableID, autoID int64) (key, value []byte) { dbKey := m.dbKey(dbID) autoTableIDKey := m.autoTableIDKey(tableID) diff --git a/session/session_test.go b/session/session_test.go index aa1e67a5b603e..263d8355bcc78 100644 --- a/session/session_test.go +++ b/session/session_test.go @@ -31,6 +31,7 @@ import ( "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/executor" "github.com/pingcap/tidb/kv" + "github.com/pingcap/tidb/meta/autoid" plannercore "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/privilege/privileges" "github.com/pingcap/tidb/session" @@ -756,6 +757,58 @@ func (s *testSessionSuite) TestAutoIncrementID(c *C) { tk.MustQuery("select last_insert_id(20)").Check(testkit.Rows(fmt.Sprint(20))) tk.MustQuery("select last_insert_id()").Check(testkit.Rows(fmt.Sprint(20))) + + // Corner cases for unsigned bigint auto_increment Columns. + 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`))") + 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.MustExec("insert into autoid values(18446744073709551614);") + _, 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)") + c.Assert(terror.ErrorEqual(err, autoid.ErrAutoincReadFailed), IsTrue) + + 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`))") + tk.MustExec("insert into autoid values()") + tk.MustQuery("select * from autoid").Check(testkit.Rows("1")) + tk.MustExec("insert into autoid values(5000)") + tk.MustQuery("select * from autoid").Check(testkit.Rows("1", "5000")) + _, err = tk.Exec("update autoid set auto_inc_id = 8000") + c.Assert(terror.ErrorEqual(err, kv.ErrKeyExists), IsTrue) + tk.MustQuery("select * from autoid").Check(testkit.Rows("1", "5000")) + tk.MustExec("update autoid set auto_inc_id = 9000 where auto_inc_id=1") + tk.MustQuery("select * from autoid").Check(testkit.Rows("9000", "5000")) + tk.MustExec("insert into autoid values()") + tk.MustQuery("select * from autoid").Check(testkit.Rows("9000", "5000", "9001")) + + // 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`))") + tk.MustExec("insert into autoid values(9223372036854775806);") + tk.MustQuery("select auto_inc_id, _tidb_rowid from autoid").Check(testkit.Rows("9223372036854775806 9223372036854775807")) + _, 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").Check(testkit.Rows("9223372036854775806 9223372036854775807")) + tk.MustQuery("select auto_inc_id, _tidb_rowid from autoid use index(auto_inc_id)").Check(testkit.Rows("9223372036854775806 9223372036854775807")) + + 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`))") + tk.MustExec("insert into autoid values()") + tk.MustQuery("select * from autoid").Check(testkit.Rows("1")) + tk.MustExec("insert into autoid values(5000)") + tk.MustQuery("select * from autoid").Check(testkit.Rows("1", "5000")) + _, err = tk.Exec("update autoid set auto_inc_id = 8000") + c.Assert(terror.ErrorEqual(err, kv.ErrKeyExists), IsTrue) + tk.MustQuery("select * from autoid").Check(testkit.Rows("1", "5000")) + tk.MustExec("update autoid set auto_inc_id = 9000 where auto_inc_id=1") + tk.MustQuery("select * from autoid").Check(testkit.Rows("9000", "5000")) + tk.MustExec("insert into autoid values()") + tk.MustQuery("select * from autoid").Check(testkit.Rows("9000", "5000", "9001")) } func (s *testSessionSuite) TestAutoIncrementWithRetry(c *C) { diff --git a/util/admin/admin_test.go b/util/admin/admin_test.go index c01e08ace9040..4946381308751 100644 --- a/util/admin/admin_test.go +++ b/util/admin/admin_test.go @@ -287,7 +287,7 @@ func (s *testSuite) TestScan(c *C) { c.Assert(err, IsNil) tbInfo := tbl.Meta() - alloc := autoid.NewAllocator(s.store, dbInfo.ID) + alloc := autoid.NewAllocator(s.store, dbInfo.ID, false) tb, err := tables.TableFromMeta(alloc, tbInfo) c.Assert(err, IsNil) indices := tb.Indices()