Skip to content

Commit 42669ad

Browse files
authored
lightning: fix insert err after import for AUTO_ID_CACHE=1 and SHARD_ROW_ID_BITS (#52712) (#53704)
close #52654
1 parent 18fcc2f commit 42669ad

15 files changed

+129
-44
lines changed

br/pkg/lightning/backend/kv/BUILD.bazel

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -46,6 +46,7 @@ go_test(
4646
name = "kv_test",
4747
timeout = "short",
4848
srcs = [
49+
"allocator_test.go",
4950
"session_internal_test.go",
5051
"session_test.go",
5152
"sql2kv_test.go",

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

Lines changed: 17 additions & 12 deletions
Original file line numberDiff line numberDiff line change
@@ -26,33 +26,38 @@ import (
2626
// panickingAllocator is an ID allocator which panics on all operations except Rebase
2727
type panickingAllocator struct {
2828
autoid.Allocator
29-
base *int64
29+
base atomic.Int64
3030
ty autoid.AllocatorType
3131
}
3232

3333
// NewPanickingAllocators creates a PanickingAllocator shared by all allocation types.
3434
// we use this to collect the max id(either _tidb_rowid or auto_increment id or auto_random) used
3535
// during import, and we will use this info to do ALTER TABLE xxx AUTO_RANDOM_BASE or AUTO_INCREMENT
3636
// on post-process phase.
37-
func NewPanickingAllocators(base int64) autoid.Allocators {
38-
sharedBase := &base
39-
return autoid.NewAllocators(
40-
false,
41-
&panickingAllocator{base: sharedBase, ty: autoid.RowIDAllocType},
42-
&panickingAllocator{base: sharedBase, ty: autoid.AutoIncrementType},
43-
&panickingAllocator{base: sharedBase, ty: autoid.AutoRandomType},
44-
)
37+
// TODO: support save all bases in checkpoint.
38+
func NewPanickingAllocators(sepAutoInc bool, base int64) autoid.Allocators {
39+
allocs := make([]autoid.Allocator, 0, 3)
40+
for _, t := range []autoid.AllocatorType{
41+
autoid.RowIDAllocType,
42+
autoid.AutoIncrementType,
43+
autoid.AutoRandomType,
44+
} {
45+
pa := &panickingAllocator{ty: t}
46+
pa.base.Store(base)
47+
allocs = append(allocs, pa)
48+
}
49+
return autoid.NewAllocators(sepAutoInc, allocs...)
4550
}
4651

4752
// Rebase implements the autoid.Allocator interface
4853
func (alloc *panickingAllocator) Rebase(ctx context.Context, newBase int64, allocIDs bool) error {
4954
// CAS
5055
for {
51-
oldBase := atomic.LoadInt64(alloc.base)
56+
oldBase := alloc.base.Load()
5257
if newBase <= oldBase {
5358
break
5459
}
55-
if atomic.CompareAndSwapInt64(alloc.base, oldBase, newBase) {
60+
if alloc.base.CompareAndSwap(oldBase, newBase) {
5661
break
5762
}
5863
}
@@ -61,7 +66,7 @@ func (alloc *panickingAllocator) Rebase(ctx context.Context, newBase int64, allo
6166

6267
// Base implements the autoid.Allocator interface
6368
func (alloc *panickingAllocator) Base() int64 {
64-
return atomic.LoadInt64(alloc.base)
69+
return alloc.base.Load()
6570
}
6671

6772
func (alloc *panickingAllocator) GetType() autoid.AllocatorType {
Lines changed: 35 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,35 @@
1+
// Copyright 2024 PingCAP, Inc.
2+
//
3+
// Licensed under the Apache License, Version 2.0 (the "License");
4+
// you may not use this file except in compliance with the License.
5+
// You may obtain a copy of the License at
6+
//
7+
// http://www.apache.org/licenses/LICENSE-2.0
8+
//
9+
// Unless required by applicable law or agreed to in writing, software
10+
// distributed under the License is distributed on an "AS IS" BASIS,
11+
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12+
// See the License for the specific language governing permissions and
13+
// limitations under the License.
14+
15+
package kv
16+
17+
import (
18+
"testing"
19+
20+
"github.com/pingcap/tidb/meta/autoid"
21+
"github.com/stretchr/testify/require"
22+
)
23+
24+
func TestAllocator(t *testing.T) {
25+
alloc := NewPanickingAllocators(true, 0)
26+
require.NoError(t, alloc.Get(autoid.RowIDAllocType).Rebase(nil, 123, false))
27+
// cannot revert back
28+
require.NoError(t, alloc.Get(autoid.RowIDAllocType).Rebase(nil, 100, false))
29+
require.NoError(t, alloc.Get(autoid.AutoIncrementType).Rebase(nil, 456, false))
30+
require.NoError(t, alloc.Get(autoid.AutoRandomType).Rebase(nil, 789, false))
31+
32+
require.EqualValues(t, 123, alloc.Get(autoid.RowIDAllocType).Base())
33+
require.EqualValues(t, 456, alloc.Get(autoid.AutoIncrementType).Base())
34+
require.EqualValues(t, 789, alloc.Get(autoid.AutoRandomType).Base())
35+
}

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

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

8888
logger := log.Logger{Logger: zap.NewNop()}
@@ -153,7 +153,7 @@ func TestDecode(t *testing.T) {
153153
c1 := &model.ColumnInfo{ID: 1, Name: model.NewCIStr("c1"), State: model.StatePublic, Offset: 0, FieldType: *types.NewFieldType(mysql.TypeTiny)}
154154
cols := []*model.ColumnInfo{c1}
155155
tblInfo := &model.TableInfo{ID: 1, Columns: cols, PKIsHandle: false, State: model.StatePublic}
156-
tbl, err := tables.TableFromMeta(lkv.NewPanickingAllocators(0), tblInfo)
156+
tbl, err := tables.TableFromMeta(lkv.NewPanickingAllocators(tblInfo.SepAutoInc(), 0), tblInfo)
157157
require.NoError(t, err)
158158
decoder, err := lkv.NewTableKVDecoder(tbl, "`test`.`c1`", &lkv.SessionOptions{
159159
SQLMode: mysql.ModeStrictAllTables,
@@ -208,7 +208,7 @@ func TestDecodeIndex(t *testing.T) {
208208
State: model.StatePublic,
209209
PKIsHandle: false,
210210
}
211-
tbl, err := tables.TableFromMeta(lkv.NewPanickingAllocators(0), tblInfo)
211+
tbl, err := tables.TableFromMeta(lkv.NewPanickingAllocators(tblInfo.SepAutoInc(), 0), tblInfo)
212212
if err != nil {
213213
fmt.Printf("error: %v", err.Error())
214214
}
@@ -249,7 +249,7 @@ func TestEncodeRowFormatV2(t *testing.T) {
249249
c1 := &model.ColumnInfo{ID: 1, Name: model.NewCIStr("c1"), State: model.StatePublic, Offset: 0, FieldType: *types.NewFieldType(mysql.TypeTiny)}
250250
cols := []*model.ColumnInfo{c1}
251251
tblInfo := &model.TableInfo{ID: 1, Columns: cols, PKIsHandle: false, State: model.StatePublic}
252-
tbl, err := tables.TableFromMeta(lkv.NewPanickingAllocators(0), tblInfo)
252+
tbl, err := tables.TableFromMeta(lkv.NewPanickingAllocators(tblInfo.SepAutoInc(), 0), tblInfo)
253253
require.NoError(t, err)
254254

255255
logger := log.Logger{Logger: zap.NewNop()}
@@ -297,7 +297,7 @@ func TestEncodeTimestamp(t *testing.T) {
297297
}
298298
cols := []*model.ColumnInfo{c1}
299299
tblInfo := &model.TableInfo{ID: 1, Columns: cols, PKIsHandle: false, State: model.StatePublic}
300-
tbl, err := tables.TableFromMeta(lkv.NewPanickingAllocators(0), tblInfo)
300+
tbl, err := tables.TableFromMeta(lkv.NewPanickingAllocators(tblInfo.SepAutoInc(), 0), tblInfo)
301301
require.NoError(t, err)
302302

303303
logger := log.Logger{Logger: zap.NewNop()}
@@ -324,7 +324,7 @@ func TestEncodeTimestamp(t *testing.T) {
324324

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

330330
logger := log.Logger{Logger: zap.NewNop()}
@@ -388,7 +388,7 @@ func TestEncodeMissingAutoValue(t *testing.T) {
388388
},
389389
} {
390390
tblInfo := mockTableInfo(t, testTblInfo.CreateStmt)
391-
tbl, err := tables.TableFromMeta(lkv.NewPanickingAllocators(0), tblInfo)
391+
tbl, err := tables.TableFromMeta(lkv.NewPanickingAllocators(tblInfo.SepAutoInc(), 0), tblInfo)
392392
require.NoError(t, err)
393393

394394
encoder, err := lkv.NewTableKVEncoder(tbl, &lkv.SessionOptions{
@@ -436,7 +436,7 @@ func TestEncodeMissingAutoValue(t *testing.T) {
436436

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

442442
encoder, err := lkv.NewTableKVEncoder(tbl, &lkv.SessionOptions{
@@ -477,7 +477,7 @@ func mockTableInfo(t *testing.T, createSQL string) *model.TableInfo {
477477

478478
func TestDefaultAutoRandoms(t *testing.T) {
479479
tblInfo := mockTableInfo(t, "create table t (id bigint unsigned NOT NULL auto_random primary key clustered, a varchar(100));")
480-
tbl, err := tables.TableFromMeta(lkv.NewPanickingAllocators(0), tblInfo)
480+
tbl, err := tables.TableFromMeta(lkv.NewPanickingAllocators(tblInfo.SepAutoInc(), 0), tblInfo)
481481
require.NoError(t, err)
482482
encoder, err := lkv.NewTableKVEncoder(tbl, &lkv.SessionOptions{
483483
SQLMode: mysql.ModeStrictAllTables,
@@ -512,7 +512,7 @@ func TestDefaultAutoRandoms(t *testing.T) {
512512

513513
func TestShardRowId(t *testing.T) {
514514
tblInfo := mockTableInfo(t, "create table t (s varchar(16)) shard_row_id_bits = 3;")
515-
tbl, err := tables.TableFromMeta(lkv.NewPanickingAllocators(0), tblInfo)
515+
tbl, err := tables.TableFromMeta(lkv.NewPanickingAllocators(tblInfo.SepAutoInc(), 0), tblInfo)
516516
require.NoError(t, err)
517517
encoder, err := lkv.NewTableKVEncoder(tbl, &lkv.SessionOptions{
518518
SQLMode: mysql.ModeStrictAllTables,
@@ -671,7 +671,7 @@ func SetUpTest(b *testing.B) *benchSQL2KVSuite {
671671
tableInfo.State = model.StatePublic
672672

673673
// Construct the corresponding KV encoder.
674-
tbl, err := tables.TableFromMeta(lkv.NewPanickingAllocators(0), tableInfo)
674+
tbl, err := tables.TableFromMeta(lkv.NewPanickingAllocators(tableInfo.SepAutoInc(), 0), tableInfo)
675675
require.NoError(b, err)
676676
encoder, err := lkv.NewTableKVEncoder(tbl, &lkv.SessionOptions{SysVars: map[string]string{"tidb_row_format_version": "2"}}, nil, log.L())
677677
require.NoError(b, err)
@@ -734,7 +734,7 @@ func TestLogKVConvertFailed(t *testing.T) {
734734
c1 := &model.ColumnInfo{ID: 1, Name: modelName, State: modelState, Offset: 0, FieldType: modelFieldType}
735735
cols := []*model.ColumnInfo{c1}
736736
tblInfo := &model.TableInfo{ID: 1, Columns: cols, PKIsHandle: false, State: model.StatePublic}
737-
_, err = tables.TableFromMeta(lkv.NewPanickingAllocators(0), tblInfo)
737+
_, err = tables.TableFromMeta(lkv.NewPanickingAllocators(tblInfo.SepAutoInc(), 0), tblInfo)
738738
require.NoError(t, err)
739739

740740
var newString strings.Builder

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

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

4545
// 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
@@ -61,7 +61,7 @@ func createMysqlSuite(t *testing.T) *mysqlSuite {
6161
cols = append(cols, col)
6262
}
6363
tblInfo := &model.TableInfo{ID: 1, Columns: cols, PKIsHandle: false, State: model.StatePublic}
64-
tbl, err := tables.TableFromMeta(kv.NewPanickingAllocators(0), tblInfo)
64+
tbl, err := tables.TableFromMeta(kv.NewPanickingAllocators(tblInfo.SepAutoInc(), 0), tblInfo)
6565
require.NoError(t, err)
6666
backend := tidb.NewTiDBBackend(context.Background(), db, config.ReplaceOnDup, errormanager.New(nil, config.NewConfig(), log.L()))
6767
return &mysqlSuite{dbHandle: db, mockDB: mock, backend: backend, tbl: tbl}
@@ -226,7 +226,7 @@ func testStrictMode(t *testing.T) {
226226
ft.SetCharset(charset.CharsetASCII)
227227
col1 := &model.ColumnInfo{ID: 2, Name: model.NewCIStr("s1"), State: model.StatePublic, Offset: 1, FieldType: ft}
228228
tblInfo := &model.TableInfo{ID: 1, Columns: []*model.ColumnInfo{col0, col1}, PKIsHandle: false, State: model.StatePublic}
229-
tbl, err := tables.TableFromMeta(kv.NewPanickingAllocators(0), tblInfo)
229+
tbl, err := tables.TableFromMeta(kv.NewPanickingAllocators(tblInfo.SepAutoInc(), 0), tblInfo)
230230
require.NoError(t, err)
231231

232232
ctx := context.Background()

br/pkg/lightning/restore/get_pre_info.go

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -618,7 +618,7 @@ func (p *PreRestoreInfoGetterImpl) sampleDataFromTable(
618618
if err != nil {
619619
return 0.0, false, errors.Trace(err)
620620
}
621-
idAlloc := kv.NewPanickingAllocators(0)
621+
idAlloc := kv.NewPanickingAllocators(tableInfo.SepAutoInc(), 0)
622622
tbl, err := tables.TableFromMeta(idAlloc, tableInfo)
623623
if err != nil {
624624
return 0.0, false, errors.Trace(err)

br/pkg/lightning/restore/meta_manager.go

Lines changed: 26 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -254,6 +254,8 @@ func (m *dbTableMetaMgr) AllocTableRowIDs(ctx context.Context, rawRowIDMax int64
254254
if curStatus == metaStatusInitial {
255255
if needAutoID {
256256
// maxRowIDMax is the max row_id that other tasks has allocated, we need to rebase the global autoid base first.
257+
// TODO this is not right when AUTO_ID_CACHE=1 and have auto row id,
258+
// the id allocators are separated in this case.
257259
if err := rebaseGlobalAutoID(ctx, maxRowIDMax, m.tr, m.tr.dbInfo.ID, m.tr.tableInfo.Core); err != nil {
258260
return errors.Trace(err)
259261
}
@@ -1190,6 +1192,30 @@ func rebaseGlobalAutoID(ctx context.Context, newBase int64, r autoid.Requirement
11901192
return nil
11911193
}
11921194

1195+
// rebaseTableAllocators rebase the allocators of a table.
1196+
// This function only rebase a table allocator when its new base is given in
1197+
// `bases` param, else it will be skipped.
1198+
// base is the max id that have been used by the table, the next usable id will
1199+
// be base + 1, see Allocator.Alloc.
1200+
func rebaseTableAllocators(ctx context.Context, bases map[autoid.AllocatorType]int64, r autoid.Requirement, dbID int64,
1201+
tblInfo *model.TableInfo) error {
1202+
allocators, err := getGlobalAutoIDAlloc(r, dbID, tblInfo)
1203+
if err != nil {
1204+
return err
1205+
}
1206+
for _, alloc := range allocators {
1207+
base, ok := bases[alloc.GetType()]
1208+
if !ok {
1209+
continue
1210+
}
1211+
err = alloc.Rebase(ctx, base, false)
1212+
if err != nil {
1213+
return err
1214+
}
1215+
}
1216+
return nil
1217+
}
1218+
11931219
func getGlobalAutoIDAlloc(r autoid.Requirement, dbID int64, tblInfo *model.TableInfo) ([]autoid.Allocator, error) {
11941220
if r == nil || r.Store() == nil {
11951221
return nil, errors.New("internal error: kv store should not be nil")

br/pkg/lightning/restore/table_restore.go

Lines changed: 14 additions & 10 deletions
Original file line numberDiff line numberDiff line change
@@ -76,7 +76,7 @@ func NewTableRestore(
7676
etcdCli *clientv3.Client,
7777
logger log.Logger,
7878
) (*TableRestore, error) {
79-
idAlloc := kv.NewPanickingAllocators(cp.AllocBase)
79+
idAlloc := kv.NewPanickingAllocators(tableInfo.Core.SepAutoInc(), cp.AllocBase)
8080
tbl, err := tables.TableFromMeta(idAlloc, tableInfo.Core)
8181
if err != nil {
8282
return nil, errors.Annotatef(err, "failed to tables.TableFromMeta %s", tableName)
@@ -754,26 +754,30 @@ func (tr *TableRestore) postProcess(
754754
if cp.Status < checkpoints.CheckpointStatusAlteredAutoInc {
755755
tblInfo := tr.tableInfo.Core
756756
var err error
757+
// TODO why we have to rebase id for tidb backend??? remove it later.
757758
if tblInfo.ContainsAutoRandomBits() {
758759
ft := &common.GetAutoRandomColumn(tblInfo).FieldType
759760
shardFmt := autoid.NewShardIDFormat(ft, tblInfo.AutoRandomBits, tblInfo.AutoRandomRangeBits)
760761
maxCap := shardFmt.IncrementalBitsCapacity()
761762
err = AlterAutoRandom(ctx, rc.tidbGlue.GetSQLExecutor(), tr.tableName, uint64(tr.alloc.Get(autoid.AutoRandomType).Base())+1, maxCap)
762763
} else if common.TableHasAutoRowID(tblInfo) || tblInfo.GetAutoIncrementColInfo() != nil {
763-
// only alter auto increment id iff table contains auto-increment column or generated handle.
764-
// ALTER TABLE xxx AUTO_INCREMENT = yyy has a bad naming.
765-
// if a table has implicit _tidb_rowid column & tbl.SepAutoID=false, then it works on _tidb_rowid
766-
// allocator, even if the table has NO auto-increment column.
767-
newBase := uint64(tr.alloc.Get(autoid.RowIDAllocType).Base()) + 1
768-
err = AlterAutoIncrement(ctx, rc.tidbGlue.GetSQLExecutor(), tr.tableName, newBase)
769-
770-
if err == nil && isLocalBackend(rc.cfg) {
764+
if isLocalBackend(rc.cfg) {
771765
// for TiDB version >= 6.5.0, a table might have separate allocators for auto_increment column and _tidb_rowid,
772766
// especially when a table has auto_increment non-clustered PK, it will use both allocators.
773767
// And in this case, ALTER TABLE xxx AUTO_INCREMENT = xxx only works on the allocator of auto_increment column,
774768
// not for allocator of _tidb_rowid.
775769
// So we need to rebase IDs for those 2 allocators explicitly.
776-
err = rebaseGlobalAutoID(ctx, adjustIDBase(newBase), tr, tr.dbInfo.ID, tr.tableInfo.Core)
770+
err = rebaseTableAllocators(ctx, map[autoid.AllocatorType]int64{
771+
autoid.RowIDAllocType: tr.alloc.Get(autoid.RowIDAllocType).Base(),
772+
autoid.AutoIncrementType: tr.alloc.Get(autoid.AutoIncrementType).Base(),
773+
}, tr, tr.dbInfo.ID, tr.tableInfo.Core)
774+
} else {
775+
// only alter auto increment id iff table contains auto-increment column or generated handle.
776+
// ALTER TABLE xxx AUTO_INCREMENT = yyy has a bad naming.
777+
// if a table has implicit _tidb_rowid column & tbl.SepAutoID=false, then it works on _tidb_rowid
778+
// allocator, even if the table has NO auto-increment column.
779+
newBase := uint64(tr.alloc.Get(autoid.RowIDAllocType).Base()) + 1
780+
err = AlterAutoIncrement(ctx, rc.tidbGlue.GetSQLExecutor(), tr.tableName, newBase)
777781
}
778782
}
779783
saveCpErr := rc.saveStatusCheckpoint(ctx, tr.tableName, checkpoints.WholeTableEngineID, err, checkpoints.CheckpointStatusAlteredAutoInc)

br/pkg/lightning/restore/table_restore_test.go

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -357,7 +357,7 @@ func (s *tableRestoreSuite) TestRestoreEngineFailed() {
357357
err := s.tr.populateChunks(ctx, rc, cp)
358358
require.NoError(s.T(), err)
359359

360-
tbl, err := tables.TableFromMeta(kv.NewPanickingAllocators(0), s.tableInfo.Core)
360+
tbl, err := tables.TableFromMeta(kv.NewPanickingAllocators(s.tableInfo.Core.SepAutoInc(), 0), s.tableInfo.Core)
361361
require.NoError(s.T(), err)
362362
_, indexUUID := backend.MakeUUID("`db`.`table`", -1)
363363
_, dataUUID := backend.MakeUUID("`db`.`table`", 0)
@@ -1410,7 +1410,7 @@ func (s *tableRestoreSuite) TestEstimate() {
14101410
controller := gomock.NewController(s.T())
14111411
defer controller.Finish()
14121412
mockBackend := mock.NewMockAbstractBackend(controller)
1413-
idAlloc := kv.NewPanickingAllocators(0)
1413+
idAlloc := kv.NewPanickingAllocators(s.tableInfo.Core.SepAutoInc(), 0)
14141414
tbl, err := tables.TableFromMeta(idAlloc, s.tableInfo.Core)
14151415
require.NoError(s.T(), err)
14161416

0 commit comments

Comments
 (0)