Skip to content

Commit

Permalink
executor: fix last_insert_id in auto_random mode (#15145) (#15149)
Browse files Browse the repository at this point in the history
  • Loading branch information
sre-bot authored Mar 5, 2020
1 parent 078fa64 commit 56ca69a
Show file tree
Hide file tree
Showing 6 changed files with 202 additions and 22 deletions.
2 changes: 1 addition & 1 deletion ddl/ddl_api.go
Original file line number Diff line number Diff line change
Expand Up @@ -1118,7 +1118,7 @@ func setTableAutoRandomBits(tbInfo *model.TableInfo, colDefs []*ast.ColumnDef) e
if autoRandBits == 0 {
return ErrInvalidAutoRandom.GenWithStackByArgs(autoid.AutoRandomNonPositive)
} else if autoRandBits >= maxFieldTypeBitsLength {
return ErrInvalidAutoRandom.GenWithStackByArgs(fmt.Sprintf(autoid.AutoRandomOverflowErrMsg, autoRandBits, maxFieldTypeBitsLength))
return ErrInvalidAutoRandom.GenWithStackByArgs(fmt.Sprintf(autoid.AutoRandomOverflowErrMsg, col.Name.Name.L, maxFieldTypeBitsLength, autoRandBits, col.Name.Name.L, maxFieldTypeBitsLength-1))
}
tbInfo.AutoRandomBits = autoRandBits
}
Expand Down
15 changes: 8 additions & 7 deletions ddl/serial_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -44,6 +44,7 @@ import (
"github.com/pingcap/tidb/util/testutil"
)

// Make it serial because config is modified in test cases.
var _ = SerialSuites(&testSerialSuite{})

type testSerialSuite struct {
Expand Down Expand Up @@ -807,8 +808,8 @@ func (s *testSerialSuite) TestAutoRandom(c *C) {
assertWithAutoInc := func(sql string) {
assertInvalidAutoRandomErr(sql, autoid.AutoRandomIncompatibleWithAutoIncErrMsg)
}
assertOverflow := func(sql string, autoRandBits, maxFieldLength uint64) {
assertInvalidAutoRandomErr(sql, autoid.AutoRandomOverflowErrMsg, autoRandBits, maxFieldLength)
assertOverflow := func(sql, colType string, autoRandBits, maxFieldLength uint64) {
assertInvalidAutoRandomErr(sql, autoid.AutoRandomOverflowErrMsg, colType, maxFieldLength, autoRandBits, colType, maxFieldLength-1)
}
assertModifyColType := func(sql string) {
assertInvalidAutoRandomErr(sql, autoid.AutoRandomModifyColTypeErrMsg)
Expand Down Expand Up @@ -852,11 +853,11 @@ func (s *testSerialSuite) TestAutoRandom(c *C) {
assertWithAutoInc("create table t (a bigint auto_random(3) auto_increment, primary key (a))")

// Overflow data type max length.
assertOverflow("create table t (a bigint auto_random(65) primary key)", 65, 64)
assertOverflow("create table t (a int auto_random(33) primary key)", 33, 32)
assertOverflow("create table t (a mediumint auto_random(25) primary key)", 25, 24)
assertOverflow("create table t (a smallint auto_random(17) primary key)", 17, 16)
assertOverflow("create table t (a tinyint auto_random(9) primary key)", 9, 8)
assertOverflow("create table t (a bigint auto_random(65) primary key)", "a", 65, 64)
assertOverflow("create table t (a int auto_random(33) primary key)", "a", 33, 32)
assertOverflow("create table t (a mediumint auto_random(25) primary key)", "a", 25, 24)
assertOverflow("create table t (a smallint auto_random(17) primary key)", "a", 17, 16)
assertOverflow("create table t (a tinyint auto_random(9) primary key)", "a", 9, 8)

assertNonPositive("create table t (a bigint auto_random(0) primary key)")

Expand Down
6 changes: 3 additions & 3 deletions executor/ddl_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -793,7 +793,7 @@ func (s *testAutoRandomSuite) TestAutoRandomBitsData(c *C) {

// Test explicit insert.
tk.MustExec("create table t (a tinyint primary key auto_random(2), b int)")
for i := 0; i < 100; i++ {
for i := 1; i <= 100; i++ {
tk.MustExec("insert into t values (?, ?)", i, i)
}
_, err = tk.Exec("insert into t (b) values (0)")
Expand Down Expand Up @@ -822,8 +822,8 @@ func (s *testAutoRandomSuite) TestAutoRandomBitsData(c *C) {
tk.MustExec("drop table t")

tk.MustExec("create table t (a tinyint primary key auto_random(2), b int)")
tk.MustExec("insert into t values (0, 2)")
tk.MustExec("update t set a = 31 where a = 0")
tk.MustExec("insert into t values (1, 2)")
tk.MustExec("update t set a = 31 where a = 1")
_, err = tk.Exec("insert into t (b) values (0)")
c.Assert(err, NotNil)
c.Assert(err.Error(), Equals, autoid.ErrAutoRandReadFailed.GenWithStackByArgs().Error())
Expand Down
40 changes: 30 additions & 10 deletions executor/insert_common.go
Original file line number Diff line number Diff line change
Expand Up @@ -717,28 +717,48 @@ func getAutoRecordID(d types.Datum, target *types.FieldType, isInsert bool) (int
}

func (e *InsertValues) adjustAutoRandomDatum(ctx context.Context, d types.Datum, hasValue bool, c *table.Column) (types.Datum, error) {
if !hasValue || d.IsNull() {
_, err := e.ctx.Txn(true)
var err error
var recordID int64
if !hasValue {
d.SetNull()
}
if !d.IsNull() {
recordID, err = getAutoRecordID(d, &c.FieldType, true)
if err != nil {
return types.Datum{}, errors.Trace(err)
return types.Datum{}, err
}
autoRandomID, err := e.allocAutoRandomID(&c.FieldType)
}
// Use the value if it's not null and not 0.
if recordID != 0 {
err = e.rebaseAutoRandomID(recordID, &c.FieldType)
if err != nil {
return types.Datum{}, err
}
d.SetAutoID(autoRandomID, c.Flag)
} else {
recordID, err := getAutoRecordID(d, &c.FieldType, true)
e.ctx.GetSessionVars().StmtCtx.InsertID = uint64(recordID)
d.SetAutoID(recordID, c.Flag)
return d, nil
}

// Change NULL to auto id.
// Change value 0 to auto id, if NoAutoValueOnZero SQL mode is not set.
if d.IsNull() || e.ctx.GetSessionVars().SQLMode&mysql.ModeNoAutoValueOnZero == 0 {
_, err := e.ctx.Txn(true)
if err != nil {
return types.Datum{}, err
return types.Datum{}, errors.Trace(err)
}
err = e.rebaseAutoRandomID(recordID, &c.FieldType)
recordID, err = e.allocAutoRandomID(&c.FieldType)
if err != nil {
return types.Datum{}, err
}
d.SetAutoID(recordID, c.Flag)
// It's compatible with mysql setting the first allocated autoID to lastInsertID.
// Cause autoID may be specified by user, judge only the first row is not suitable.
if e.lastInsertID == 0 {
e.lastInsertID = uint64(recordID)
}
}

d.SetAutoID(recordID, c.Flag)

casted, err := table.CastValue(e.ctx, d, c.ToInfo())
if err != nil {
return types.Datum{}, err
Expand Down
159 changes: 159 additions & 0 deletions executor/insert_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -15,10 +15,12 @@ package executor_test

import (
"fmt"
"strconv"
"strings"

. "github.com/pingcap/check"
"github.com/pingcap/parser/terror"
"github.com/pingcap/tidb/config"
"github.com/pingcap/tidb/sessionctx/variable"
"github.com/pingcap/tidb/table"
"github.com/pingcap/tidb/types"
Expand Down Expand Up @@ -861,3 +863,160 @@ func (s *testSuite3) TestAutoIDIncrementAndOffset(c *C) {
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]")
}

func (s *testAutoRandomSuite) TestAutoRandomID(c *C) {
allowAutoRandom := config.GetGlobalConfig().Experimental.AllowAutoRandom
if !allowAutoRandom {
config.GetGlobalConfig().Experimental.AllowAutoRandom = true
defer func() {
config.GetGlobalConfig().Experimental.AllowAutoRandom = false
}()
}

tk := testkit.NewTestKit(c, s.store)
tk.MustExec(`use test`)
tk.MustExec(`drop table if exists ar`)
tk.MustExec(`create table ar (id int key auto_random, name char(10))`)

tk.MustExec(`insert into ar(id) values (null)`)
rs := tk.MustQuery(`select id from ar`)
c.Assert(len(rs.Rows()), Equals, 1)
firstValue, err := strconv.Atoi(rs.Rows()[0][0].(string))
c.Assert(err, IsNil)
c.Assert(firstValue, Greater, 0)
tk.MustQuery(`select last_insert_id()`).Check(testkit.Rows(fmt.Sprintf("%d", firstValue)))
tk.MustExec(`delete from ar`)

tk.MustExec(`insert into ar(id) values (0)`)
rs = tk.MustQuery(`select id from ar`)
c.Assert(len(rs.Rows()), Equals, 1)
firstValue, err = strconv.Atoi(rs.Rows()[0][0].(string))
c.Assert(err, IsNil)
c.Assert(firstValue, Greater, 0)
tk.MustQuery(`select last_insert_id()`).Check(testkit.Rows(fmt.Sprintf("%d", firstValue)))
tk.MustExec(`delete from ar`)

tk.MustExec(`insert into ar(name) values ('a')`)
rs = tk.MustQuery(`select id from ar`)
c.Assert(len(rs.Rows()), Equals, 1)
firstValue, err = strconv.Atoi(rs.Rows()[0][0].(string))
c.Assert(err, IsNil)
c.Assert(firstValue, Greater, 0)
tk.MustQuery(`select last_insert_id()`).Check(testkit.Rows(fmt.Sprintf("%d", firstValue)))

tk.MustExec(`drop table ar`)
}

func (s *testAutoRandomSuite) TestMultiAutoRandomID(c *C) {
allowAutoRandom := config.GetGlobalConfig().Experimental.AllowAutoRandom
if !allowAutoRandom {
config.GetGlobalConfig().Experimental.AllowAutoRandom = true
defer func() {
config.GetGlobalConfig().Experimental.AllowAutoRandom = false
}()
}

tk := testkit.NewTestKit(c, s.store)
tk.MustExec(`use test`)
tk.MustExec(`drop table if exists ar`)
tk.MustExec(`create table ar (id int key auto_random, name char(10))`)

tk.MustExec(`insert into ar(id) values (null),(null),(null)`)
rs := tk.MustQuery(`select id from ar order by id`)
c.Assert(len(rs.Rows()), Equals, 3)
firstValue, err := strconv.Atoi(rs.Rows()[0][0].(string))
c.Assert(err, IsNil)
c.Assert(firstValue, Greater, 0)
c.Assert(rs.Rows()[1][0].(string), Equals, fmt.Sprintf("%d", firstValue+1))
c.Assert(rs.Rows()[2][0].(string), Equals, fmt.Sprintf("%d", firstValue+2))
tk.MustQuery(`select last_insert_id()`).Check(testkit.Rows(fmt.Sprintf("%d", firstValue)))
tk.MustExec(`delete from ar`)

tk.MustExec(`insert into ar(id) values (0),(0),(0)`)
rs = tk.MustQuery(`select id from ar order by id`)
c.Assert(len(rs.Rows()), Equals, 3)
firstValue, err = strconv.Atoi(rs.Rows()[0][0].(string))
c.Assert(err, IsNil)
c.Assert(firstValue, Greater, 0)
c.Assert(rs.Rows()[1][0].(string), Equals, fmt.Sprintf("%d", firstValue+1))
c.Assert(rs.Rows()[2][0].(string), Equals, fmt.Sprintf("%d", firstValue+2))
tk.MustQuery(`select last_insert_id()`).Check(testkit.Rows(fmt.Sprintf("%d", firstValue)))
tk.MustExec(`delete from ar`)

tk.MustExec(`insert into ar(name) values ('a'),('a'),('a')`)
rs = tk.MustQuery(`select id from ar order by id`)
c.Assert(len(rs.Rows()), Equals, 3)
firstValue, err = strconv.Atoi(rs.Rows()[0][0].(string))
c.Assert(err, IsNil)
c.Assert(firstValue, Greater, 0)
c.Assert(rs.Rows()[1][0].(string), Equals, fmt.Sprintf("%d", firstValue+1))
c.Assert(rs.Rows()[2][0].(string), Equals, fmt.Sprintf("%d", firstValue+2))
tk.MustQuery(`select last_insert_id()`).Check(testkit.Rows(fmt.Sprintf("%d", firstValue)))

tk.MustExec(`drop table ar`)
}

func (s *testAutoRandomSuite) TestAutoRandomIDAllowZero(c *C) {
allowAutoRandom := config.GetGlobalConfig().Experimental.AllowAutoRandom
if !allowAutoRandom {
config.GetGlobalConfig().Experimental.AllowAutoRandom = true
defer func() {
config.GetGlobalConfig().Experimental.AllowAutoRandom = false
}()
}

tk := testkit.NewTestKit(c, s.store)
tk.MustExec(`use test`)
tk.MustExec(`drop table if exists ar`)
tk.MustExec(`create table ar (id int key auto_random, name char(10))`)

rs := tk.MustQuery(`select @@session.sql_mode`)
sqlMode := rs.Rows()[0][0].(string)
tk.MustExec(fmt.Sprintf(`set session sql_mode="%s,%s"`, sqlMode, "NO_AUTO_VALUE_ON_ZERO"))

tk.MustExec(`insert into ar(id) values (0)`)
rs = tk.MustQuery(`select id from ar`)
c.Assert(len(rs.Rows()), Equals, 1)
firstValue, err := strconv.Atoi(rs.Rows()[0][0].(string))
c.Assert(err, IsNil)
c.Assert(firstValue, Equals, 0)
tk.MustQuery(`select last_insert_id()`).Check(testkit.Rows(fmt.Sprintf("%d", firstValue)))
tk.MustExec(`delete from ar`)

tk.MustExec(`insert into ar(id) values (null)`)
rs = tk.MustQuery(`select id from ar`)
c.Assert(len(rs.Rows()), Equals, 1)
firstValue, err = strconv.Atoi(rs.Rows()[0][0].(string))
c.Assert(err, IsNil)
c.Assert(firstValue, Greater, 0)
tk.MustQuery(`select last_insert_id()`).Check(testkit.Rows(fmt.Sprintf("%d", firstValue)))

tk.MustExec(`drop table ar`)
}

func (s *testAutoRandomSuite) TestAutoRandomIDExplicit(c *C) {
allowAutoRandom := config.GetGlobalConfig().Experimental.AllowAutoRandom
if !allowAutoRandom {
config.GetGlobalConfig().Experimental.AllowAutoRandom = true
defer func() {
config.GetGlobalConfig().Experimental.AllowAutoRandom = false
}()
}

tk := testkit.NewTestKit(c, s.store)
tk.MustExec(`use test`)
tk.MustExec(`drop table if exists ar`)
tk.MustExec(`create table ar (id int key auto_random, name char(10))`)

tk.MustExec(`insert into ar(id) values (1)`)
tk.MustQuery(`select id from ar`).Check(testkit.Rows("1"))
tk.MustQuery(`select last_insert_id()`).Check(testkit.Rows("0"))
tk.MustExec(`delete from ar`)

tk.MustExec(`insert into ar(id) values (1), (2)`)
tk.MustQuery(`select id from ar`).Check(testkit.Rows("1", "2"))
tk.MustQuery(`select last_insert_id()`).Check(testkit.Rows("0"))
tk.MustExec(`delete from ar`)

tk.MustExec(`drop table ar`)
}
2 changes: 1 addition & 1 deletion meta/autoid/errors.go
Original file line number Diff line number Diff line change
Expand Up @@ -51,7 +51,7 @@ const (
// AutoRandomIncompatibleWithDefaultValueErrMsg is reported when auto_random and default are specified on the same column.
AutoRandomIncompatibleWithDefaultValueErrMsg = "auto_random is incompatible with default"
// AutoRandomOverflowErrMsg is reported when auto_random is greater than max length of a MySQL data type.
AutoRandomOverflowErrMsg = "auto_random = %d will overflow. The max length of bits is %d"
AutoRandomOverflowErrMsg = "Bits of column `%s` is %d, but auto_random bits is %d. Max allowed auto_random bits for column `%s` is %d"
// AutoRandomModifyColTypeErrMsg is reported when a user is trying to modify the type of a column specified with auto_random.
AutoRandomModifyColTypeErrMsg = "modifying the auto_random column type is not supported"
// AutoRandomAlterErrMsg is reported when a user is trying to add/drop/modify the value of auto_random attribute.
Expand Down

0 comments on commit 56ca69a

Please sign in to comment.