-
Notifications
You must be signed in to change notification settings - Fork 5.9k
/
backfilling.go
642 lines (566 loc) · 22 KB
/
backfilling.go
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
// Copyright 2020 PingCAP, Inc.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// See the License for the specific language governing permissions and
// limitations under the License.
package ddl
import (
"context"
"math"
"strconv"
"sync/atomic"
"time"
"github.com/pingcap/errors"
"github.com/pingcap/failpoint"
"github.com/pingcap/parser/model"
"github.com/pingcap/parser/terror"
ddlutil "github.com/pingcap/tidb/ddl/util"
"github.com/pingcap/tidb/expression"
"github.com/pingcap/tidb/kv"
"github.com/pingcap/tidb/metrics"
"github.com/pingcap/tidb/sessionctx"
"github.com/pingcap/tidb/sessionctx/variable"
"github.com/pingcap/tidb/store/tikv"
"github.com/pingcap/tidb/table"
"github.com/pingcap/tidb/tablecodec"
"github.com/pingcap/tidb/util"
"github.com/pingcap/tidb/util/logutil"
decoder "github.com/pingcap/tidb/util/rowDecoder"
"go.uber.org/zap"
)
type backfillWorkerType byte
const (
typeAddIndexWorker backfillWorkerType = 0
typeUpdateColumnWorker backfillWorkerType = 1
typeCleanUpIndexWorker backfillWorkerType = 2
)
func (bWT backfillWorkerType) String() string {
switch bWT {
case typeAddIndexWorker:
return "add index"
case typeUpdateColumnWorker:
return "update column"
case typeCleanUpIndexWorker:
return "clean up index"
default:
return "unknown"
}
}
type backfiller interface {
BackfillDataInTxn(handleRange reorgBackfillTask) (taskCtx backfillTaskContext, errInTxn error)
AddMetricInfo(float64)
}
type backfillResult struct {
addedCount int
scanCount int
nextHandle kv.Handle
err error
}
// backfillTaskContext is the context of the batch adding indices or updating column values.
// After finishing the batch adding indices or updating column values, result in backfillTaskContext will be merged into backfillResult.
type backfillTaskContext struct {
nextHandle kv.Handle
done bool
addedCount int
scanCount int
warnings map[errors.ErrorID]*terror.Error
warningsCount map[errors.ErrorID]int64
}
type backfillWorker struct {
id int
ddlWorker *worker
batchCnt int
sessCtx sessionctx.Context
taskCh chan *reorgBackfillTask
resultCh chan *backfillResult
table table.Table
closed bool
priority int
}
func newBackfillWorker(sessCtx sessionctx.Context, worker *worker, id int, t table.PhysicalTable) *backfillWorker {
return &backfillWorker{
id: id,
table: t,
ddlWorker: worker,
batchCnt: int(variable.GetDDLReorgBatchSize()),
sessCtx: sessCtx,
taskCh: make(chan *reorgBackfillTask, 1),
resultCh: make(chan *backfillResult, 1),
priority: kv.PriorityLow,
}
}
func (w *backfillWorker) Close() {
if !w.closed {
w.closed = true
close(w.taskCh)
}
}
func closeBackfillWorkers(workers []*backfillWorker) {
for _, worker := range workers {
worker.Close()
}
}
type reorgBackfillTask struct {
physicalTableID int64
startHandle kv.Handle
endHandle kv.Handle
// endIncluded indicates whether the range include the endHandle.
// When the last handle is math.MaxInt64, set endIncluded to true to
// tell worker backfilling index of endHandle.
endIncluded bool
}
func (r *reorgBackfillTask) String() string {
rightParenthesis := ")"
if r.endIncluded {
rightParenthesis = "]"
}
return "physicalTableID_" + strconv.FormatInt(r.physicalTableID, 10) + "_" + "[" + r.startHandle.String() + "," + r.endHandle.String() + rightParenthesis
}
func logSlowOperations(elapsed time.Duration, slowMsg string, threshold uint32) {
if threshold == 0 {
threshold = atomic.LoadUint32(&variable.DDLSlowOprThreshold)
}
if elapsed >= time.Duration(threshold)*time.Millisecond {
logutil.BgLogger().Info("[ddl] slow operations", zap.Duration("takeTimes", elapsed), zap.String("msg", slowMsg))
}
}
// mergeBackfillCtxToResult merge partial result in taskCtx into result.
func mergeBackfillCtxToResult(taskCtx *backfillTaskContext, result *backfillResult) {
result.nextHandle = taskCtx.nextHandle
result.addedCount += taskCtx.addedCount
result.scanCount += taskCtx.scanCount
}
func mergeWarningsAndWarningsCount(partWarnings, totalWarnings map[errors.ErrorID]*terror.Error, partWarningsCount, totalWarningsCount map[errors.ErrorID]int64) (map[errors.ErrorID]*terror.Error, map[errors.ErrorID]int64) {
for _, warn := range partWarnings {
if _, ok := totalWarningsCount[warn.ID()]; ok {
totalWarningsCount[warn.ID()] += partWarningsCount[warn.ID()]
} else {
totalWarningsCount[warn.ID()] = partWarningsCount[warn.ID()]
totalWarnings[warn.ID()] = warn
}
}
return totalWarnings, totalWarningsCount
}
// handleBackfillTask backfills range [task.startHandle, task.endHandle) handle's index to table.
func (w *backfillWorker) handleBackfillTask(d *ddlCtx, task *reorgBackfillTask, bf backfiller) *backfillResult {
handleRange := *task
result := &backfillResult{
err: nil,
addedCount: 0,
nextHandle: handleRange.startHandle,
}
lastLogCount := 0
lastLogTime := time.Now()
startTime := lastLogTime
for {
// Give job chance to be canceled, if we not check it here,
// if there is panic in bf.BackfillDataInTxn we will never cancel the job.
// Because reorgRecordTask may run a long time,
// we should check whether this ddl job is still runnable.
err := w.ddlWorker.isReorgRunnable(d)
if err != nil {
result.err = err
return result
}
taskCtx, err := bf.BackfillDataInTxn(handleRange)
if err != nil {
result.err = err
return result
}
bf.AddMetricInfo(float64(taskCtx.addedCount))
mergeBackfillCtxToResult(&taskCtx, result)
// Although `handleRange` is for data in one region, but back fill worker still split it into many
// small reorg batch size slices and reorg them in many different kv txn.
// If a task failed, it may contained some committed small kv txn which has already finished the
// small range reorganization.
// In the next round of reorganization, the target handle range may overlap with last committed
// small ranges. This will cause the `redo` action in reorganization.
// So for added count and warnings collection, it is recommended to collect the statistics in every
// successfully committed small ranges rather than fetching it in the total result.
w.ddlWorker.reorgCtx.increaseRowCount(int64(taskCtx.addedCount))
w.ddlWorker.reorgCtx.mergeWarnings(taskCtx.warnings, taskCtx.warningsCount)
if num := result.scanCount - lastLogCount; num >= 30000 {
lastLogCount = result.scanCount
logutil.BgLogger().Info("[ddl] backfill worker back fill index", zap.Int("workerID", w.id), zap.Int("addedCount", result.addedCount),
zap.Int("scanCount", result.scanCount), zap.String("nextHandle", toString(taskCtx.nextHandle)), zap.Float64("speed(rows/s)", float64(num)/time.Since(lastLogTime).Seconds()))
lastLogTime = time.Now()
}
handleRange.startHandle = taskCtx.nextHandle
if taskCtx.done {
break
}
}
logutil.BgLogger().Info("[ddl] backfill worker finish task", zap.Int("workerID", w.id),
zap.String("task", task.String()), zap.Int("addedCount", result.addedCount),
zap.Int("scanCount", result.scanCount), zap.String("nextHandle", toString(result.nextHandle)),
zap.String("takeTime", time.Since(startTime).String()))
return result
}
func (w *backfillWorker) run(d *ddlCtx, bf backfiller) {
logutil.BgLogger().Info("[ddl] backfill worker start", zap.Int("workerID", w.id))
defer func() {
w.resultCh <- &backfillResult{err: errReorgPanic}
}()
defer util.Recover(metrics.LabelDDL, "backfillWorker.run", nil, false)
for {
task, more := <-w.taskCh
if !more {
break
}
logutil.BgLogger().Debug("[ddl] backfill worker got task", zap.Int("workerID", w.id), zap.String("task", task.String()))
failpoint.Inject("mockBackfillRunErr", func() {
if w.id == 0 {
result := &backfillResult{addedCount: 0, nextHandle: nil, err: errors.Errorf("mock backfill error")}
w.resultCh <- result
failpoint.Continue()
}
})
// Dynamic change batch size.
w.batchCnt = int(variable.GetDDLReorgBatchSize())
result := w.handleBackfillTask(d, task, bf)
w.resultCh <- result
}
logutil.BgLogger().Info("[ddl] backfill worker exit", zap.Int("workerID", w.id))
}
// splitTableRanges uses PD region's key ranges to split the backfilling table key range space,
// to speed up backfilling data in table with disperse handle.
// The `t` should be a non-partitioned table or a partition.
func splitTableRanges(t table.PhysicalTable, store kv.Storage, startHandle, endHandle kv.Handle) ([]kv.KeyRange, error) {
startRecordKey := t.RecordKey(startHandle)
endRecordKey := t.RecordKey(endHandle)
logutil.BgLogger().Info("[ddl] split table range from PD", zap.Int64("physicalTableID", t.GetPhysicalID()),
zap.String("startHandle", toString(startHandle)), zap.String("endHandle", toString(endHandle)))
kvRange := kv.KeyRange{StartKey: startRecordKey, EndKey: endRecordKey}
s, ok := store.(tikv.Storage)
if !ok {
// Only support split ranges in tikv.Storage now.
return []kv.KeyRange{kvRange}, nil
}
maxSleep := 10000 // ms
bo := tikv.NewBackofferWithVars(context.Background(), maxSleep, nil)
ranges, err := tikv.SplitRegionRanges(bo, s.GetRegionCache(), []kv.KeyRange{kvRange})
if err != nil {
return nil, errors.Trace(err)
}
if len(ranges) == 0 {
return nil, errors.Trace(errInvalidSplitRegionRanges)
}
return ranges, nil
}
func (w *worker) waitTaskResults(workers []*backfillWorker, taskCnt int, totalAddedCount *int64, startHandle kv.Handle) (kv.Handle, int64, error) {
var (
addedCount int64
nextHandle = startHandle
firstErr error
)
for i := 0; i < taskCnt; i++ {
worker := workers[i]
result := <-worker.resultCh
if firstErr == nil && result.err != nil {
firstErr = result.err
// We should wait all working workers exits, any way.
continue
}
if result.err != nil {
logutil.BgLogger().Warn("[ddl] backfill worker failed", zap.Int("workerID", worker.id),
zap.Error(result.err))
}
if firstErr == nil {
*totalAddedCount += int64(result.addedCount)
addedCount += int64(result.addedCount)
nextHandle = result.nextHandle
}
}
return nextHandle, addedCount, errors.Trace(firstErr)
}
// handleReorgTasks sends tasks to workers, and waits for all the running workers to return results,
// there are taskCnt running workers.
func (w *worker) handleReorgTasks(reorgInfo *reorgInfo, totalAddedCount *int64, workers []*backfillWorker, batchTasks []*reorgBackfillTask) error {
for i, task := range batchTasks {
workers[i].taskCh <- task
}
startHandle := batchTasks[0].startHandle
taskCnt := len(batchTasks)
startTime := time.Now()
nextHandle, taskAddedCount, err := w.waitTaskResults(workers, taskCnt, totalAddedCount, startHandle)
elapsedTime := time.Since(startTime)
if err == nil {
err = w.isReorgRunnable(reorgInfo.d)
}
if err != nil {
// Update the reorg handle that has been processed.
err1 := reorgInfo.UpdateReorgMeta(nextHandle)
metrics.BatchAddIdxHistogram.WithLabelValues(metrics.LblError).Observe(elapsedTime.Seconds())
logutil.BgLogger().Warn("[ddl] backfill worker handle batch tasks failed",
zap.ByteString("elementType", reorgInfo.currElement.TypeKey), zap.Int64("elementID", reorgInfo.currElement.ID),
zap.Int64("totalAddedCount", *totalAddedCount), zap.String("startHandle", toString(startHandle)),
zap.String("nextHandle", toString(nextHandle)), zap.Int64("batchAddedCount", taskAddedCount),
zap.String("taskFailedError", err.Error()), zap.String("takeTime", elapsedTime.String()),
zap.NamedError("updateHandleError", err1))
return errors.Trace(err)
}
// nextHandle will be updated periodically in runReorgJob, so no need to update it here.
w.reorgCtx.setNextHandle(nextHandle)
metrics.BatchAddIdxHistogram.WithLabelValues(metrics.LblOK).Observe(elapsedTime.Seconds())
logutil.BgLogger().Info("[ddl] backfill workers successfully processed batch",
zap.ByteString("elementType", reorgInfo.currElement.TypeKey), zap.Int64("elementID", reorgInfo.currElement.ID),
zap.Int64("totalAddedCount", *totalAddedCount), zap.String("startHandle", toString(startHandle)),
zap.String("nextHandle", toString(nextHandle)), zap.Int64("batchAddedCount", taskAddedCount), zap.String("takeTime", elapsedTime.String()))
return nil
}
func decodeHandleRange(keyRange kv.KeyRange) (kv.Handle, kv.Handle, error) {
startHandle, err := tablecodec.DecodeRowKey(keyRange.StartKey)
if err != nil {
return nil, nil, errors.Trace(err)
}
endHandle, err := tablecodec.DecodeRowKey(keyRange.EndKey)
if err != nil {
return nil, nil, errors.Trace(err)
}
return startHandle, endHandle, nil
}
// sendRangeTaskToWorkers sends tasks to workers, and returns remaining kvRanges that is not handled.
func (w *worker) sendRangeTaskToWorkers(workers []*backfillWorker, reorgInfo *reorgInfo,
totalAddedCount *int64, kvRanges []kv.KeyRange, globalEndHandle kv.Handle) ([]kv.KeyRange, error) {
batchTasks := make([]*reorgBackfillTask, 0, len(workers))
physicalTableID := reorgInfo.PhysicalTableID
// Build reorg tasks.
for _, keyRange := range kvRanges {
startHandle, endHandle, err := decodeHandleRange(keyRange)
if err != nil {
return nil, errors.Trace(err)
}
endIncluded := false
if endHandle.Equal(globalEndHandle) {
endIncluded = true
}
task := &reorgBackfillTask{physicalTableID, startHandle, endHandle, endIncluded}
batchTasks = append(batchTasks, task)
if len(batchTasks) >= len(workers) {
break
}
}
if len(batchTasks) == 0 {
return nil, nil
}
// Wait tasks finish.
err := w.handleReorgTasks(reorgInfo, totalAddedCount, workers, batchTasks)
if err != nil {
return nil, errors.Trace(err)
}
if len(batchTasks) < len(kvRanges) {
// There are kvRanges not handled.
remains := kvRanges[len(batchTasks):]
return remains, nil
}
return nil, nil
}
var (
// TestCheckWorkerNumCh use for test adjust backfill worker.
TestCheckWorkerNumCh = make(chan struct{})
// TestCheckWorkerNumber use for test adjust backfill worker.
TestCheckWorkerNumber = int32(16)
// TestCheckReorgTimeout is used to mock timeout when reorg data.
TestCheckReorgTimeout = int32(0)
)
func loadDDLReorgVars(w *worker) error {
// Get sessionctx from context resource pool.
var ctx sessionctx.Context
ctx, err := w.sessPool.get()
if err != nil {
return errors.Trace(err)
}
defer w.sessPool.put(ctx)
return ddlutil.LoadDDLReorgVars(ctx)
}
func makeupDecodeColMap(sessCtx sessionctx.Context, t table.Table) (map[int64]decoder.Column, error) {
dbName := model.NewCIStr(sessCtx.GetSessionVars().CurrentDB)
writableColInfos := make([]*model.ColumnInfo, 0, len(t.WritableCols()))
for _, col := range t.WritableCols() {
writableColInfos = append(writableColInfos, col.ColumnInfo)
}
exprCols, _, err := expression.ColumnInfos2ColumnsAndNames(sessCtx, dbName, t.Meta().Name, writableColInfos, t.Meta())
if err != nil {
return nil, err
}
mockSchema := expression.NewSchema(exprCols...)
decodeColMap := decoder.BuildFullDecodeColMap(t.WritableCols(), mockSchema)
return decodeColMap, nil
}
// writePhysicalTableRecord handles the "add index" or "modify/change column" reorganization state for a non-partitioned table or a partition.
// For a partitioned table, it should be handled partition by partition.
//
// How to "add index" or "update column value" in reorganization state?
// Concurrently process the @@tidb_ddl_reorg_worker_cnt tasks. Each task deals with a handle range of the index/row record.
// The handle range is split from PD regions now. Each worker deal with a region table key range one time.
// Each handle range by estimation, concurrent processing needs to perform after the handle range has been acquired.
// The operation flow is as follows:
// 1. Open numbers of defaultWorkers goroutines.
// 2. Split table key range from PD regions.
// 3. Send tasks to running workers by workers's task channel. Each task deals with a region key ranges.
// 4. Wait all these running tasks finished, then continue to step 3, until all tasks is done.
// The above operations are completed in a transaction.
// Finally, update the concurrent processing of the total number of rows, and store the completed handle value.
func (w *worker) writePhysicalTableRecord(t table.PhysicalTable, bfWorkerType backfillWorkerType, indexInfo *model.IndexInfo, oldColInfo, colInfo *model.ColumnInfo, reorgInfo *reorgInfo) error {
job := reorgInfo.Job
totalAddedCount := job.GetRowCount()
startHandle, endHandle := reorgInfo.StartHandle, reorgInfo.EndHandle
sessCtx := newContext(reorgInfo.d.store)
decodeColMap, err := makeupDecodeColMap(sessCtx, t)
if err != nil {
return errors.Trace(err)
}
if err := w.isReorgRunnable(reorgInfo.d); err != nil {
return errors.Trace(err)
}
if startHandle == nil && endHandle == nil {
return nil
}
failpoint.Inject("MockCaseWhenParseFailure", func(val failpoint.Value) {
if val.(bool) {
failpoint.Return(errors.New("job.ErrCount:" + strconv.Itoa(int(job.ErrorCount)) + ", mock unknown type: ast.whenClause."))
}
})
// variable.ddlReorgWorkerCounter can be modified by system variable "tidb_ddl_reorg_worker_cnt".
workerCnt := variable.GetDDLReorgWorkerCounter()
backfillWorkers := make([]*backfillWorker, 0, workerCnt)
defer func() {
closeBackfillWorkers(backfillWorkers)
}()
for {
kvRanges, err := splitTableRanges(t, reorgInfo.d.store, startHandle, endHandle)
if err != nil {
return errors.Trace(err)
}
// For dynamic adjust backfill worker number.
if err := loadDDLReorgVars(w); err != nil {
logutil.BgLogger().Error("[ddl] load DDL reorganization variable failed", zap.Error(err))
}
workerCnt = variable.GetDDLReorgWorkerCounter()
// If only have 1 range, we can only start 1 worker.
if len(kvRanges) < int(workerCnt) {
workerCnt = int32(len(kvRanges))
}
// Enlarge the worker size.
for i := len(backfillWorkers); i < int(workerCnt); i++ {
sessCtx := newContext(reorgInfo.d.store)
sessCtx.GetSessionVars().StmtCtx.IsDDLJobInQueue = true
switch bfWorkerType {
case typeAddIndexWorker:
idxWorker := newAddIndexWorker(sessCtx, w, i, t, indexInfo, decodeColMap, reorgInfo.ReorgMeta.SQLMode)
idxWorker.priority = job.Priority
backfillWorkers = append(backfillWorkers, idxWorker.backfillWorker)
go idxWorker.backfillWorker.run(reorgInfo.d, idxWorker)
case typeUpdateColumnWorker:
updateWorker := newUpdateColumnWorker(sessCtx, w, i, t, oldColInfo, colInfo, decodeColMap, reorgInfo.ReorgMeta.SQLMode)
updateWorker.priority = job.Priority
backfillWorkers = append(backfillWorkers, updateWorker.backfillWorker)
go updateWorker.backfillWorker.run(reorgInfo.d, updateWorker)
case typeCleanUpIndexWorker:
idxWorker := newCleanUpIndexWorker(sessCtx, w, i, t, decodeColMap, reorgInfo.ReorgMeta.SQLMode)
idxWorker.priority = job.Priority
backfillWorkers = append(backfillWorkers, idxWorker.backfillWorker)
go idxWorker.backfillWorker.run(reorgInfo.d, idxWorker)
default:
return errors.New("unknow backfill type")
}
}
// Shrink the worker size.
if len(backfillWorkers) > int(workerCnt) {
workers := backfillWorkers[workerCnt:]
backfillWorkers = backfillWorkers[:workerCnt]
closeBackfillWorkers(workers)
}
failpoint.Inject("checkBackfillWorkerNum", func(val failpoint.Value) {
if val.(bool) {
num := int(atomic.LoadInt32(&TestCheckWorkerNumber))
if num != 0 {
if num > len(kvRanges) {
if len(backfillWorkers) != len(kvRanges) {
failpoint.Return(errors.Errorf("check backfill worker num error, len kv ranges is: %v, check backfill worker num is: %v, actual record num is: %v", len(kvRanges), num, len(backfillWorkers)))
}
} else if num != len(backfillWorkers) {
failpoint.Return(errors.Errorf("check backfill worker num error, len kv ranges is: %v, check backfill worker num is: %v, actual record num is: %v", len(kvRanges), num, len(backfillWorkers)))
}
TestCheckWorkerNumCh <- struct{}{}
}
}
})
logutil.BgLogger().Info("[ddl] start backfill workers to reorg record", zap.Int("workerCnt", len(backfillWorkers)),
zap.Int("regionCnt", len(kvRanges)), zap.String("startHandle", toString(startHandle)), zap.String("endHandle", toString(endHandle)))
remains, err := w.sendRangeTaskToWorkers(backfillWorkers, reorgInfo, &totalAddedCount, kvRanges, endHandle)
if err != nil {
return errors.Trace(err)
}
if len(remains) == 0 {
break
}
startHandle, _, err = decodeHandleRange(remains[0])
if err != nil {
return errors.Trace(err)
}
}
return nil
}
// recordIterFunc is used for low-level record iteration.
type recordIterFunc func(h kv.Handle, rowKey kv.Key, rawRecord []byte) (more bool, err error)
func iterateSnapshotRows(store kv.Storage, priority int, t table.Table, version uint64, startHandle kv.Handle, endHandle kv.Handle, endIncluded bool, fn recordIterFunc) error {
var firstKey kv.Key
if startHandle == nil {
firstKey = t.RecordPrefix()
} else {
firstKey = t.RecordKey(startHandle)
}
var upperBound kv.Key
if endHandle == nil {
upperBound = t.RecordPrefix().PrefixNext()
} else {
if endIncluded {
if endHandle.IsInt() && endHandle.IntValue() == math.MaxInt64 {
upperBound = t.RecordKey(endHandle).PrefixNext()
} else {
upperBound = t.RecordKey(endHandle.Next())
}
} else {
upperBound = t.RecordKey(endHandle)
}
}
ver := kv.Version{Ver: version}
snap := store.GetSnapshot(ver)
snap.SetOption(kv.Priority, priority)
it, err := snap.Iter(firstKey, upperBound)
if err != nil {
return errors.Trace(err)
}
defer it.Close()
for it.Valid() {
if !it.Key().HasPrefix(t.RecordPrefix()) {
break
}
var handle kv.Handle
handle, err = tablecodec.DecodeRowKey(it.Key())
if err != nil {
return errors.Trace(err)
}
rk := t.RecordKey(handle)
more, err := fn(handle, rk, it.Value())
if !more || err != nil {
return errors.Trace(err)
}
err = kv.NextUntil(it, util.RowKeyPrefixFilter(rk))
if err != nil {
if kv.ErrNotExist.Equal(err) {
break
}
return errors.Trace(err)
}
}
return nil
}