diff --git a/br/pkg/lightning/backend/local/local.go b/br/pkg/lightning/backend/local/local.go index 34a1117b55fc9..746d2af819a1a 100644 --- a/br/pkg/lightning/backend/local/local.go +++ b/br/pkg/lightning/backend/local/local.go @@ -1086,7 +1086,7 @@ func (local *Backend) prepareAndSendJob( failpoint.Break() }) - err = local.SplitAndScatterRegionInBatches(ctx, initialSplitRanges, engine.tableInfo, needSplit, regionSplitSize, maxBatchSplitRanges) + err = local.SplitAndScatterRegionInBatches(ctx, initialSplitRanges, needSplit, maxBatchSplitRanges) if err == nil || common.IsContextCanceledError(err) { break } diff --git a/br/pkg/lightning/backend/local/localhelper.go b/br/pkg/lightning/backend/local/localhelper.go index ae2384a5fd8b3..72fbab2c65ede 100644 --- a/br/pkg/lightning/backend/local/localhelper.go +++ b/br/pkg/lightning/backend/local/localhelper.go @@ -113,9 +113,7 @@ func (g *TableRegionSizeGetterImpl) GetTableRegionSize(ctx context.Context, tabl func (local *Backend) SplitAndScatterRegionInBatches( ctx context.Context, ranges []Range, - tableInfo *checkpoints.TidbTableInfo, needSplit bool, - regionSplitSize int64, batchCnt int, ) error { for i := 0; i < len(ranges); i += batchCnt { @@ -123,7 +121,7 @@ func (local *Backend) SplitAndScatterRegionInBatches( if len(batch) > batchCnt { batch = batch[:batchCnt] } - if err := local.SplitAndScatterRegionByRanges(ctx, batch, tableInfo, needSplit, regionSplitSize); err != nil { + if err := local.SplitAndScatterRegionByRanges(ctx, batch, needSplit); err != nil { return errors.Trace(err) } } @@ -137,9 +135,7 @@ func (local *Backend) SplitAndScatterRegionInBatches( func (local *Backend) SplitAndScatterRegionByRanges( ctx context.Context, ranges []Range, - tableInfo *checkpoints.TidbTableInfo, needSplit bool, - regionSplitSize int64, ) (err error) { if len(ranges) == 0 { return nil @@ -160,7 +156,6 @@ func (local *Backend) SplitAndScatterRegionByRanges( scatterRegions := make([]*split.RegionInfo, 0) var retryKeys [][]byte waitTime := splitRegionBaseBackOffTime - skippedKeys := 0 for i := 0; i < splitRetryTimes; i++ { log.FromContext(ctx).Info("split and scatter region", logutil.Key("minKey", minKey), @@ -222,16 +217,6 @@ func (local *Backend) SplitAndScatterRegionByRanges( return nil } - var tableRegionStats map[uint64]int64 - if tableInfo != nil { - tableRegionStats, err = local.regionSizeGetter.GetTableRegionSize(ctx, tableInfo.ID) - if err != nil { - log.FromContext(ctx).Warn("fetch table region size statistics failed", - zap.String("table", tableInfo.Name), zap.Error(err)) - tableRegionStats, err = make(map[uint64]int64), nil - } - } - regionMap := make(map[uint64]*split.RegionInfo) for _, region := range regions { regionMap[region.Region.GetId()] = region @@ -342,15 +327,6 @@ func (local *Backend) SplitAndScatterRegionByRanges( } sendLoop: for regionID, keys := range splitKeyMap { - // if region not in tableRegionStats, that means this region is newly split, so - // we can skip split it again. - regionSize, ok := tableRegionStats[regionID] - if !ok { - log.FromContext(ctx).Warn("region stats not found", zap.Uint64("region", regionID)) - } - if len(keys) == 1 && regionSize < regionSplitSize { - skippedKeys++ - } select { case ch <- &splitInfo{region: regionMap[regionID], keys: keys}: case <-ctx.Done(): @@ -386,11 +362,9 @@ func (local *Backend) SplitAndScatterRegionByRanges( scatterCount, err := local.waitForScatterRegions(ctx, scatterRegions) if scatterCount == len(scatterRegions) { log.FromContext(ctx).Info("waiting for scattering regions done", - zap.Int("skipped_keys", skippedKeys), zap.Int("regions", len(scatterRegions)), zap.Duration("take", time.Since(startTime))) } else { log.FromContext(ctx).Info("waiting for scattering regions timeout", - zap.Int("skipped_keys", skippedKeys), zap.Int("scatterCount", scatterCount), zap.Int("regions", len(scatterRegions)), zap.Duration("take", time.Since(startTime)), diff --git a/br/pkg/lightning/backend/local/localhelper_test.go b/br/pkg/lightning/backend/local/localhelper_test.go index d677e9c1dc7ba..206822be4a50d 100644 --- a/br/pkg/lightning/backend/local/localhelper_test.go +++ b/br/pkg/lightning/backend/local/localhelper_test.go @@ -477,7 +477,7 @@ func doTestBatchSplitRegionByRanges(ctx context.Context, t *testing.T, hook clie start = end } - err = local.SplitAndScatterRegionByRanges(ctx, ranges, nil, true, 1000) + err = local.SplitAndScatterRegionByRanges(ctx, ranges, true) if len(errPat) == 0 { require.NoError(t, err) } else { @@ -569,7 +569,7 @@ func TestMissingScatter(t *testing.T) { start = end } - err = local.SplitAndScatterRegionByRanges(ctx, ranges, nil, true, 1000) + err = local.SplitAndScatterRegionByRanges(ctx, ranges, true) require.NoError(t, err) splitHook.check(t, client) @@ -732,7 +732,7 @@ func TestSplitAndScatterRegionInBatches(t *testing.T) { }) } - err := local.SplitAndScatterRegionInBatches(ctx, ranges, nil, true, 1000, 4) + err := local.SplitAndScatterRegionInBatches(ctx, ranges, true, 4) require.NoError(t, err) rangeStart := codec.EncodeBytes([]byte{}, []byte("a")) @@ -828,7 +828,7 @@ func doTestBatchSplitByRangesWithClusteredIndex(t *testing.T, hook clientHook) { start = e } - err := local.SplitAndScatterRegionByRanges(ctx, ranges, nil, true, 1000) + err := local.SplitAndScatterRegionByRanges(ctx, ranges, true) require.NoError(t, err) startKey := codec.EncodeBytes([]byte{}, rangeKeys[0])