Skip to content

Commit 3a55235

Browse files
authored
pkg/lightning : remove get_regions call in physical backend (#46202) (#46337)
close #45507
1 parent eacc493 commit 3a55235

File tree

3 files changed

+6
-32
lines changed

3 files changed

+6
-32
lines changed

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

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -1086,7 +1086,7 @@ func (local *Backend) prepareAndSendJob(
10861086
failpoint.Break()
10871087
})
10881088

1089-
err = local.SplitAndScatterRegionInBatches(ctx, initialSplitRanges, engine.tableInfo, needSplit, regionSplitSize, maxBatchSplitRanges)
1089+
err = local.SplitAndScatterRegionInBatches(ctx, initialSplitRanges, needSplit, maxBatchSplitRanges)
10901090
if err == nil || common.IsContextCanceledError(err) {
10911091
break
10921092
}

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

Lines changed: 1 addition & 27 deletions
Original file line numberDiff line numberDiff line change
@@ -113,17 +113,15 @@ func (g *TableRegionSizeGetterImpl) GetTableRegionSize(ctx context.Context, tabl
113113
func (local *Backend) SplitAndScatterRegionInBatches(
114114
ctx context.Context,
115115
ranges []Range,
116-
tableInfo *checkpoints.TidbTableInfo,
117116
needSplit bool,
118-
regionSplitSize int64,
119117
batchCnt int,
120118
) error {
121119
for i := 0; i < len(ranges); i += batchCnt {
122120
batch := ranges[i:]
123121
if len(batch) > batchCnt {
124122
batch = batch[:batchCnt]
125123
}
126-
if err := local.SplitAndScatterRegionByRanges(ctx, batch, tableInfo, needSplit, regionSplitSize); err != nil {
124+
if err := local.SplitAndScatterRegionByRanges(ctx, batch, needSplit); err != nil {
127125
return errors.Trace(err)
128126
}
129127
}
@@ -137,9 +135,7 @@ func (local *Backend) SplitAndScatterRegionInBatches(
137135
func (local *Backend) SplitAndScatterRegionByRanges(
138136
ctx context.Context,
139137
ranges []Range,
140-
tableInfo *checkpoints.TidbTableInfo,
141138
needSplit bool,
142-
regionSplitSize int64,
143139
) (err error) {
144140
if len(ranges) == 0 {
145141
return nil
@@ -160,7 +156,6 @@ func (local *Backend) SplitAndScatterRegionByRanges(
160156
scatterRegions := make([]*split.RegionInfo, 0)
161157
var retryKeys [][]byte
162158
waitTime := splitRegionBaseBackOffTime
163-
skippedKeys := 0
164159
for i := 0; i < splitRetryTimes; i++ {
165160
log.FromContext(ctx).Info("split and scatter region",
166161
logutil.Key("minKey", minKey),
@@ -222,16 +217,6 @@ func (local *Backend) SplitAndScatterRegionByRanges(
222217
return nil
223218
}
224219

225-
var tableRegionStats map[uint64]int64
226-
if tableInfo != nil {
227-
tableRegionStats, err = local.regionSizeGetter.GetTableRegionSize(ctx, tableInfo.ID)
228-
if err != nil {
229-
log.FromContext(ctx).Warn("fetch table region size statistics failed",
230-
zap.String("table", tableInfo.Name), zap.Error(err))
231-
tableRegionStats, err = make(map[uint64]int64), nil
232-
}
233-
}
234-
235220
regionMap := make(map[uint64]*split.RegionInfo)
236221
for _, region := range regions {
237222
regionMap[region.Region.GetId()] = region
@@ -342,15 +327,6 @@ func (local *Backend) SplitAndScatterRegionByRanges(
342327
}
343328
sendLoop:
344329
for regionID, keys := range splitKeyMap {
345-
// if region not in tableRegionStats, that means this region is newly split, so
346-
// we can skip split it again.
347-
regionSize, ok := tableRegionStats[regionID]
348-
if !ok {
349-
log.FromContext(ctx).Warn("region stats not found", zap.Uint64("region", regionID))
350-
}
351-
if len(keys) == 1 && regionSize < regionSplitSize {
352-
skippedKeys++
353-
}
354330
select {
355331
case ch <- &splitInfo{region: regionMap[regionID], keys: keys}:
356332
case <-ctx.Done():
@@ -386,11 +362,9 @@ func (local *Backend) SplitAndScatterRegionByRanges(
386362
scatterCount, err := local.waitForScatterRegions(ctx, scatterRegions)
387363
if scatterCount == len(scatterRegions) {
388364
log.FromContext(ctx).Info("waiting for scattering regions done",
389-
zap.Int("skipped_keys", skippedKeys),
390365
zap.Int("regions", len(scatterRegions)), zap.Duration("take", time.Since(startTime)))
391366
} else {
392367
log.FromContext(ctx).Info("waiting for scattering regions timeout",
393-
zap.Int("skipped_keys", skippedKeys),
394368
zap.Int("scatterCount", scatterCount),
395369
zap.Int("regions", len(scatterRegions)),
396370
zap.Duration("take", time.Since(startTime)),

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

Lines changed: 4 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -481,7 +481,7 @@ func doTestBatchSplitRegionByRanges(ctx context.Context, t *testing.T, hook clie
481481
start = end
482482
}
483483

484-
err = local.SplitAndScatterRegionByRanges(ctx, ranges, nil, true, 1000)
484+
err = local.SplitAndScatterRegionByRanges(ctx, ranges, true)
485485
if len(errPat) == 0 {
486486
require.NoError(t, err)
487487
} else {
@@ -573,7 +573,7 @@ func TestMissingScatter(t *testing.T) {
573573
start = end
574574
}
575575

576-
err = local.SplitAndScatterRegionByRanges(ctx, ranges, nil, true, 1000)
576+
err = local.SplitAndScatterRegionByRanges(ctx, ranges, true)
577577
require.NoError(t, err)
578578

579579
splitHook.check(t, client)
@@ -736,7 +736,7 @@ func TestSplitAndScatterRegionInBatches(t *testing.T) {
736736
})
737737
}
738738

739-
err := local.SplitAndScatterRegionInBatches(ctx, ranges, nil, true, 1000, 4)
739+
err := local.SplitAndScatterRegionInBatches(ctx, ranges, true, 4)
740740
require.NoError(t, err)
741741

742742
rangeStart := codec.EncodeBytes([]byte{}, []byte("a"))
@@ -832,7 +832,7 @@ func doTestBatchSplitByRangesWithClusteredIndex(t *testing.T, hook clientHook) {
832832
start = e
833833
}
834834

835-
err := local.SplitAndScatterRegionByRanges(ctx, ranges, nil, true, 1000)
835+
err := local.SplitAndScatterRegionByRanges(ctx, ranges, true)
836836
require.NoError(t, err)
837837

838838
startKey := codec.EncodeBytes([]byte{}, rangeKeys[0])

0 commit comments

Comments
 (0)