Skip to content

Commit

Permalink
*: fix 'Duplicate entry' error when @@auto_increment_increment and @@…
Browse files Browse the repository at this point in the history
…auto_increment_offset is set (#52626) (#53057)

close #52622
  • Loading branch information
ti-chi-bot authored May 23, 2024
1 parent 45d1613 commit 1cc3f9b
Show file tree
Hide file tree
Showing 5 changed files with 73 additions and 20 deletions.
5 changes: 3 additions & 2 deletions pkg/autoid_service/autoid.go
Original file line number Diff line number Diff line change
Expand Up @@ -200,7 +200,7 @@ func (alloc *autoIDValue) rebase4Unsigned(ctx context.Context,
return nil
}
// Satisfied by alloc.end, need to update alloc.base.
if requiredBase <= uint64(alloc.end) {
if requiredBase > uint64(alloc.base) && requiredBase <= uint64(alloc.end) {
alloc.base = int64(requiredBase)
return nil
}
Expand Down Expand Up @@ -243,7 +243,7 @@ func (alloc *autoIDValue) rebase4Signed(ctx context.Context, store kv.Storage, d
return nil
}
// Satisfied by alloc.end, need to update alloc.base.
if requiredBase <= alloc.end {
if requiredBase > alloc.base && requiredBase <= alloc.end {
alloc.base = requiredBase
return nil
}
Expand Down Expand Up @@ -501,6 +501,7 @@ func (s *Service) allocAutoID(ctx context.Context, req *autoid.AutoIDRequest) (*
if err1 != nil {
return err1
}
val.base = currentEnd
val.end = currentEnd
return nil
})
Expand Down
2 changes: 1 addition & 1 deletion pkg/executor/test/autoidtest/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -9,7 +9,7 @@ go_test(
],
flaky = True,
race = "on",
shard_count = 10,
shard_count = 11,
deps = [
"//pkg/autoid_service",
"//pkg/config",
Expand Down
58 changes: 49 additions & 9 deletions pkg/executor/test/autoidtest/autoid_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -606,7 +606,7 @@ func TestAutoIDIncrementAndOffset(t *testing.T) {
for _, str := range []string{"", " AUTO_ID_CACHE 1"} {
tk.MustExec(`create table io (a int key auto_increment)` + str)
tk.MustExec(`insert into io values (null),(null),(null)`)
tk.MustQuery(`select * from io`).Check(testkit.Rows("10", "15", "20"))
tk.MustQuery(`select * from io`).Check(testkit.Rows("1", "6", "11"))
tk.MustExec(`drop table io`)
}

Expand All @@ -616,23 +616,23 @@ func TestAutoIDIncrementAndOffset(t *testing.T) {
tk.Session().GetSessionVars().AutoIncrementOffset = 10
tk.Session().GetSessionVars().AutoIncrementIncrement = 2
tk.MustExec(`insert into io values (),(),()`)
tk.MustQuery(`select * from io`).Check(testkit.Rows("10", "12", "14"))
tk.MustQuery(`select * from io`).Check(testkit.Rows("1", "3", "5"))
tk.MustExec(`delete from io`)

// Test reset the increment.
tk.Session().GetSessionVars().AutoIncrementIncrement = 5
tk.MustExec(`insert into io values (),(),()`)
tk.MustQuery(`select * from io`).Check(testkit.Rows("15", "20", "25"))
tk.MustQuery(`select * from io`).Check(testkit.Rows("6", "11", "16"))
tk.MustExec(`delete from io`)

tk.Session().GetSessionVars().AutoIncrementIncrement = 10
tk.MustExec(`insert into io values (),(),()`)
tk.MustQuery(`select * from io`).Check(testkit.Rows("30", "40", "50"))
tk.MustQuery(`select * from io`).Check(testkit.Rows("20", "30", "40"))
tk.MustExec(`delete from io`)

tk.Session().GetSessionVars().AutoIncrementIncrement = 5
tk.MustExec(`insert into io values (),(),()`)
tk.MustQuery(`select * from io`).Check(testkit.Rows("55", "60", "65"))
tk.MustQuery(`select * from io`).Check(testkit.Rows("41", "46", "51"))
tk.MustExec(`drop table io`)
}

Expand All @@ -643,10 +643,10 @@ func TestAutoIDIncrementAndOffset(t *testing.T) {
tk.MustExec(`create table io (a int, b int auto_increment, key(b))` + str)
tk.MustExec(`insert into io(b) values (null),(null),(null)`)
// AutoID allocation will take increment and offset into consideration.
tk.MustQuery(`select b from io`).Check(testkit.Rows("10", "12", "14"))
tk.MustQuery(`select b from io`).Check(testkit.Rows("1", "3", "5"))
if str == "" {
// 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("6", "7", "8"))
} else {
// Separate row id and auto inc id, increment and offset works on auto inc id
tk.MustQuery(`select _tidb_rowid from io`).Check(testkit.Rows("1", "2", "3"))
Expand All @@ -655,9 +655,9 @@ func TestAutoIDIncrementAndOffset(t *testing.T) {

tk.Session().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 b from io`).Check(testkit.Rows("10", "20", "30"))
if str == "" {
tk.MustQuery(`select _tidb_rowid from io`).Check(testkit.Rows("41", "42", "43"))
tk.MustQuery(`select _tidb_rowid from io`).Check(testkit.Rows("31", "32", "33"))
} else {
tk.MustQuery(`select _tidb_rowid from io`).Check(testkit.Rows("4", "5", "6"))
}
Expand Down Expand Up @@ -806,3 +806,43 @@ func TestAutoIDConstraint(t *testing.T) {
tk.MustExec("create table tt2 (id int, c int auto_increment, key c_idx(c))")
tk.MustExec("alter table tt2 drop index c_idx")
}

func TestIssue52622(t *testing.T) {
store := testkit.CreateMockStore(t)

tk := testkit.NewTestKit(t, store)
tk.MustExec("use test")
tk.MustExec(`set @@auto_increment_increment = 66;`)
tk.MustExec(`set @@auto_increment_offset = 9527;`)

tk.MustQuery(`select @@auto_increment_increment;`).Check(testkit.Rows("66"))
tk.MustQuery(`select @@auto_increment_offset;`).Check(testkit.Rows("9527"))

for i := 0; i < 2; i++ {
createTableSQL := "create table issue52622 (id int primary key auto_increment, k int)"
if i == 0 {
createTableSQL = createTableSQL + " AUTO_ID_CACHE 1"
}

tk.MustExec(createTableSQL)
tk.MustExec("insert into issue52622 (k) values (1),(2),(3);")
tk.MustQuery("select * from issue52622").Check(testkit.Rows("1 1", "67 2", "133 3"))
if i == 0 {
tk.MustQuery("show create table issue52622").CheckContain("134")
}
tk.MustExec("insert into issue52622 (k) values (4);")
tk.MustQuery("select * from issue52622").Check(testkit.Rows("1 1", "67 2", "133 3", "199 4"))

tk.MustExec("truncate table issue52622;")
tk.MustExec("insert into issue52622 (k) values (1)")
tk.MustExec("insert into issue52622 (k) values (2)")
tk.MustExec("insert into issue52622 (k) values (3)")
if i == 0 {
tk.MustQuery("show create table issue52622").CheckContain("134")
}
tk.MustExec("insert into issue52622 (k) values (4);")
tk.MustQuery("select * from issue52622").Check(testkit.Rows("1 1", "67 2", "133 3", "199 4"))

tk.MustExec("drop table issue52622;")
}
}
1 change: 1 addition & 0 deletions pkg/table/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,7 @@ go_library(
"//pkg/parser/types",
"//pkg/sessionctx",
"//pkg/sessionctx/stmtctx",
"//pkg/sessionctx/variable",
"//pkg/types",
"//pkg/util/chunk",
"//pkg/util/dbterror",
Expand Down
27 changes: 19 additions & 8 deletions pkg/table/table.go
Original file line number Diff line number Diff line change
Expand Up @@ -27,6 +27,7 @@ import (
"github.com/pingcap/tidb/pkg/meta/autoid"
"github.com/pingcap/tidb/pkg/parser/model"
"github.com/pingcap/tidb/pkg/sessionctx"
"github.com/pingcap/tidb/pkg/sessionctx/variable"
"github.com/pingcap/tidb/pkg/types"
"github.com/pingcap/tidb/pkg/util/dbterror"
"github.com/pingcap/tidb/pkg/util/sqlexec"
Expand Down Expand Up @@ -206,12 +207,23 @@ type Table interface {
GetPartitionedTable() PartitionedTable
}

func getIncrementAndOffset(vars *variable.SessionVars) (int, int) {
increment := vars.AutoIncrementIncrement
offset := vars.AutoIncrementOffset
// When the value of auto_increment_offset is greater than that of auto_increment_increment,
// the value of auto_increment_offset is ignored.
// Ref https://dev.mysql.com/doc/refman/8.0/en/replication-options-source.html
if offset > increment {
offset = 1
}
return increment, offset
}

// AllocAutoIncrementValue allocates an auto_increment value for a new row.
func AllocAutoIncrementValue(ctx context.Context, t Table, sctx sessionctx.Context) (int64, error) {
r, ctx := tracing.StartRegionEx(ctx, "table.AllocAutoIncrementValue")
defer r.End()
increment := sctx.GetSessionVars().AutoIncrementIncrement
offset := sctx.GetSessionVars().AutoIncrementOffset
increment, offset := getIncrementAndOffset(sctx.GetSessionVars())
alloc := t.Allocators(sctx).Get(autoid.AutoIncrementType)
_, max, err := alloc.Alloc(ctx, uint64(1), int64(increment), int64(offset))
if err != nil {
Expand All @@ -222,18 +234,17 @@ func AllocAutoIncrementValue(ctx context.Context, t Table, sctx sessionctx.Conte

// AllocBatchAutoIncrementValue allocates batch auto_increment value for rows, returning firstID, increment and err.
// The caller can derive the autoID by adding increment to firstID for N-1 times.
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)
func AllocBatchAutoIncrementValue(ctx context.Context, t Table, sctx sessionctx.Context, N int) ( /* firstID */ int64 /* increment */, int64 /* err */, error) {
increment1, offset := getIncrementAndOffset(sctx.GetSessionVars())
alloc := t.Allocators(sctx).Get(autoid.AutoIncrementType)
min, max, err := alloc.Alloc(ctx, uint64(N), increment, offset)
min, max, err := alloc.Alloc(ctx, uint64(N), int64(increment1), int64(offset))
if err != nil {
return min, max, err
}
// SeekToFirstAutoIDUnSigned seeks to first autoID. Because AutoIncrement always allocate from 1,
// signed and unsigned value can be unified as the unsigned handle.
nr := int64(autoid.SeekToFirstAutoIDUnSigned(uint64(min), uint64(increment), uint64(offset)))
return nr, increment, nil
nr := int64(autoid.SeekToFirstAutoIDUnSigned(uint64(min), uint64(increment1), uint64(offset)))
return nr, int64(increment1), nil
}

// PhysicalTable is an abstraction for two kinds of table representation: partition or non-partitioned table.
Expand Down

0 comments on commit 1cc3f9b

Please sign in to comment.