forked from pingcap/tidb
/
chunk_process.go
593 lines (550 loc) · 19.4 KB
/
chunk_process.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
// Copyright 2023 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,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
package importer
import (
"context"
"fmt"
"io"
"time"
"github.com/pingcap/errors"
"github.com/pingcap/failpoint"
"github.com/twotigers93/tidb/br/pkg/lightning/backend"
"github.com/twotigers93/tidb/br/pkg/lightning/backend/encode"
"github.com/twotigers93/tidb/br/pkg/lightning/backend/tidb"
"github.com/twotigers93/tidb/br/pkg/lightning/checkpoints"
"github.com/twotigers93/tidb/br/pkg/lightning/common"
"github.com/twotigers93/tidb/br/pkg/lightning/config"
"github.com/twotigers93/tidb/br/pkg/lightning/log"
"github.com/twotigers93/tidb/br/pkg/lightning/metric"
"github.com/twotigers93/tidb/br/pkg/lightning/mydump"
verify "github.com/twotigers93/tidb/br/pkg/lightning/verification"
"github.com/twotigers93/tidb/br/pkg/lightning/worker"
"github.com/twotigers93/tidb/br/pkg/storage"
"github.com/twotigers93/tidb/keyspace"
"github.com/twotigers93/tidb/parser/model"
"github.com/twotigers93/tidb/types"
"go.uber.org/zap"
)
// chunkProcessor process data chunk
// for local backend it encodes and writes KV to local disk
// for tidb backend it transforms data into sql and executes them.
type chunkProcessor struct {
parser mydump.Parser
index int
chunk *checkpoints.ChunkCheckpoint
}
func newChunkProcessor(
ctx context.Context,
index int,
cfg *config.Config,
chunk *checkpoints.ChunkCheckpoint,
ioWorkers *worker.Pool,
store storage.ExternalStorage,
tableInfo *checkpoints.TidbTableInfo,
) (*chunkProcessor, error) {
blockBufSize := int64(cfg.Mydumper.ReadBlockSize)
reader, err := mydump.OpenReader(ctx, &chunk.FileMeta, store)
if err != nil {
return nil, errors.Trace(err)
}
var parser mydump.Parser
switch chunk.FileMeta.Type {
case mydump.SourceTypeCSV:
hasHeader := cfg.Mydumper.CSV.Header && chunk.Chunk.Offset == 0
// Create a utf8mb4 convertor to encode and decode data with the charset of CSV files.
charsetConvertor, err := mydump.NewCharsetConvertor(cfg.Mydumper.DataCharacterSet, cfg.Mydumper.DataInvalidCharReplace)
if err != nil {
return nil, err
}
parser, err = mydump.NewCSVParser(ctx, &cfg.Mydumper.CSV, reader, blockBufSize, ioWorkers, hasHeader, charsetConvertor)
if err != nil {
return nil, errors.Trace(err)
}
case mydump.SourceTypeSQL:
parser = mydump.NewChunkParser(ctx, cfg.TiDB.SQLMode, reader, blockBufSize, ioWorkers)
case mydump.SourceTypeParquet:
parser, err = mydump.NewParquetParser(ctx, store, reader, chunk.FileMeta.Path)
if err != nil {
return nil, errors.Trace(err)
}
default:
panic(fmt.Sprintf("file '%s' with unknown source type '%s'", chunk.Key.Path, chunk.FileMeta.Type.String()))
}
if chunk.FileMeta.Compression == mydump.CompressionNone {
if err = parser.SetPos(chunk.Chunk.Offset, chunk.Chunk.PrevRowIDMax); err != nil {
return nil, errors.Trace(err)
}
} else {
if err = mydump.ReadUntil(parser, chunk.Chunk.Offset); err != nil {
return nil, errors.Trace(err)
}
parser.SetRowID(chunk.Chunk.PrevRowIDMax)
}
if len(chunk.ColumnPermutation) > 0 {
parser.SetColumns(getColumnNames(tableInfo.Core, chunk.ColumnPermutation))
}
return &chunkProcessor{
parser: parser,
index: index,
chunk: chunk,
}, nil
}
func getColumnNames(tableInfo *model.TableInfo, permutation []int) []string {
colIndexes := make([]int, 0, len(permutation))
for i := 0; i < len(permutation); i++ {
colIndexes = append(colIndexes, -1)
}
colCnt := 0
for i, p := range permutation {
if p >= 0 {
colIndexes[p] = i
colCnt++
}
}
names := make([]string, 0, colCnt)
for _, idx := range colIndexes {
// skip columns with index -1
if idx >= 0 {
// original fields contains _tidb_rowid field
if idx == len(tableInfo.Columns) {
names = append(names, model.ExtraHandleName.O)
} else {
names = append(names, tableInfo.Columns[idx].Name.O)
}
}
}
return names
}
func (cr *chunkProcessor) process(
ctx context.Context,
t *TableImporter,
engineID int32,
dataEngine, indexEngine backend.EngineWriter,
rc *Controller,
) error {
logger := t.logger.With(
zap.Int32("engineNumber", engineID),
zap.Int("fileIndex", cr.index),
zap.Stringer("path", &cr.chunk.Key),
)
// Create the encoder.
kvEncoder, err := rc.encBuilder.NewEncoder(ctx, &encode.EncodingConfig{
SessionOptions: encode.SessionOptions{
SQLMode: rc.cfg.TiDB.SQLMode,
Timestamp: cr.chunk.Timestamp,
SysVars: rc.sysVars,
// use chunk.PrevRowIDMax as the auto random seed, so it can stay the same value after recover from checkpoint.
AutoRandomSeed: cr.chunk.Chunk.PrevRowIDMax,
},
Path: cr.chunk.Key.Path,
Table: t.encTable,
Logger: logger,
})
if err != nil {
return err
}
defer kvEncoder.Close()
kvsCh := make(chan []deliveredKVs, maxKVQueueSize)
deliverCompleteCh := make(chan deliverResult)
go func() {
defer close(deliverCompleteCh)
dur, err := cr.deliverLoop(ctx, kvsCh, t, engineID, dataEngine, indexEngine, rc)
select {
case <-ctx.Done():
case deliverCompleteCh <- deliverResult{dur, err}:
}
}()
logTask := logger.Begin(zap.InfoLevel, "restore file")
readTotalDur, encodeTotalDur, encodeErr := cr.encodeLoop(ctx, kvsCh, t, logger, kvEncoder, deliverCompleteCh, rc)
var deliverErr error
select {
case deliverResult, ok := <-deliverCompleteCh:
if ok {
logTask.End(zap.ErrorLevel, deliverResult.err,
zap.Duration("readDur", readTotalDur),
zap.Duration("encodeDur", encodeTotalDur),
zap.Duration("deliverDur", deliverResult.totalDur),
zap.Object("checksum", &cr.chunk.Checksum),
)
deliverErr = deliverResult.err
} else {
// else, this must cause by ctx cancel
deliverErr = ctx.Err()
}
case <-ctx.Done():
deliverErr = ctx.Err()
}
return errors.Trace(firstErr(encodeErr, deliverErr))
}
//nolint:nakedret // TODO: refactor
func (cr *chunkProcessor) encodeLoop(
ctx context.Context,
kvsCh chan<- []deliveredKVs,
t *TableImporter,
logger log.Logger,
kvEncoder encode.Encoder,
deliverCompleteCh <-chan deliverResult,
rc *Controller,
) (readTotalDur time.Duration, encodeTotalDur time.Duration, err error) {
defer close(kvsCh)
send := func(kvs []deliveredKVs) error {
select {
case kvsCh <- kvs:
return nil
case <-ctx.Done():
return ctx.Err()
case deliverResult, ok := <-deliverCompleteCh:
if deliverResult.err == nil && !ok {
deliverResult.err = ctx.Err()
}
if deliverResult.err == nil {
deliverResult.err = errors.New("unexpected premature fulfillment")
logger.DPanic("unexpected: deliverCompleteCh prematurely fulfilled with no error", zap.Bool("chIsOpen", ok))
}
return errors.Trace(deliverResult.err)
}
}
pauser, maxKvPairsCnt := rc.pauser, rc.cfg.TikvImporter.MaxKVPairs
initializedColumns, reachEOF := false, false
// filteredColumns is column names that excluded ignored columns
// WARN: this might be not correct when different SQL statements contains different fields,
// but since ColumnPermutation also depends on the hypothesis that the columns in one source file is the same
// so this should be ok.
var (
filteredColumns []string
extendVals []types.Datum
)
ignoreColumns, err1 := rc.cfg.Mydumper.IgnoreColumns.GetIgnoreColumns(t.dbInfo.Name, t.tableInfo.Core.Name.O, rc.cfg.Mydumper.CaseSensitive)
if err1 != nil {
err = err1
return
}
for !reachEOF {
if err = pauser.Wait(ctx); err != nil {
return
}
offset, _ := cr.parser.Pos()
if offset >= cr.chunk.Chunk.EndOffset {
break
}
var readDur, encodeDur time.Duration
canDeliver := false
kvPacket := make([]deliveredKVs, 0, maxKvPairsCnt)
curOffset := offset
var newOffset, rowID, newScannedOffset int64
var scannedOffset int64 = -1
var kvSize uint64
var scannedOffsetErr error
outLoop:
for !canDeliver {
readDurStart := time.Now()
err = cr.parser.ReadRow()
columnNames := cr.parser.Columns()
newOffset, rowID = cr.parser.Pos()
if cr.chunk.FileMeta.Compression != mydump.CompressionNone || cr.chunk.FileMeta.Type == mydump.SourceTypeParquet {
newScannedOffset, scannedOffsetErr = cr.parser.ScannedPos()
if scannedOffsetErr != nil {
logger.Warn("fail to get data engine ScannedPos, progress may not be accurate",
log.ShortError(scannedOffsetErr), zap.String("file", cr.chunk.FileMeta.Path))
}
if scannedOffset == -1 {
scannedOffset = newScannedOffset
}
}
switch errors.Cause(err) {
case nil:
if !initializedColumns {
if len(cr.chunk.ColumnPermutation) == 0 {
if err = t.initializeColumns(columnNames, cr.chunk); err != nil {
return
}
}
filteredColumns = columnNames
ignoreColsMap := ignoreColumns.ColumnsMap()
if len(ignoreColsMap) > 0 || len(cr.chunk.FileMeta.ExtendData.Columns) > 0 {
filteredColumns, extendVals = filterColumns(columnNames, cr.chunk.FileMeta.ExtendData, ignoreColsMap, t.tableInfo.Core)
}
lastRow := cr.parser.LastRow()
lastRowLen := len(lastRow.Row)
extendColsMap := make(map[string]int)
for i, c := range cr.chunk.FileMeta.ExtendData.Columns {
extendColsMap[c] = lastRowLen + i
}
for i, col := range t.tableInfo.Core.Columns {
if p, ok := extendColsMap[col.Name.O]; ok {
cr.chunk.ColumnPermutation[i] = p
}
}
initializedColumns = true
}
case io.EOF:
reachEOF = true
break outLoop
default:
err = common.ErrEncodeKV.Wrap(err).GenWithStackByArgs(&cr.chunk.Key, newOffset)
return
}
readDur += time.Since(readDurStart)
encodeDurStart := time.Now()
lastRow := cr.parser.LastRow()
lastRow.Row = append(lastRow.Row, extendVals...)
// sql -> kv
kvs, encodeErr := kvEncoder.Encode(lastRow.Row, lastRow.RowID, cr.chunk.ColumnPermutation, curOffset)
encodeDur += time.Since(encodeDurStart)
hasIgnoredEncodeErr := false
if encodeErr != nil {
rowText := tidb.EncodeRowForRecord(ctx, t.encTable, rc.cfg.TiDB.SQLMode, lastRow.Row, cr.chunk.ColumnPermutation)
encodeErr = rc.errorMgr.RecordTypeError(ctx, logger, t.tableName, cr.chunk.Key.Path, newOffset, rowText, encodeErr)
if encodeErr != nil {
err = common.ErrEncodeKV.Wrap(encodeErr).GenWithStackByArgs(&cr.chunk.Key, newOffset)
}
hasIgnoredEncodeErr = true
}
cr.parser.RecycleRow(lastRow)
curOffset = newOffset
if err != nil {
return
}
if hasIgnoredEncodeErr {
continue
}
kvPacket = append(kvPacket, deliveredKVs{kvs: kvs, columns: filteredColumns, offset: newOffset,
rowID: rowID, realOffset: newScannedOffset})
kvSize += kvs.Size()
failpoint.Inject("mock-kv-size", func(val failpoint.Value) {
kvSize += uint64(val.(int))
})
// pebble cannot allow > 4.0G kv in one batch.
// we will meet pebble panic when import sql file and each kv has the size larger than 4G / maxKvPairsCnt.
// so add this check.
if kvSize >= minDeliverBytes || len(kvPacket) >= maxKvPairsCnt || newOffset == cr.chunk.Chunk.EndOffset {
canDeliver = true
kvSize = 0
}
}
encodeTotalDur += encodeDur
readTotalDur += readDur
if m, ok := metric.FromContext(ctx); ok {
m.RowEncodeSecondsHistogram.Observe(encodeDur.Seconds())
m.RowReadSecondsHistogram.Observe(readDur.Seconds())
if cr.chunk.FileMeta.Type == mydump.SourceTypeParquet {
m.RowReadBytesHistogram.Observe(float64(newScannedOffset - scannedOffset))
} else {
m.RowReadBytesHistogram.Observe(float64(newOffset - offset))
}
}
if len(kvPacket) != 0 {
deliverKvStart := time.Now()
if err = send(kvPacket); err != nil {
return
}
if m, ok := metric.FromContext(ctx); ok {
m.RowKVDeliverSecondsHistogram.Observe(time.Since(deliverKvStart).Seconds())
}
}
}
err = send([]deliveredKVs{{offset: cr.chunk.Chunk.EndOffset, realOffset: cr.chunk.FileMeta.FileSize}})
return
}
//nolint:nakedret // TODO: refactor
func (cr *chunkProcessor) deliverLoop(
ctx context.Context,
kvsCh <-chan []deliveredKVs,
t *TableImporter,
engineID int32,
dataEngine, indexEngine backend.EngineWriter,
rc *Controller,
) (deliverTotalDur time.Duration, err error) {
deliverLogger := t.logger.With(
zap.Int32("engineNumber", engineID),
zap.Int("fileIndex", cr.index),
zap.Stringer("path", &cr.chunk.Key),
zap.String("task", "deliver"),
)
// Fetch enough KV pairs from the source.
dataKVs := rc.encBuilder.MakeEmptyRows()
indexKVs := rc.encBuilder.MakeEmptyRows()
dataSynced := true
hasMoreKVs := true
var startRealOffset, currRealOffset int64 // save to 0 at first
for hasMoreKVs {
c := keyspace.CodecV1
if t.kvStore != nil {
c = t.kvStore.GetCodec()
}
var (
dataChecksum = verify.NewKVChecksumWithKeyspace(c)
indexChecksum = verify.NewKVChecksumWithKeyspace(c)
)
var columns []string
var kvPacket []deliveredKVs
// init these two field as checkpoint current value, so even if there are no kv pairs delivered,
// chunk checkpoint should stay the same
startOffset := cr.chunk.Chunk.Offset
currOffset := startOffset
startRealOffset = cr.chunk.Chunk.RealOffset
currRealOffset = startRealOffset
rowID := cr.chunk.Chunk.PrevRowIDMax
populate:
for dataChecksum.SumSize()+indexChecksum.SumSize() < minDeliverBytes {
select {
case kvPacket = <-kvsCh:
if len(kvPacket) == 0 {
hasMoreKVs = false
break populate
}
for _, p := range kvPacket {
if p.kvs == nil {
// This is the last message.
currOffset = p.offset
currRealOffset = p.realOffset
hasMoreKVs = false
break populate
}
p.kvs.ClassifyAndAppend(&dataKVs, dataChecksum, &indexKVs, indexChecksum)
columns = p.columns
currOffset = p.offset
currRealOffset = p.realOffset
rowID = p.rowID
}
case <-ctx.Done():
err = ctx.Err()
return
}
}
err = func() error {
// We use `TryRLock` with sleep here to avoid blocking current goroutine during importing when disk-quota is
// triggered, so that we can save chunkCheckpoint as soon as possible after `FlushEngine` is called.
// This implementation may not be very elegant or even completely correct, but it is currently a relatively
// simple and effective solution.
for !rc.diskQuotaLock.TryRLock() {
// try to update chunk checkpoint, this can help save checkpoint after importing when disk-quota is triggered
if !dataSynced {
dataSynced = cr.maybeSaveCheckpoint(rc, t, engineID, cr.chunk, dataEngine, indexEngine)
}
time.Sleep(time.Millisecond)
}
defer rc.diskQuotaLock.RUnlock()
// Write KVs into the engine
start := time.Now()
if err = dataEngine.AppendRows(ctx, columns, dataKVs); err != nil {
if !common.IsContextCanceledError(err) {
deliverLogger.Error("write to data engine failed", log.ShortError(err))
}
return errors.Trace(err)
}
if err = indexEngine.AppendRows(ctx, columns, indexKVs); err != nil {
if !common.IsContextCanceledError(err) {
deliverLogger.Error("write to index engine failed", log.ShortError(err))
}
return errors.Trace(err)
}
if m, ok := metric.FromContext(ctx); ok {
deliverDur := time.Since(start)
deliverTotalDur += deliverDur
m.BlockDeliverSecondsHistogram.Observe(deliverDur.Seconds())
m.BlockDeliverBytesHistogram.WithLabelValues(metric.BlockDeliverKindData).Observe(float64(dataChecksum.SumSize()))
m.BlockDeliverBytesHistogram.WithLabelValues(metric.BlockDeliverKindIndex).Observe(float64(indexChecksum.SumSize()))
m.BlockDeliverKVPairsHistogram.WithLabelValues(metric.BlockDeliverKindData).Observe(float64(dataChecksum.SumKVS()))
m.BlockDeliverKVPairsHistogram.WithLabelValues(metric.BlockDeliverKindIndex).Observe(float64(indexChecksum.SumKVS()))
}
return nil
}()
if err != nil {
return
}
dataSynced = false
dataKVs = dataKVs.Clear()
indexKVs = indexKVs.Clear()
// Update the table, and save a checkpoint.
// (the write to the importer is effective immediately, thus update these here)
// No need to apply a lock since this is the only thread updating `cr.chunk.**`.
// In local mode, we should write these checkpoints after engine flushed.
lastOffset := cr.chunk.Chunk.Offset
cr.chunk.Checksum.Add(dataChecksum)
cr.chunk.Checksum.Add(indexChecksum)
cr.chunk.Chunk.Offset = currOffset
cr.chunk.Chunk.RealOffset = currRealOffset
cr.chunk.Chunk.PrevRowIDMax = rowID
if m, ok := metric.FromContext(ctx); ok {
// value of currOffset comes from parser.pos which increase monotonically. the init value of parser.pos
// comes from chunk.Chunk.Offset. so it shouldn't happen that currOffset - startOffset < 0.
// but we met it one time, but cannot reproduce it now, we add this check to make code more robust
// TODO: reproduce and find the root cause and fix it completely
var lowOffset, highOffset int64
if cr.chunk.FileMeta.Compression != mydump.CompressionNone {
lowOffset, highOffset = startRealOffset, currRealOffset
} else {
lowOffset, highOffset = startOffset, currOffset
}
delta := highOffset - lowOffset
if delta >= 0 {
if cr.chunk.FileMeta.Type == mydump.SourceTypeParquet {
if currRealOffset > startRealOffset {
m.BytesCounter.WithLabelValues(metric.StateRestored).Add(float64(currRealOffset - startRealOffset))
}
m.RowsCounter.WithLabelValues(metric.StateRestored, t.tableName).Add(float64(delta))
} else {
m.BytesCounter.WithLabelValues(metric.StateRestored).Add(float64(delta))
m.RowsCounter.WithLabelValues(metric.StateRestored, t.tableName).Add(float64(dataChecksum.SumKVS()))
}
if rc.status != nil && rc.status.backend == config.BackendTiDB {
rc.status.FinishedFileSize.Add(delta)
}
} else {
deliverLogger.Warn("offset go back", zap.Int64("curr", highOffset),
zap.Int64("start", lowOffset))
}
}
if currOffset > lastOffset || dataChecksum.SumKVS() != 0 || indexChecksum.SumKVS() != 0 {
// No need to save checkpoint if nothing was delivered.
dataSynced = cr.maybeSaveCheckpoint(rc, t, engineID, cr.chunk, dataEngine, indexEngine)
}
failpoint.Inject("SlowDownWriteRows", func() {
deliverLogger.Warn("Slowed down write rows")
finished := rc.status.FinishedFileSize.Load()
total := rc.status.TotalFileSize.Load()
deliverLogger.Warn("PrintStatus Failpoint",
zap.Int64("finished", finished),
zap.Int64("total", total))
})
failpoint.Inject("FailAfterWriteRows", nil)
// TODO: for local backend, we may save checkpoint more frequently, e.g. after written
// 10GB kv pairs to data engine, we can do a flush for both data & index engine, then we
// can safely update current checkpoint.
failpoint.Inject("LocalBackendSaveCheckpoint", func() {
if !isLocalBackend(rc.cfg) && (dataChecksum.SumKVS() != 0 || indexChecksum.SumKVS() != 0) {
// No need to save checkpoint if nothing was delivered.
saveCheckpoint(rc, t, engineID, cr.chunk)
}
})
}
return
}
func (*chunkProcessor) maybeSaveCheckpoint(
rc *Controller,
t *TableImporter,
engineID int32,
chunk *checkpoints.ChunkCheckpoint,
data, index backend.EngineWriter,
) bool {
if data.IsSynced() && index.IsSynced() {
saveCheckpoint(rc, t, engineID, chunk)
return true
}
return false
}
func (cr *chunkProcessor) close() {
_ = cr.parser.Close()
}