Skip to content

Commit f8d5c54

Browse files
authored
lightning: fix id too large after parallel import (#57398) (#57928)
close #56814
1 parent 9f4dec9 commit f8d5c54

32 files changed

+568
-311
lines changed

Makefile

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -425,7 +425,7 @@ br_bins:
425425
@rm tmp_parser.go
426426

427427
data_parsers: tools/bin/vfsgendev br/pkg/lightning/mydump/parser_generated.go br_web
428-
PATH="$(GOPATH)/bin":"$(PATH)":"$(TOOLS)" protoc -I. -I"$(GOPATH)/src" br/pkg/lightning/checkpoints/checkpointspb/file_checkpoints.proto --gogofaster_out=.
428+
PATH="$(GOPATH)/bin":"$(PATH)":"$(TOOLS)" protoc -I. -I"$(GOMODCACHE)" br/pkg/lightning/checkpoints/checkpointspb/file_checkpoints.proto --gogofaster_out=.
429429
tools/bin/vfsgendev -source='"github.com/pingcap/tidb/br/pkg/lightning/web".Res' && mv res_vfsdata.go br/pkg/lightning/web/
430430

431431
build_dumpling:

Makefile.common

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -14,6 +14,7 @@
1414

1515
PROJECT=tidb
1616
GOPATH ?= $(shell go env GOPATH)
17+
GOMODCACHE ?= $(shell go env GOMODCACHE)
1718
P=8
1819

1920
# Ensure GOPATH is set before running build process.

br/pkg/lightning/backend/kv/allocator.go

Lines changed: 17 additions & 9 deletions
Original file line numberDiff line numberDiff line change
@@ -30,20 +30,28 @@ type panickingAllocator struct {
3030
ty autoid.AllocatorType
3131
}
3232

33-
// NewPanickingAllocators creates a PanickingAllocator shared by all allocation types.
33+
// NewPanickingAllocators creates a PanickingAllocator with default base values.
34+
func NewPanickingAllocators(sepAutoInc bool) autoid.Allocators {
35+
return NewPanickingAllocatorsWithBase(sepAutoInc, 0, 0, 0)
36+
}
37+
38+
// NewPanickingAllocatorsWithBase creates a PanickingAllocator shared by all allocation types.
3439
// we use this to collect the max id(either _tidb_rowid or auto_increment id or auto_random) used
3540
// during import, and we will use this info to do ALTER TABLE xxx AUTO_RANDOM_BASE or AUTO_INCREMENT
3641
// on post-process phase.
37-
// TODO: support save all bases in checkpoint.
38-
func NewPanickingAllocators(sepAutoInc bool, base int64) autoid.Allocators {
42+
func NewPanickingAllocatorsWithBase(sepAutoInc bool, autoRandBase, autoIncrBase,
43+
autoRowIDBase int64) autoid.Allocators {
3944
allocs := make([]autoid.Allocator, 0, 3)
40-
for _, t := range []autoid.AllocatorType{
41-
autoid.RowIDAllocType,
42-
autoid.AutoIncrementType,
43-
autoid.AutoRandomType,
45+
for _, t := range []struct {
46+
Type autoid.AllocatorType
47+
Base int64
48+
}{
49+
{Type: autoid.AutoRandomType, Base: autoRandBase},
50+
{Type: autoid.AutoIncrementType, Base: autoIncrBase},
51+
{Type: autoid.RowIDAllocType, Base: autoRowIDBase},
4452
} {
45-
pa := &panickingAllocator{ty: t}
46-
pa.base.Store(base)
53+
pa := &panickingAllocator{ty: t.Type}
54+
pa.base.Store(t.Base)
4755
allocs = append(allocs, pa)
4856
}
4957
return autoid.NewAllocators(sepAutoInc, allocs...)

br/pkg/lightning/backend/kv/allocator_test.go

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -22,7 +22,7 @@ import (
2222
)
2323

2424
func TestAllocator(t *testing.T) {
25-
alloc := NewPanickingAllocators(true, 0)
25+
alloc := NewPanickingAllocators(true)
2626
require.NoError(t, alloc.Get(autoid.RowIDAllocType).Rebase(nil, 123, false))
2727
// cannot revert back
2828
require.NoError(t, alloc.Get(autoid.RowIDAllocType).Rebase(nil, 100, false))

br/pkg/lightning/backend/kv/base_test.go

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -43,7 +43,7 @@ func TestLogKVConvertFailed(t *testing.T) {
4343
cols := []*model.ColumnInfo{c1}
4444
tblInfo := &model.TableInfo{ID: 1, Columns: cols, PKIsHandle: false, State: model.StatePublic}
4545
var tbl table.Table
46-
tbl, err = tables.TableFromMeta(NewPanickingAllocators(tblInfo.SepAutoInc(), 0), tblInfo)
46+
tbl, err = tables.TableFromMeta(NewPanickingAllocators(tblInfo.SepAutoInc()), tblInfo)
4747
require.NoError(t, err)
4848

4949
var baseKVEncoder *BaseKVEncoder

br/pkg/lightning/backend/kv/kv2sql_test.go

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -42,7 +42,7 @@ func TestIterRawIndexKeysClusteredPK(t *testing.T) {
4242
require.NoError(t, err)
4343
info.State = model.StatePublic
4444
require.True(t, info.IsCommonHandle)
45-
tbl, err := tables.TableFromMeta(kv.NewPanickingAllocators(info.SepAutoInc(), 0), info)
45+
tbl, err := tables.TableFromMeta(kv.NewPanickingAllocators(info.SepAutoInc()), info)
4646
require.NoError(t, err)
4747

4848
sessionOpts := &encode.SessionOptions{
@@ -81,7 +81,7 @@ func TestIterRawIndexKeysIntPK(t *testing.T) {
8181
require.NoError(t, err)
8282
info.State = model.StatePublic
8383
require.True(t, info.PKIsHandle)
84-
tbl, err := tables.TableFromMeta(kv.NewPanickingAllocators(info.SepAutoInc(), 0), info)
84+
tbl, err := tables.TableFromMeta(kv.NewPanickingAllocators(info.SepAutoInc()), info)
8585
require.NoError(t, err)
8686

8787
sessionOpts := &encode.SessionOptions{

br/pkg/lightning/backend/kv/sql2kv_test.go

Lines changed: 11 additions & 11 deletions
Original file line numberDiff line numberDiff line change
@@ -80,7 +80,7 @@ func TestEncode(t *testing.T) {
8080
c1 := &model.ColumnInfo{ID: 1, Name: model.NewCIStr("c1"), State: model.StatePublic, Offset: 0, FieldType: *types.NewFieldType(mysql.TypeTiny)}
8181
cols := []*model.ColumnInfo{c1}
8282
tblInfo := &model.TableInfo{ID: 1, Columns: cols, PKIsHandle: false, State: model.StatePublic}
83-
tbl, err := tables.TableFromMeta(lkv.NewPanickingAllocators(tblInfo.SepAutoInc(), 0), tblInfo)
83+
tbl, err := tables.TableFromMeta(lkv.NewPanickingAllocators(tblInfo.SepAutoInc()), tblInfo)
8484
require.NoError(t, err)
8585

8686
logger := log.Logger{Logger: zap.NewNop()}
@@ -163,7 +163,7 @@ func TestDecode(t *testing.T) {
163163
c1 := &model.ColumnInfo{ID: 1, Name: model.NewCIStr("c1"), State: model.StatePublic, Offset: 0, FieldType: *types.NewFieldType(mysql.TypeTiny)}
164164
cols := []*model.ColumnInfo{c1}
165165
tblInfo := &model.TableInfo{ID: 1, Columns: cols, PKIsHandle: false, State: model.StatePublic}
166-
tbl, err := tables.TableFromMeta(lkv.NewPanickingAllocators(tblInfo.SepAutoInc(), 0), tblInfo)
166+
tbl, err := tables.TableFromMeta(lkv.NewPanickingAllocators(tblInfo.SepAutoInc()), tblInfo)
167167
require.NoError(t, err)
168168
decoder, err := lkv.NewTableKVDecoder(tbl, "`test`.`c1`", &encode.SessionOptions{
169169
SQLMode: mysql.ModeStrictAllTables,
@@ -217,7 +217,7 @@ func TestDecodeIndex(t *testing.T) {
217217
State: model.StatePublic,
218218
PKIsHandle: false,
219219
}
220-
tbl, err := tables.TableFromMeta(lkv.NewPanickingAllocators(tblInfo.SepAutoInc(), 0), tblInfo)
220+
tbl, err := tables.TableFromMeta(lkv.NewPanickingAllocators(tblInfo.SepAutoInc()), tblInfo)
221221
if err != nil {
222222
fmt.Printf("error: %v", err.Error())
223223
}
@@ -262,7 +262,7 @@ func TestEncodeRowFormatV2(t *testing.T) {
262262
c1 := &model.ColumnInfo{ID: 1, Name: model.NewCIStr("c1"), State: model.StatePublic, Offset: 0, FieldType: *types.NewFieldType(mysql.TypeTiny)}
263263
cols := []*model.ColumnInfo{c1}
264264
tblInfo := &model.TableInfo{ID: 1, Columns: cols, PKIsHandle: false, State: model.StatePublic}
265-
tbl, err := tables.TableFromMeta(lkv.NewPanickingAllocators(tblInfo.SepAutoInc(), 0), tblInfo)
265+
tbl, err := tables.TableFromMeta(lkv.NewPanickingAllocators(tblInfo.SepAutoInc()), tblInfo)
266266
require.NoError(t, err)
267267

268268
rows := []types.Datum{
@@ -313,7 +313,7 @@ func TestEncodeTimestamp(t *testing.T) {
313313
}
314314
cols := []*model.ColumnInfo{c1}
315315
tblInfo := &model.TableInfo{ID: 1, Columns: cols, PKIsHandle: false, State: model.StatePublic}
316-
tbl, err := tables.TableFromMeta(lkv.NewPanickingAllocators(tblInfo.SepAutoInc(), 0), tblInfo)
316+
tbl, err := tables.TableFromMeta(lkv.NewPanickingAllocators(tblInfo.SepAutoInc()), tblInfo)
317317
require.NoError(t, err)
318318

319319
encoder, err := lkv.NewTableKVEncoder(&encode.EncodingConfig{
@@ -342,7 +342,7 @@ func TestEncodeTimestamp(t *testing.T) {
342342

343343
func TestEncodeDoubleAutoIncrement(t *testing.T) {
344344
tblInfo := mockTableInfo(t, "create table t (id double not null auto_increment, unique key `u_id` (`id`));")
345-
tbl, err := tables.TableFromMeta(lkv.NewPanickingAllocators(tblInfo.SepAutoInc(), 0), tblInfo)
345+
tbl, err := tables.TableFromMeta(lkv.NewPanickingAllocators(tblInfo.SepAutoInc()), tblInfo)
346346
require.NoError(t, err)
347347

348348
encoder, err := lkv.NewTableKVEncoder(&encode.EncodingConfig{
@@ -406,7 +406,7 @@ func TestEncodeMissingAutoValue(t *testing.T) {
406406
},
407407
} {
408408
tblInfo := mockTableInfo(t, testTblInfo.CreateStmt)
409-
tbl, err := tables.TableFromMeta(lkv.NewPanickingAllocators(tblInfo.SepAutoInc(), 0), tblInfo)
409+
tbl, err := tables.TableFromMeta(lkv.NewPanickingAllocators(tblInfo.SepAutoInc()), tblInfo)
410410
require.NoError(t, err)
411411

412412
encoder, err := lkv.NewTableKVEncoder(&encode.EncodingConfig{
@@ -458,7 +458,7 @@ func TestEncodeMissingAutoValue(t *testing.T) {
458458

459459
func TestEncodeExpressionColumn(t *testing.T) {
460460
tblInfo := mockTableInfo(t, "create table t (id varchar(40) not null DEFAULT uuid(), unique key `u_id` (`id`));")
461-
tbl, err := tables.TableFromMeta(lkv.NewPanickingAllocators(tblInfo.SepAutoInc(), 0), tblInfo)
461+
tbl, err := tables.TableFromMeta(lkv.NewPanickingAllocators(tblInfo.SepAutoInc()), tblInfo)
462462
require.NoError(t, err)
463463

464464
encoder, err := lkv.NewTableKVEncoder(&encode.EncodingConfig{
@@ -503,7 +503,7 @@ func mockTableInfo(t *testing.T, createSQL string) *model.TableInfo {
503503

504504
func TestDefaultAutoRandoms(t *testing.T) {
505505
tblInfo := mockTableInfo(t, "create table t (id bigint unsigned NOT NULL auto_random primary key clustered, a varchar(100));")
506-
tbl, err := tables.TableFromMeta(lkv.NewPanickingAllocators(tblInfo.SepAutoInc(), 0), tblInfo)
506+
tbl, err := tables.TableFromMeta(lkv.NewPanickingAllocators(tblInfo.SepAutoInc()), tblInfo)
507507
require.NoError(t, err)
508508
encoder, err := lkv.NewTableKVEncoder(&encode.EncodingConfig{
509509
Table: tbl,
@@ -541,7 +541,7 @@ func TestDefaultAutoRandoms(t *testing.T) {
541541

542542
func TestShardRowId(t *testing.T) {
543543
tblInfo := mockTableInfo(t, "create table t (s varchar(16)) shard_row_id_bits = 3;")
544-
tbl, err := tables.TableFromMeta(lkv.NewPanickingAllocators(tblInfo.SepAutoInc(), 0), tblInfo)
544+
tbl, err := tables.TableFromMeta(lkv.NewPanickingAllocators(tblInfo.SepAutoInc()), tblInfo)
545545
require.NoError(t, err)
546546
encoder, err := lkv.NewTableKVEncoder(&encode.EncodingConfig{
547547
Table: tbl,
@@ -703,7 +703,7 @@ func SetUpTest(b *testing.B) *benchSQL2KVSuite {
703703
tableInfo.State = model.StatePublic
704704

705705
// Construct the corresponding KV encoder.
706-
tbl, err := tables.TableFromMeta(lkv.NewPanickingAllocators(tableInfo.SepAutoInc(), 0), tableInfo)
706+
tbl, err := tables.TableFromMeta(lkv.NewPanickingAllocators(tableInfo.SepAutoInc()), tableInfo)
707707
require.NoError(b, err)
708708
encoder, err := lkv.NewTableKVEncoder(&encode.EncodingConfig{
709709
Table: tbl,

br/pkg/lightning/backend/local/duplicate_test.go

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -40,7 +40,7 @@ func TestBuildDupTask(t *testing.T) {
4040
info, err := ddl.MockTableInfo(mock.NewContext(), node[0].(*ast.CreateTableStmt), 1)
4141
require.NoError(t, err)
4242
info.State = model.StatePublic
43-
tbl, err := tables.TableFromMeta(lkv.NewPanickingAllocators(info.SepAutoInc(), 0), info)
43+
tbl, err := tables.TableFromMeta(lkv.NewPanickingAllocators(info.SepAutoInc()), info)
4444
require.NoError(t, err)
4545

4646
// Test build duplicate detecting task.

br/pkg/lightning/backend/tidb/tidb_test.go

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -67,7 +67,7 @@ func createMysqlSuite(t *testing.T) *mysqlSuite {
6767
cols = append(cols, col)
6868
}
6969
tblInfo := &model.TableInfo{ID: 1, Columns: cols, PKIsHandle: false, State: model.StatePublic}
70-
tbl, err := tables.TableFromMeta(kv.NewPanickingAllocators(tblInfo.SepAutoInc(), 0), tblInfo)
70+
tbl, err := tables.TableFromMeta(kv.NewPanickingAllocators(tblInfo.SepAutoInc()), tblInfo)
7171
require.NoError(t, err)
7272
cfg := config.NewConfig()
7373
cfg.Conflict.Strategy = config.ReplaceOnDup
@@ -286,7 +286,7 @@ func testStrictMode(t *testing.T) {
286286
ft.SetCharset(charset.CharsetASCII)
287287
col1 := &model.ColumnInfo{ID: 2, Name: model.NewCIStr("s1"), State: model.StatePublic, Offset: 1, FieldType: ft}
288288
tblInfo := &model.TableInfo{ID: 1, Columns: []*model.ColumnInfo{col0, col1}, PKIsHandle: false, State: model.StatePublic}
289-
tbl, err := tables.TableFromMeta(kv.NewPanickingAllocators(tblInfo.SepAutoInc(), 0), tblInfo)
289+
tbl, err := tables.TableFromMeta(kv.NewPanickingAllocators(tblInfo.SepAutoInc()), tblInfo)
290290
require.NoError(t, err)
291291

292292
ctx := context.Background()

br/pkg/lightning/checkpoints/BUILD.bazel

Lines changed: 1 addition & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -22,7 +22,6 @@ go_library(
2222
"//pkg/parser/model",
2323
"//pkg/types",
2424
"//pkg/util/chunk",
25-
"//pkg/util/mathutil",
2625
"//pkg/util/sqlexec",
2726
"@com_github_joho_sqltocsv//:sqltocsv",
2827
"@com_github_pingcap_errors//:errors",
@@ -42,7 +41,7 @@ go_test(
4241
embed = [":checkpoints"],
4342
flaky = True,
4443
race = "on",
45-
shard_count = 23,
44+
shard_count = 24,
4645
deps = [
4746
"//br/pkg/lightning/checkpoints/checkpointspb",
4847
"//br/pkg/lightning/config",

0 commit comments

Comments
 (0)