From b6cf59e4cd83f6a69ea148802da054557396e095 Mon Sep 17 00:00:00 2001 From: lance6716 Date: Thu, 26 Sep 2024 20:13:28 +0800 Subject: [PATCH 1/5] This is an automated cherry-pick of #56345 Signed-off-by: ti-chi-bot --- br/pkg/lightning/backend/local/local.go | 24 +++- br/pkg/lightning/backend/local/local_test.go | 44 ++++++++ br/pkg/lightning/backend/local/region_job.go | 4 + .../addindextest/integration_test.go | 105 ++++++++++++++++++ 4 files changed, 174 insertions(+), 3 deletions(-) diff --git a/br/pkg/lightning/backend/local/local.go b/br/pkg/lightning/backend/local/local.go index 5e2c6ddd72831..a5449664b1020 100644 --- a/br/pkg/lightning/backend/local/local.go +++ b/br/pkg/lightning/backend/local/local.go @@ -76,9 +76,6 @@ const ( dialTimeout = 5 * time.Minute maxRetryTimes = 5 defaultRetryBackoffTime = 3 * time.Second - // maxWriteAndIngestRetryTimes is the max retry times for write and ingest. - // A large retry times is for tolerating tikv cluster failures. - maxWriteAndIngestRetryTimes = 30 gRPCKeepAliveTime = 10 * time.Minute gRPCKeepAliveTimeout = 5 * time.Minute @@ -111,6 +108,10 @@ var ( errorEngineClosed = errors.New("engine is closed") maxRetryBackoffSecond = 30 + + // MaxWriteAndIngestRetryTimes is the max retry times for write and ingest. + // A large retry times is for tolerating tikv cluster failures. + MaxWriteAndIngestRetryTimes = 30 ) // ImportClientFactory is factory to create new import client for specific store. @@ -1523,11 +1524,28 @@ func (local *Backend) doImport(ctx context.Context, engine *Engine, regionRanges switch job.stage { case regionScanned, wrote: job.retryCount++ +<<<<<<< HEAD:br/pkg/lightning/backend/local/local.go if job.retryCount > maxWriteAndIngestRetryTimes { firstErr.Set(job.lastRetryableErr) workerCancel() jobWg.Done() continue +======= + if job.retryCount > MaxWriteAndIngestRetryTimes { + job.done(&jobWg) + lastErr := job.lastRetryableErr + intest.Assert(lastErr != nil, "lastRetryableErr should not be nil") + if lastErr == nil { + lastErr = errors.New("retry limit exceeded") + log.FromContext(ctx).Error( + "lastRetryableErr should not be nil", + logutil.Key("startKey", job.keyRange.Start), + logutil.Key("endKey", job.keyRange.End), + zap.Stringer("stage", job.stage), + zap.Error(lastErr)) + } + return lastErr +>>>>>>> 448d56910cd (lightning: fix forget to set lastRetryableErr when ingest RPC fail (#56345)):pkg/lightning/backend/local/local.go } // max retry backoff time: 2+4+8+16+30*26=810s sleepSecond := math.Pow(2, float64(job.retryCount)) diff --git a/br/pkg/lightning/backend/local/local_test.go b/br/pkg/lightning/backend/local/local_test.go index c54ceac27ef99..3699c74aed389 100644 --- a/br/pkg/lightning/backend/local/local_test.go +++ b/br/pkg/lightning/backend/local/local_test.go @@ -1919,9 +1919,15 @@ func TestDoImport(t *testing.T) { {"a", "b"}: { jobs: []*regionJob{ { +<<<<<<< HEAD:br/pkg/lightning/backend/local/local_test.go keyRange: Range{start: []byte{'a'}, end: []byte{'b'}}, engine: &Engine{}, retryCount: maxWriteAndIngestRetryTimes - 1, +======= + keyRange: common.Range{Start: []byte{'a'}, End: []byte{'b'}}, + ingestData: &Engine{}, + retryCount: MaxWriteAndIngestRetryTimes - 1, +>>>>>>> 448d56910cd (lightning: fix forget to set lastRetryableErr when ingest RPC fail (#56345)):pkg/lightning/backend/local/local_test.go injected: getSuccessInjectedBehaviour(), }, }, @@ -1929,9 +1935,15 @@ func TestDoImport(t *testing.T) { {"b", "c"}: { jobs: []*regionJob{ { +<<<<<<< HEAD:br/pkg/lightning/backend/local/local_test.go keyRange: Range{start: []byte{'b'}, end: []byte{'c'}}, engine: &Engine{}, retryCount: maxWriteAndIngestRetryTimes - 1, +======= + keyRange: common.Range{Start: []byte{'b'}, End: []byte{'c'}}, + ingestData: &Engine{}, + retryCount: MaxWriteAndIngestRetryTimes - 1, +>>>>>>> 448d56910cd (lightning: fix forget to set lastRetryableErr when ingest RPC fail (#56345)):pkg/lightning/backend/local/local_test.go injected: getSuccessInjectedBehaviour(), }, }, @@ -1939,9 +1951,15 @@ func TestDoImport(t *testing.T) { {"c", "d"}: { jobs: []*regionJob{ { +<<<<<<< HEAD:br/pkg/lightning/backend/local/local_test.go keyRange: Range{start: []byte{'c'}, end: []byte{'d'}}, engine: &Engine{}, retryCount: maxWriteAndIngestRetryTimes - 2, +======= + keyRange: common.Range{Start: []byte{'c'}, End: []byte{'d'}}, + ingestData: &Engine{}, + retryCount: MaxWriteAndIngestRetryTimes - 2, +>>>>>>> 448d56910cd (lightning: fix forget to set lastRetryableErr when ingest RPC fail (#56345)):pkg/lightning/backend/local/local_test.go injected: []injectedBehaviour{ { write: injectedWriteBehaviour{ @@ -1992,13 +2010,39 @@ func TestRegionJobResetRetryCounter(t *testing.T) { keyRange: Range{start: []byte{'c'}, end: []byte{'c', '2'}}, engine: &Engine{}, injected: getNeedRescanWhenIngestBehaviour(), +<<<<<<< HEAD:br/pkg/lightning/backend/local/local_test.go retryCount: maxWriteAndIngestRetryTimes, +======= + retryCount: MaxWriteAndIngestRetryTimes, + region: &split.RegionInfo{ + Region: &metapb.Region{ + Peers: []*metapb.Peer{ + {Id: 1, StoreId: 1}, + {Id: 2, StoreId: 2}, + {Id: 3, StoreId: 3}, + }, + }, + }, +>>>>>>> 448d56910cd (lightning: fix forget to set lastRetryableErr when ingest RPC fail (#56345)):pkg/lightning/backend/local/local_test.go }, { keyRange: Range{start: []byte{'c', '2'}, end: []byte{'d'}}, engine: &Engine{}, injected: getSuccessInjectedBehaviour(), +<<<<<<< HEAD:br/pkg/lightning/backend/local/local_test.go retryCount: maxWriteAndIngestRetryTimes, +======= + retryCount: MaxWriteAndIngestRetryTimes, + region: &split.RegionInfo{ + Region: &metapb.Region{ + Peers: []*metapb.Peer{ + {Id: 4, StoreId: 4}, + {Id: 5, StoreId: 5}, + {Id: 6, StoreId: 6}, + }, + }, + }, +>>>>>>> 448d56910cd (lightning: fix forget to set lastRetryableErr when ingest RPC fail (#56345)):pkg/lightning/backend/local/local_test.go }, }, }, diff --git a/br/pkg/lightning/backend/local/region_job.go b/br/pkg/lightning/backend/local/region_job.go index a021937bdcad9..e332a2d3e59cd 100644 --- a/br/pkg/lightning/backend/local/region_job.go +++ b/br/pkg/lightning/backend/local/region_job.go @@ -451,6 +451,7 @@ func (local *Backend) ingest(ctx context.Context, j *regionJob) (err error) { log.FromContext(ctx).Warn("meet underlying error, will retry ingest", log.ShortError(err), logutil.SSTMetas(j.writeResult.sstMeta), logutil.Region(j.region.Region), logutil.Leader(j.region.Leader)) + j.lastRetryableErr = err continue } canContinue, err := j.convertStageOnIngestError(resp) @@ -501,6 +502,9 @@ func (local *Backend) checkWriteStall( // doIngest send ingest commands to TiKV based on regionJob.writeResult.sstMeta. // When meet error, it will remove finished sstMetas before return. func (local *Backend) doIngest(ctx context.Context, j *regionJob) (*sst.IngestResponse, error) { + failpoint.Inject("doIngestFailed", func() { + failpoint.Return(nil, errors.New("injected error")) + }) clientFactory := local.importClientFactory supportMultiIngest := local.supportMultiIngest shouldCheckWriteStall := local.ShouldCheckWriteStall diff --git a/tests/realtikvtest/addindextest/integration_test.go b/tests/realtikvtest/addindextest/integration_test.go index bb6af7e925673..06a30e00ef65f 100644 --- a/tests/realtikvtest/addindextest/integration_test.go +++ b/tests/realtikvtest/addindextest/integration_test.go @@ -565,3 +565,108 @@ func TestAddUniqueIndexDuplicatedError(t *testing.T) { tk.MustExec("INSERT INTO `b1cce552` (`f5d9aecb`, `d9337060`, `4c74082f`, `9215adc3`, `85ad5a07`, `8c60260f`, `8069da7b`, `91e218e1`) VALUES ('2031-12-22 06:44:52', 'duplicatevalue', 2028, NULL, 846, 'N6QD1=@ped@owVoJx', '9soPM2d6H', 'Tv%'), ('2031-12-22 06:44:52', 'duplicatevalue', 2028, NULL, 9052, '_HWaf#gD!bw', '9soPM2d6H', 'Tv%');") tk.MustGetErrCode("ALTER TABLE `b1cce552` ADD unique INDEX `65290727` (`4c74082f`, `d9337060`, `8069da7b`);", errno.ErrDupEntry) } +<<<<<<< HEAD:tests/realtikvtest/addindextest/integration_test.go +======= + +func TestFirstLitSlowStart(t *testing.T) { + store := realtikvtest.CreateMockStoreAndSetup(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("drop database if exists addindexlit;") + tk.MustExec("create database addindexlit;") + tk.MustExec("use addindexlit;") + tk.MustExec(`set global tidb_ddl_enable_fast_reorg=on;`) + tk.MustExec(`set global tidb_enable_dist_task=off;`) + + tk.MustExec("create table t(a int, b int);") + tk.MustExec("insert into t values (1, 1), (2, 2), (3, 3);") + tk.MustExec("create table t2(a int, b int);") + tk.MustExec("insert into t2 values (1, 1), (2, 2), (3, 3);") + + tk1 := testkit.NewTestKit(t, store) + tk1.MustExec("use addindexlit;") + + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/pkg/ddl/ingest/beforeCreateLocalBackend", "1*return()")) + t.Cleanup(func() { + require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/pkg/ddl/ingest/beforeCreateLocalBackend")) + }) + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/pkg/ddl/ownerResignAfterDispatchLoopCheck", "return()")) + t.Cleanup(func() { + require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/pkg/ddl/ownerResignAfterDispatchLoopCheck")) + }) + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/pkg/lightning/backend/local/slowCreateFS", "return()")) + t.Cleanup(func() { + require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/pkg/lightning/backend/local/slowCreateFS")) + }) + + var wg sync.WaitGroup + wg.Add(2) + go func() { + defer wg.Done() + tk.MustExec("alter table t add unique index idx(a);") + }() + go func() { + defer wg.Done() + tk1.MustExec("alter table t2 add unique index idx(a);") + }() + wg.Wait() +} + +func TestConcFastReorg(t *testing.T) { + store := realtikvtest.CreateMockStoreAndSetup(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("drop database if exists addindexlit;") + tk.MustExec("create database addindexlit;") + tk.MustExec("use addindexlit;") + tk.MustExec(`set global tidb_ddl_enable_fast_reorg=on;`) + + tblNum := 10 + for i := 0; i < tblNum; i++ { + tk.MustExec(fmt.Sprintf("create table t%d(a int);", i)) + } + + var wg sync.WaitGroup + wg.Add(tblNum) + for i := 0; i < tblNum; i++ { + i := i + go func() { + defer wg.Done() + tk2 := testkit.NewTestKit(t, store) + tk2.MustExec("use addindexlit;") + tk2.MustExec(fmt.Sprintf("insert into t%d values (1), (2), (3);", i)) + + if i%2 == 0 { + tk2.MustExec(fmt.Sprintf("alter table t%d add index idx(a);", i)) + } else { + tk2.MustExec(fmt.Sprintf("alter table t%d add unique index idx(a);", i)) + } + }() + } + + wg.Wait() +} + +func TestIssue55808(t *testing.T) { + store := realtikvtest.CreateMockStoreAndSetup(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("drop database if exists addindexlit;") + tk.MustExec("create database addindexlit;") + tk.MustExec("use addindexlit;") + tk.MustExec(`set global tidb_ddl_enable_fast_reorg=on;`) + tk.MustExec("set global tidb_enable_dist_task = off;") + tk.MustExec("set global tidb_ddl_error_count_limit = 0") + + backup := local.MaxWriteAndIngestRetryTimes + local.MaxWriteAndIngestRetryTimes = 1 + t.Cleanup(func() { + local.MaxWriteAndIngestRetryTimes = backup + }) + + tk.MustExec("create table t (a int primary key, b int);") + for i := 0; i < 4; i++ { + tk.MustExec(fmt.Sprintf("insert into t values (%d, %d);", i*10000, i*10000)) + } + testfailpoint.Enable(t, "github.com/pingcap/tidb/pkg/lightning/backend/local/doIngestFailed", "return()") + err := tk.ExecToErr("alter table t add index idx(a);") + require.ErrorContains(t, err, "injected error") +} +>>>>>>> 448d56910cd (lightning: fix forget to set lastRetryableErr when ingest RPC fail (#56345)):tests/realtikvtest/addindextest3/ingest_test.go From 01c83dc4a2ea00566f2b0f7b0aa5768e2d040ff7 Mon Sep 17 00:00:00 2001 From: lance6716 Date: Tue, 29 Oct 2024 13:55:51 +0800 Subject: [PATCH 2/5] fix conflict Signed-off-by: lance6716 --- br/pkg/lightning/backend/local/local.go | 23 +++++----- br/pkg/lightning/backend/local/local_test.go | 44 -------------------- 2 files changed, 10 insertions(+), 57 deletions(-) diff --git a/br/pkg/lightning/backend/local/local.go b/br/pkg/lightning/backend/local/local.go index a5449664b1020..7b2b3bb96e980 100644 --- a/br/pkg/lightning/backend/local/local.go +++ b/br/pkg/lightning/backend/local/local.go @@ -56,6 +56,7 @@ import ( "github.com/pingcap/tidb/tablecodec" "github.com/pingcap/tidb/util/codec" "github.com/pingcap/tidb/util/engine" + "github.com/pingcap/tidb/util/intest" "github.com/pingcap/tidb/util/mathutil" "github.com/tikv/client-go/v2/oracle" tikvclient "github.com/tikv/client-go/v2/tikv" @@ -1524,28 +1525,24 @@ func (local *Backend) doImport(ctx context.Context, engine *Engine, regionRanges switch job.stage { case regionScanned, wrote: job.retryCount++ -<<<<<<< HEAD:br/pkg/lightning/backend/local/local.go - if job.retryCount > maxWriteAndIngestRetryTimes { - firstErr.Set(job.lastRetryableErr) - workerCancel() - jobWg.Done() - continue -======= if job.retryCount > MaxWriteAndIngestRetryTimes { - job.done(&jobWg) lastErr := job.lastRetryableErr - intest.Assert(lastErr != nil, "lastRetryableErr should not be nil") if lastErr == nil { + if intest.InTest { + panic("lastRetryableErr should not be nil") + } lastErr = errors.New("retry limit exceeded") log.FromContext(ctx).Error( "lastRetryableErr should not be nil", - logutil.Key("startKey", job.keyRange.Start), - logutil.Key("endKey", job.keyRange.End), + logutil.Key("startKey", job.keyRange.start), + logutil.Key("endKey", job.keyRange.end), zap.Stringer("stage", job.stage), zap.Error(lastErr)) } - return lastErr ->>>>>>> 448d56910cd (lightning: fix forget to set lastRetryableErr when ingest RPC fail (#56345)):pkg/lightning/backend/local/local.go + firstErr.Set(lastErr) + workerCancel() + jobWg.Done() + continue } // max retry backoff time: 2+4+8+16+30*26=810s sleepSecond := math.Pow(2, float64(job.retryCount)) diff --git a/br/pkg/lightning/backend/local/local_test.go b/br/pkg/lightning/backend/local/local_test.go index 3699c74aed389..64bdf94dcd96d 100644 --- a/br/pkg/lightning/backend/local/local_test.go +++ b/br/pkg/lightning/backend/local/local_test.go @@ -1919,15 +1919,9 @@ func TestDoImport(t *testing.T) { {"a", "b"}: { jobs: []*regionJob{ { -<<<<<<< HEAD:br/pkg/lightning/backend/local/local_test.go keyRange: Range{start: []byte{'a'}, end: []byte{'b'}}, engine: &Engine{}, - retryCount: maxWriteAndIngestRetryTimes - 1, -======= - keyRange: common.Range{Start: []byte{'a'}, End: []byte{'b'}}, - ingestData: &Engine{}, retryCount: MaxWriteAndIngestRetryTimes - 1, ->>>>>>> 448d56910cd (lightning: fix forget to set lastRetryableErr when ingest RPC fail (#56345)):pkg/lightning/backend/local/local_test.go injected: getSuccessInjectedBehaviour(), }, }, @@ -1935,15 +1929,9 @@ func TestDoImport(t *testing.T) { {"b", "c"}: { jobs: []*regionJob{ { -<<<<<<< HEAD:br/pkg/lightning/backend/local/local_test.go keyRange: Range{start: []byte{'b'}, end: []byte{'c'}}, engine: &Engine{}, - retryCount: maxWriteAndIngestRetryTimes - 1, -======= - keyRange: common.Range{Start: []byte{'b'}, End: []byte{'c'}}, - ingestData: &Engine{}, retryCount: MaxWriteAndIngestRetryTimes - 1, ->>>>>>> 448d56910cd (lightning: fix forget to set lastRetryableErr when ingest RPC fail (#56345)):pkg/lightning/backend/local/local_test.go injected: getSuccessInjectedBehaviour(), }, }, @@ -1951,15 +1939,9 @@ func TestDoImport(t *testing.T) { {"c", "d"}: { jobs: []*regionJob{ { -<<<<<<< HEAD:br/pkg/lightning/backend/local/local_test.go keyRange: Range{start: []byte{'c'}, end: []byte{'d'}}, engine: &Engine{}, - retryCount: maxWriteAndIngestRetryTimes - 2, -======= - keyRange: common.Range{Start: []byte{'c'}, End: []byte{'d'}}, - ingestData: &Engine{}, retryCount: MaxWriteAndIngestRetryTimes - 2, ->>>>>>> 448d56910cd (lightning: fix forget to set lastRetryableErr when ingest RPC fail (#56345)):pkg/lightning/backend/local/local_test.go injected: []injectedBehaviour{ { write: injectedWriteBehaviour{ @@ -2010,39 +1992,13 @@ func TestRegionJobResetRetryCounter(t *testing.T) { keyRange: Range{start: []byte{'c'}, end: []byte{'c', '2'}}, engine: &Engine{}, injected: getNeedRescanWhenIngestBehaviour(), -<<<<<<< HEAD:br/pkg/lightning/backend/local/local_test.go - retryCount: maxWriteAndIngestRetryTimes, -======= retryCount: MaxWriteAndIngestRetryTimes, - region: &split.RegionInfo{ - Region: &metapb.Region{ - Peers: []*metapb.Peer{ - {Id: 1, StoreId: 1}, - {Id: 2, StoreId: 2}, - {Id: 3, StoreId: 3}, - }, - }, - }, ->>>>>>> 448d56910cd (lightning: fix forget to set lastRetryableErr when ingest RPC fail (#56345)):pkg/lightning/backend/local/local_test.go }, { keyRange: Range{start: []byte{'c', '2'}, end: []byte{'d'}}, engine: &Engine{}, injected: getSuccessInjectedBehaviour(), -<<<<<<< HEAD:br/pkg/lightning/backend/local/local_test.go - retryCount: maxWriteAndIngestRetryTimes, -======= retryCount: MaxWriteAndIngestRetryTimes, - region: &split.RegionInfo{ - Region: &metapb.Region{ - Peers: []*metapb.Peer{ - {Id: 4, StoreId: 4}, - {Id: 5, StoreId: 5}, - {Id: 6, StoreId: 6}, - }, - }, - }, ->>>>>>> 448d56910cd (lightning: fix forget to set lastRetryableErr when ingest RPC fail (#56345)):pkg/lightning/backend/local/local_test.go }, }, }, From d07f8d62080b75f69fc3b23a015e83d27fb4d3e5 Mon Sep 17 00:00:00 2001 From: lance6716 Date: Tue, 29 Oct 2024 14:08:24 +0800 Subject: [PATCH 3/5] fix conflict Signed-off-by: lance6716 --- .../addindextest/integration_test.go | 83 +------------------ 1 file changed, 2 insertions(+), 81 deletions(-) diff --git a/tests/realtikvtest/addindextest/integration_test.go b/tests/realtikvtest/addindextest/integration_test.go index 06a30e00ef65f..2eab9473231c7 100644 --- a/tests/realtikvtest/addindextest/integration_test.go +++ b/tests/realtikvtest/addindextest/integration_test.go @@ -565,85 +565,6 @@ func TestAddUniqueIndexDuplicatedError(t *testing.T) { tk.MustExec("INSERT INTO `b1cce552` (`f5d9aecb`, `d9337060`, `4c74082f`, `9215adc3`, `85ad5a07`, `8c60260f`, `8069da7b`, `91e218e1`) VALUES ('2031-12-22 06:44:52', 'duplicatevalue', 2028, NULL, 846, 'N6QD1=@ped@owVoJx', '9soPM2d6H', 'Tv%'), ('2031-12-22 06:44:52', 'duplicatevalue', 2028, NULL, 9052, '_HWaf#gD!bw', '9soPM2d6H', 'Tv%');") tk.MustGetErrCode("ALTER TABLE `b1cce552` ADD unique INDEX `65290727` (`4c74082f`, `d9337060`, `8069da7b`);", errno.ErrDupEntry) } -<<<<<<< HEAD:tests/realtikvtest/addindextest/integration_test.go -======= - -func TestFirstLitSlowStart(t *testing.T) { - store := realtikvtest.CreateMockStoreAndSetup(t) - tk := testkit.NewTestKit(t, store) - tk.MustExec("drop database if exists addindexlit;") - tk.MustExec("create database addindexlit;") - tk.MustExec("use addindexlit;") - tk.MustExec(`set global tidb_ddl_enable_fast_reorg=on;`) - tk.MustExec(`set global tidb_enable_dist_task=off;`) - - tk.MustExec("create table t(a int, b int);") - tk.MustExec("insert into t values (1, 1), (2, 2), (3, 3);") - tk.MustExec("create table t2(a int, b int);") - tk.MustExec("insert into t2 values (1, 1), (2, 2), (3, 3);") - - tk1 := testkit.NewTestKit(t, store) - tk1.MustExec("use addindexlit;") - - require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/pkg/ddl/ingest/beforeCreateLocalBackend", "1*return()")) - t.Cleanup(func() { - require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/pkg/ddl/ingest/beforeCreateLocalBackend")) - }) - require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/pkg/ddl/ownerResignAfterDispatchLoopCheck", "return()")) - t.Cleanup(func() { - require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/pkg/ddl/ownerResignAfterDispatchLoopCheck")) - }) - require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/pkg/lightning/backend/local/slowCreateFS", "return()")) - t.Cleanup(func() { - require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/pkg/lightning/backend/local/slowCreateFS")) - }) - - var wg sync.WaitGroup - wg.Add(2) - go func() { - defer wg.Done() - tk.MustExec("alter table t add unique index idx(a);") - }() - go func() { - defer wg.Done() - tk1.MustExec("alter table t2 add unique index idx(a);") - }() - wg.Wait() -} - -func TestConcFastReorg(t *testing.T) { - store := realtikvtest.CreateMockStoreAndSetup(t) - tk := testkit.NewTestKit(t, store) - tk.MustExec("drop database if exists addindexlit;") - tk.MustExec("create database addindexlit;") - tk.MustExec("use addindexlit;") - tk.MustExec(`set global tidb_ddl_enable_fast_reorg=on;`) - - tblNum := 10 - for i := 0; i < tblNum; i++ { - tk.MustExec(fmt.Sprintf("create table t%d(a int);", i)) - } - - var wg sync.WaitGroup - wg.Add(tblNum) - for i := 0; i < tblNum; i++ { - i := i - go func() { - defer wg.Done() - tk2 := testkit.NewTestKit(t, store) - tk2.MustExec("use addindexlit;") - tk2.MustExec(fmt.Sprintf("insert into t%d values (1), (2), (3);", i)) - - if i%2 == 0 { - tk2.MustExec(fmt.Sprintf("alter table t%d add index idx(a);", i)) - } else { - tk2.MustExec(fmt.Sprintf("alter table t%d add unique index idx(a);", i)) - } - }() - } - - wg.Wait() -} func TestIssue55808(t *testing.T) { store := realtikvtest.CreateMockStoreAndSetup(t) @@ -665,8 +586,8 @@ func TestIssue55808(t *testing.T) { for i := 0; i < 4; i++ { tk.MustExec(fmt.Sprintf("insert into t values (%d, %d);", i*10000, i*10000)) } - testfailpoint.Enable(t, "github.com/pingcap/tidb/pkg/lightning/backend/local/doIngestFailed", "return()") + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/pkg/lightning/backend/local/doIngestFailed", "return()")) err := tk.ExecToErr("alter table t add index idx(a);") require.ErrorContains(t, err, "injected error") + require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/pkg/lightning/backend/local/doIngestFailed")) } ->>>>>>> 448d56910cd (lightning: fix forget to set lastRetryableErr when ingest RPC fail (#56345)):tests/realtikvtest/addindextest3/ingest_test.go From e2809bf7c2bb541b0682bee1aa387319c1965928 Mon Sep 17 00:00:00 2001 From: lance6716 Date: Tue, 29 Oct 2024 14:10:01 +0800 Subject: [PATCH 4/5] fix failpoint path Signed-off-by: lance6716 --- tests/realtikvtest/addindextest/integration_test.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/realtikvtest/addindextest/integration_test.go b/tests/realtikvtest/addindextest/integration_test.go index 2eab9473231c7..d28dcd61d0043 100644 --- a/tests/realtikvtest/addindextest/integration_test.go +++ b/tests/realtikvtest/addindextest/integration_test.go @@ -586,8 +586,8 @@ func TestIssue55808(t *testing.T) { for i := 0; i < 4; i++ { tk.MustExec(fmt.Sprintf("insert into t values (%d, %d);", i*10000, i*10000)) } - require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/pkg/lightning/backend/local/doIngestFailed", "return()")) + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/br/pkg/lightning/backend/local/doIngestFailed", "return()")) err := tk.ExecToErr("alter table t add index idx(a);") require.ErrorContains(t, err, "injected error") - require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/pkg/lightning/backend/local/doIngestFailed")) + require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/br/pkg/lightning/backend/local/doIngestFailed")) } From 53a8c69fd4426c4957071c62c75cf0c911212834 Mon Sep 17 00:00:00 2001 From: lance6716 Date: Tue, 29 Oct 2024 14:25:24 +0800 Subject: [PATCH 5/5] fix bazel Signed-off-by: lance6716 --- br/pkg/lightning/backend/local/BUILD.bazel | 1 + 1 file changed, 1 insertion(+) diff --git a/br/pkg/lightning/backend/local/BUILD.bazel b/br/pkg/lightning/backend/local/BUILD.bazel index af57eeb6f7003..ccc91c1163708 100644 --- a/br/pkg/lightning/backend/local/BUILD.bazel +++ b/br/pkg/lightning/backend/local/BUILD.bazel @@ -55,6 +55,7 @@ go_library( "//util/codec", "//util/engine", "//util/hack", + "//util/intest", "//util/mathutil", "//util/ranger", "@com_github_cockroachdb_pebble//:pebble",