/
checkpoints.go
1791 lines (1604 loc) · 54.3 KB
/
checkpoints.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
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900
901
902
903
904
905
906
907
908
909
910
911
912
913
914
915
916
917
918
919
920
921
922
923
924
925
926
927
928
929
930
931
932
933
934
935
936
937
938
939
940
941
942
943
944
945
946
947
948
949
950
951
952
953
954
955
956
957
958
959
960
961
962
963
964
965
966
967
968
969
970
971
972
973
974
975
976
977
978
979
980
981
982
983
984
985
986
987
988
989
990
991
992
993
994
995
996
997
998
999
1000
// Copyright 2019 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 checkpoints
import (
"context"
"database/sql"
"encoding/json"
"fmt"
"io"
"math"
"path"
"sort"
"strings"
"sync"
"github.com/joho/sqltocsv"
"github.com/pingcap/errors"
"github.com/pingcap/tidb/br/pkg/lightning/checkpoints/checkpointspb"
"github.com/pingcap/tidb/br/pkg/lightning/common"
"github.com/pingcap/tidb/br/pkg/lightning/config"
"github.com/pingcap/tidb/br/pkg/lightning/log"
"github.com/pingcap/tidb/br/pkg/lightning/mydump"
verify "github.com/pingcap/tidb/br/pkg/lightning/verification"
"github.com/pingcap/tidb/br/pkg/storage"
"github.com/pingcap/tidb/br/pkg/version/build"
"github.com/pingcap/tidb/util/mathutil"
"go.uber.org/zap"
"golang.org/x/exp/slices"
)
type CheckpointStatus uint8
const (
CheckpointStatusMissing CheckpointStatus = 0
CheckpointStatusMaxInvalid CheckpointStatus = 25
CheckpointStatusLoaded CheckpointStatus = 30
CheckpointStatusAllWritten CheckpointStatus = 60
CheckpointStatusClosed CheckpointStatus = 90
CheckpointStatusImported CheckpointStatus = 120
CheckpointStatusIndexImported CheckpointStatus = 140
CheckpointStatusAlteredAutoInc CheckpointStatus = 150
CheckpointStatusChecksumSkipped CheckpointStatus = 170
CheckpointStatusChecksummed CheckpointStatus = 180
CheckpointStatusAnalyzeSkipped CheckpointStatus = 200
CheckpointStatusAnalyzed CheckpointStatus = 210
)
const WholeTableEngineID = math.MaxInt32
const (
// the table names to store each kind of checkpoint in the checkpoint database
// remember to increase the version number in case of incompatible change.
CheckpointTableNameTask = "task_v2"
CheckpointTableNameTable = "table_v7"
CheckpointTableNameEngine = "engine_v5"
CheckpointTableNameChunk = "chunk_v5"
// Some frequently used table name or constants.
allTables = "all"
stringLitAll = "'all'"
columnTableName = "table_name"
)
const (
// shared by MySQLCheckpointsDB and GlueCheckpointsDB
CreateDBTemplate = "CREATE DATABASE IF NOT EXISTS %s;"
CreateTaskTableTemplate = `
CREATE TABLE IF NOT EXISTS %s.%s (
id tinyint(1) PRIMARY KEY,
task_id bigint NOT NULL,
source_dir varchar(256) NOT NULL,
backend varchar(16) NOT NULL,
importer_addr varchar(256),
tidb_host varchar(128) NOT NULL,
tidb_port int NOT NULL,
pd_addr varchar(128) NOT NULL,
sorted_kv_dir varchar(256) NOT NULL,
lightning_ver varchar(48) NOT NULL
);`
CreateTableTableTemplate = `
CREATE TABLE IF NOT EXISTS %s.%s (
task_id bigint NOT NULL,
table_name varchar(261) NOT NULL PRIMARY KEY,
hash binary(32) NOT NULL,
status tinyint unsigned DEFAULT 30,
alloc_base bigint NOT NULL DEFAULT 0,
table_id bigint NOT NULL DEFAULT 0,
create_time timestamp NOT NULL DEFAULT CURRENT_TIMESTAMP,
update_time timestamp NOT NULL DEFAULT CURRENT_TIMESTAMP ON UPDATE CURRENT_TIMESTAMP,
kv_bytes bigint unsigned NOT NULL DEFAULT 0,
kv_kvs bigint unsigned NOT NULL DEFAULT 0,
kv_checksum bigint unsigned NOT NULL DEFAULT 0,
INDEX(task_id)
);`
CreateEngineTableTemplate = `
CREATE TABLE IF NOT EXISTS %s.%s (
table_name varchar(261) NOT NULL,
engine_id int NOT NULL,
status tinyint unsigned DEFAULT 30,
create_time timestamp NOT NULL DEFAULT CURRENT_TIMESTAMP,
update_time timestamp NOT NULL DEFAULT CURRENT_TIMESTAMP ON UPDATE CURRENT_TIMESTAMP,
PRIMARY KEY(table_name, engine_id DESC)
);`
CreateChunkTableTemplate = `
CREATE TABLE IF NOT EXISTS %s.%s (
table_name varchar(261) NOT NULL,
engine_id int unsigned NOT NULL,
path varchar(2048) NOT NULL,
offset bigint NOT NULL,
type int NOT NULL,
compression int NOT NULL,
sort_key varchar(256) NOT NULL,
file_size bigint NOT NULL,
columns text NULL,
should_include_row_id BOOL NOT NULL,
end_offset bigint NOT NULL,
pos bigint NOT NULL,
prev_rowid_max bigint NOT NULL,
rowid_max bigint NOT NULL,
kvc_bytes bigint unsigned NOT NULL DEFAULT 0,
kvc_kvs bigint unsigned NOT NULL DEFAULT 0,
kvc_checksum bigint unsigned NOT NULL DEFAULT 0,
create_time timestamp NOT NULL DEFAULT CURRENT_TIMESTAMP,
update_time timestamp NOT NULL DEFAULT CURRENT_TIMESTAMP ON UPDATE CURRENT_TIMESTAMP,
PRIMARY KEY(table_name, engine_id, path(500), offset)
);`
InitTaskTemplate = `
REPLACE INTO %s.%s (id, task_id, source_dir, backend, importer_addr, tidb_host, tidb_port, pd_addr, sorted_kv_dir, lightning_ver)
VALUES (1, ?, ?, ?, ?, ?, ?, ?, ?, ?);`
InitTableTemplate = `
INSERT INTO %s.%s (task_id, table_name, hash, table_id) VALUES (?, ?, ?, ?)
ON DUPLICATE KEY UPDATE task_id = CASE
WHEN hash = VALUES(hash)
THEN VALUES(task_id)
END;`
ReadTaskTemplate = `
SELECT task_id, source_dir, backend, importer_addr, tidb_host, tidb_port, pd_addr, sorted_kv_dir, lightning_ver FROM %s.%s WHERE id = 1;`
ReadEngineTemplate = `
SELECT engine_id, status FROM %s.%s WHERE table_name = ? ORDER BY engine_id DESC;`
ReadChunkTemplate = `
SELECT
engine_id, path, offset, type, compression, sort_key, file_size, columns,
pos, end_offset, prev_rowid_max, rowid_max,
kvc_bytes, kvc_kvs, kvc_checksum, unix_timestamp(create_time)
FROM %s.%s WHERE table_name = ?
ORDER BY engine_id, path, offset;`
ReadTableRemainTemplate = `
SELECT status, alloc_base, table_id, kv_bytes, kv_kvs, kv_checksum FROM %s.%s WHERE table_name = ?;`
ReplaceEngineTemplate = `
REPLACE INTO %s.%s (table_name, engine_id, status) VALUES (?, ?, ?);`
ReplaceChunkTemplate = `
REPLACE INTO %s.%s (
table_name, engine_id,
path, offset, type, compression, sort_key, file_size, columns, should_include_row_id,
pos, end_offset, prev_rowid_max, rowid_max,
kvc_bytes, kvc_kvs, kvc_checksum, create_time
) VALUES (
?, ?,
?, ?, ?, ?, ?, ?, ?, FALSE,
?, ?, ?, ?,
0, 0, 0, from_unixtime(?)
);`
UpdateChunkTemplate = `
UPDATE %s.%s SET pos = ?, prev_rowid_max = ?, kvc_bytes = ?, kvc_kvs = ?, kvc_checksum = ?, columns = ?
WHERE (table_name, engine_id, path, offset) = (?, ?, ?, ?);`
UpdateTableRebaseTemplate = `
UPDATE %s.%s SET alloc_base = GREATEST(?, alloc_base) WHERE table_name = ?;`
UpdateTableStatusTemplate = `
UPDATE %s.%s SET status = ? WHERE table_name = ?;`
UpdateTableChecksumTemplate = `UPDATE %s.%s SET kv_bytes = ?, kv_kvs = ?, kv_checksum = ? WHERE table_name = ?;`
UpdateEngineTemplate = `
UPDATE %s.%s SET status = ? WHERE (table_name, engine_id) = (?, ?);`
DeleteCheckpointRecordTemplate = "DELETE FROM %s.%s WHERE table_name = ?;"
)
func IsCheckpointTable(name string) bool {
switch name {
case CheckpointTableNameTask, CheckpointTableNameTable, CheckpointTableNameEngine, CheckpointTableNameChunk:
return true
default:
return false
}
}
func (status CheckpointStatus) MetricName() string {
switch status {
case CheckpointStatusLoaded:
return "pending"
case CheckpointStatusAllWritten:
return "written"
case CheckpointStatusClosed:
return "closed"
case CheckpointStatusImported:
return "imported"
case CheckpointStatusIndexImported:
return "index_imported"
case CheckpointStatusAlteredAutoInc:
return "altered_auto_inc"
case CheckpointStatusChecksummed, CheckpointStatusChecksumSkipped:
return "checksum"
case CheckpointStatusAnalyzed, CheckpointStatusAnalyzeSkipped:
return "analyzed"
case CheckpointStatusMissing:
return "missing"
default:
return "invalid"
}
}
type ChunkCheckpointKey struct {
Path string
Offset int64
}
func (key *ChunkCheckpointKey) String() string {
return fmt.Sprintf("%s:%d", key.Path, key.Offset)
}
func (key *ChunkCheckpointKey) less(other *ChunkCheckpointKey) bool {
switch {
case key.Path < other.Path:
return true
case key.Path > other.Path:
return false
default:
return key.Offset < other.Offset
}
}
type ChunkCheckpoint struct {
Key ChunkCheckpointKey
FileMeta mydump.SourceFileMeta
ColumnPermutation []int
Chunk mydump.Chunk
Checksum verify.KVChecksum
Timestamp int64
}
func (ccp *ChunkCheckpoint) DeepCopy() *ChunkCheckpoint {
colPerm := make([]int, 0, len(ccp.ColumnPermutation))
colPerm = append(colPerm, ccp.ColumnPermutation...)
return &ChunkCheckpoint{
Key: ccp.Key,
FileMeta: ccp.FileMeta,
ColumnPermutation: colPerm,
Chunk: ccp.Chunk,
Checksum: ccp.Checksum,
Timestamp: ccp.Timestamp,
}
}
type EngineCheckpoint struct {
Status CheckpointStatus
Chunks []*ChunkCheckpoint // a sorted array
}
func (engine *EngineCheckpoint) DeepCopy() *EngineCheckpoint {
chunks := make([]*ChunkCheckpoint, 0, len(engine.Chunks))
for _, chunk := range engine.Chunks {
chunks = append(chunks, chunk.DeepCopy())
}
return &EngineCheckpoint{
Status: engine.Status,
Chunks: chunks,
}
}
type TableCheckpoint struct {
Status CheckpointStatus
AllocBase int64
Engines map[int32]*EngineCheckpoint
TableID int64
// remote checksum before restore
Checksum verify.KVChecksum
}
func (cp *TableCheckpoint) DeepCopy() *TableCheckpoint {
engines := make(map[int32]*EngineCheckpoint, len(cp.Engines))
for engineID, engine := range cp.Engines {
engines[engineID] = engine.DeepCopy()
}
return &TableCheckpoint{
Status: cp.Status,
AllocBase: cp.AllocBase,
Engines: engines,
TableID: cp.TableID,
Checksum: cp.Checksum,
}
}
func (cp *TableCheckpoint) CountChunks() int {
result := 0
for _, engine := range cp.Engines {
result += len(engine.Chunks)
}
return result
}
type chunkCheckpointDiff struct {
pos int64
rowID int64
checksum verify.KVChecksum
columnPermutation []int
}
type engineCheckpointDiff struct {
hasStatus bool
status CheckpointStatus
chunks map[ChunkCheckpointKey]chunkCheckpointDiff
}
type TableCheckpointDiff struct {
hasStatus bool
hasRebase bool
hasChecksum bool
status CheckpointStatus
allocBase int64
engines map[int32]engineCheckpointDiff
checksum verify.KVChecksum
}
func NewTableCheckpointDiff() *TableCheckpointDiff {
return &TableCheckpointDiff{
engines: make(map[int32]engineCheckpointDiff),
}
}
func (cpd *TableCheckpointDiff) insertEngineCheckpointDiff(engineID int32, newDiff engineCheckpointDiff) {
if oldDiff, ok := cpd.engines[engineID]; ok {
if newDiff.hasStatus {
oldDiff.hasStatus = true
oldDiff.status = newDiff.status
}
for key, chunkDiff := range newDiff.chunks {
oldDiff.chunks[key] = chunkDiff
}
newDiff = oldDiff
}
cpd.engines[engineID] = newDiff
}
func (cpd *TableCheckpointDiff) String() string {
return fmt.Sprintf(
"{hasStatus:%v, hasRebase:%v, status:%d, allocBase:%d, engines:[%d]}",
cpd.hasStatus, cpd.hasRebase, cpd.status, cpd.allocBase, len(cpd.engines),
)
}
// Apply the diff to the existing chunk and engine checkpoints in `cp`.
func (cp *TableCheckpoint) Apply(cpd *TableCheckpointDiff) {
if cpd.hasStatus {
cp.Status = cpd.status
}
if cpd.hasRebase {
cp.AllocBase = cpd.allocBase
}
for engineID, engineDiff := range cpd.engines {
engine := cp.Engines[engineID]
if engine == nil {
continue
}
if engineDiff.hasStatus {
engine.Status = engineDiff.status
}
for key, diff := range engineDiff.chunks {
checkpointKey := key
index := sort.Search(len(engine.Chunks), func(i int) bool {
return !engine.Chunks[i].Key.less(&checkpointKey)
})
if index >= len(engine.Chunks) {
continue
}
chunk := engine.Chunks[index]
if chunk.Key != checkpointKey {
continue
}
chunk.Chunk.Offset = diff.pos
chunk.Chunk.PrevRowIDMax = diff.rowID
chunk.Checksum = diff.checksum
}
}
}
type TableCheckpointMerger interface {
// MergeInto the table checkpoint diff from a status update or chunk update.
// If there are multiple updates to the same table, only the last one will
// take effect. Therefore, the caller must ensure events for the same table
// are properly ordered by the global time (an old event must be merged
// before the new one).
MergeInto(cpd *TableCheckpointDiff)
}
type StatusCheckpointMerger struct {
EngineID int32 // WholeTableEngineID == apply to whole table.
Status CheckpointStatus
}
func (merger *StatusCheckpointMerger) SetInvalid() {
merger.Status /= 10
}
func (merger *StatusCheckpointMerger) MergeInto(cpd *TableCheckpointDiff) {
if merger.EngineID == WholeTableEngineID || merger.Status <= CheckpointStatusMaxInvalid {
cpd.status = merger.Status
cpd.hasStatus = true
}
if merger.EngineID != WholeTableEngineID {
cpd.insertEngineCheckpointDiff(merger.EngineID, engineCheckpointDiff{
hasStatus: true,
status: merger.Status,
chunks: make(map[ChunkCheckpointKey]chunkCheckpointDiff),
})
}
}
type ChunkCheckpointMerger struct {
EngineID int32
Key ChunkCheckpointKey
Checksum verify.KVChecksum
Pos int64
RowID int64
ColumnPermutation []int
EndOffset int64 // For test only.
}
func (merger *ChunkCheckpointMerger) MergeInto(cpd *TableCheckpointDiff) {
cpd.insertEngineCheckpointDiff(merger.EngineID, engineCheckpointDiff{
chunks: map[ChunkCheckpointKey]chunkCheckpointDiff{
merger.Key: {
pos: merger.Pos,
rowID: merger.RowID,
checksum: merger.Checksum,
columnPermutation: merger.ColumnPermutation,
},
},
})
}
type TableChecksumMerger struct {
Checksum verify.KVChecksum
}
func (m *TableChecksumMerger) MergeInto(cpd *TableCheckpointDiff) {
cpd.hasChecksum = true
cpd.checksum = m.Checksum
}
type RebaseCheckpointMerger struct {
AllocBase int64
}
func (merger *RebaseCheckpointMerger) MergeInto(cpd *TableCheckpointDiff) {
cpd.hasRebase = true
cpd.allocBase = mathutil.Max(cpd.allocBase, merger.AllocBase)
}
type DestroyedTableCheckpoint struct {
TableName string
MinEngineID int32
MaxEngineID int32
}
type TaskCheckpoint struct {
TaskID int64
SourceDir string
Backend string
ImporterAddr string
TiDBHost string
TiDBPort int
PdAddr string
SortedKVDir string
LightningVer string
}
type DB interface {
Initialize(ctx context.Context, cfg *config.Config, dbInfo map[string]*TidbDBInfo) error
TaskCheckpoint(ctx context.Context) (*TaskCheckpoint, error)
Get(ctx context.Context, tableName string) (*TableCheckpoint, error)
Close() error
// InsertEngineCheckpoints initializes the checkpoints related to a table.
// It assumes the entire table has not been imported before and will fill in
// default values for the column permutations and checksums.
InsertEngineCheckpoints(ctx context.Context, tableName string, checkpoints map[int32]*EngineCheckpoint) error
Update(taskCtx context.Context, checkpointDiffs map[string]*TableCheckpointDiff) error
RemoveCheckpoint(ctx context.Context, tableName string) error
// MoveCheckpoints renames the checkpoint schema to include a suffix
// including the taskID (e.g. `tidb_lightning_checkpoints.1234567890.bak`).
MoveCheckpoints(ctx context.Context, taskID int64) error
// GetLocalStoringTables returns a map containing tables have engine files stored on local disk.
// currently only meaningful for local backend
GetLocalStoringTables(ctx context.Context) (map[string][]int32, error)
IgnoreErrorCheckpoint(ctx context.Context, tableName string) error
DestroyErrorCheckpoint(ctx context.Context, tableName string) ([]DestroyedTableCheckpoint, error)
DumpTables(ctx context.Context, csv io.Writer) error
DumpEngines(ctx context.Context, csv io.Writer) error
DumpChunks(ctx context.Context, csv io.Writer) error
}
func OpenCheckpointsDB(ctx context.Context, cfg *config.Config) (DB, error) {
if !cfg.Checkpoint.Enable {
return NewNullCheckpointsDB(), nil
}
switch cfg.Checkpoint.Driver {
case config.CheckpointDriverMySQL:
var (
db *sql.DB
err error
)
if cfg.Checkpoint.MySQLParam != nil {
db, err = cfg.Checkpoint.MySQLParam.Connect()
} else {
db, err = sql.Open("mysql", cfg.Checkpoint.DSN)
}
if err != nil {
return nil, errors.Trace(err)
}
cpdb, err := NewMySQLCheckpointsDB(ctx, db, cfg.Checkpoint.Schema)
if err != nil {
_ = db.Close()
return nil, errors.Trace(err)
}
return cpdb, nil
case config.CheckpointDriverFile:
cpdb, err := NewFileCheckpointsDB(ctx, cfg.Checkpoint.DSN)
if err != nil {
return nil, errors.Trace(err)
}
return cpdb, nil
default:
return nil, common.ErrUnknownCheckpointDriver.GenWithStackByArgs(cfg.Checkpoint.Driver)
}
}
func IsCheckpointsDBExists(ctx context.Context, cfg *config.Config) (bool, error) {
if !cfg.Checkpoint.Enable {
return false, nil
}
switch cfg.Checkpoint.Driver {
case config.CheckpointDriverMySQL:
var (
db *sql.DB
err error
)
if cfg.Checkpoint.MySQLParam != nil {
db, err = cfg.Checkpoint.MySQLParam.Connect()
} else {
db, err = sql.Open("mysql", cfg.Checkpoint.DSN)
}
if err != nil {
return false, errors.Trace(err)
}
//nolint: errcheck
defer db.Close()
checkSQL := "SHOW DATABASES WHERE `DATABASE` = ?"
rows, err := db.QueryContext(ctx, checkSQL, cfg.Checkpoint.Schema)
if err != nil {
return false, errors.Trace(err)
}
//nolint: errcheck
defer rows.Close()
result := rows.Next()
if err := rows.Err(); err != nil {
return false, errors.Trace(err)
}
return result, nil
case config.CheckpointDriverFile:
s, fileName, err := createExstorageByCompletePath(ctx, cfg.Checkpoint.DSN)
if err != nil {
return false, errors.Trace(err)
}
result, err := s.FileExists(ctx, fileName)
if err != nil {
return false, errors.Trace(err)
}
return result, nil
default:
return false, common.ErrUnknownCheckpointDriver.GenWithStackByArgs(cfg.Checkpoint.Driver)
}
}
// NullCheckpointsDB is a checkpoints database with no checkpoints.
type NullCheckpointsDB struct{}
func NewNullCheckpointsDB() *NullCheckpointsDB {
return &NullCheckpointsDB{}
}
func (*NullCheckpointsDB) Initialize(context.Context, *config.Config, map[string]*TidbDBInfo) error {
return nil
}
func (*NullCheckpointsDB) TaskCheckpoint(ctx context.Context) (*TaskCheckpoint, error) {
return nil, nil
}
func (*NullCheckpointsDB) Close() error {
return nil
}
func (*NullCheckpointsDB) Get(_ context.Context, _ string) (*TableCheckpoint, error) {
return &TableCheckpoint{
Status: CheckpointStatusLoaded,
Engines: map[int32]*EngineCheckpoint{},
}, nil
}
func (*NullCheckpointsDB) InsertEngineCheckpoints(_ context.Context, _ string, _ map[int32]*EngineCheckpoint) error {
return nil
}
func (*NullCheckpointsDB) Update(context.Context, map[string]*TableCheckpointDiff) error {
return nil
}
type MySQLCheckpointsDB struct {
db *sql.DB
schema string
}
func NewMySQLCheckpointsDB(ctx context.Context, db *sql.DB, schemaName string) (*MySQLCheckpointsDB, error) {
schema := common.EscapeIdentifier(schemaName)
sql := common.SQLWithRetry{
DB: db,
Logger: log.FromContext(ctx).With(zap.String("schema", schemaName)),
HideQueryLog: true,
}
err := sql.Exec(ctx, "create checkpoints database", fmt.Sprintf(CreateDBTemplate, schema))
if err != nil {
return nil, errors.Trace(err)
}
err = sql.Exec(ctx, "create task checkpoints table", fmt.Sprintf(CreateTaskTableTemplate, schema, CheckpointTableNameTask))
if err != nil {
return nil, errors.Trace(err)
}
err = sql.Exec(ctx, "create table checkpoints table", fmt.Sprintf(CreateTableTableTemplate, schema, CheckpointTableNameTable))
if err != nil {
return nil, errors.Trace(err)
}
err = sql.Exec(ctx, "create engine checkpoints table", fmt.Sprintf(CreateEngineTableTemplate, schema, CheckpointTableNameEngine))
if err != nil {
return nil, errors.Trace(err)
}
err = sql.Exec(ctx, "create chunks checkpoints table", fmt.Sprintf(CreateChunkTableTemplate, schema, CheckpointTableNameChunk))
if err != nil {
return nil, errors.Trace(err)
}
return &MySQLCheckpointsDB{
db: db,
schema: schema,
}, nil
}
func (cpdb *MySQLCheckpointsDB) Initialize(ctx context.Context, cfg *config.Config, dbInfo map[string]*TidbDBInfo) error {
// We can have at most 65535 placeholders https://stackoverflow.com/q/4922345/
// Since this step is not performance critical, we just insert the rows one-by-one.
s := common.SQLWithRetry{DB: cpdb.db, Logger: log.FromContext(ctx)}
err := s.Transact(ctx, "insert checkpoints", func(c context.Context, tx *sql.Tx) error {
taskStmt, err := tx.PrepareContext(c, fmt.Sprintf(InitTaskTemplate, cpdb.schema, CheckpointTableNameTask))
if err != nil {
return errors.Trace(err)
}
//nolint: errcheck
defer taskStmt.Close()
_, err = taskStmt.ExecContext(ctx, cfg.TaskID, cfg.Mydumper.SourceDir, cfg.TikvImporter.Backend,
cfg.TikvImporter.Addr, cfg.TiDB.Host, cfg.TiDB.Port, cfg.TiDB.PdAddr, cfg.TikvImporter.SortedKVDir,
build.ReleaseVersion)
if err != nil {
return errors.Trace(err)
}
// If `hash` is not the same but the `table_name` duplicates,
// the CASE expression will return NULL, which can be used to violate
// the NOT NULL requirement of `task_id` column, and caused this INSERT
// statement to fail with an irrecoverable error.
// We do need to capture the error is display a user friendly message
// (multiple nodes cannot import the same table) though.
stmt, err := tx.PrepareContext(c, fmt.Sprintf(InitTableTemplate, cpdb.schema, CheckpointTableNameTable))
if err != nil {
return errors.Trace(err)
}
//nolint: errcheck
defer stmt.Close()
for _, db := range dbInfo {
for _, table := range db.Tables {
tableName := common.UniqueTable(db.Name, table.Name)
_, err = stmt.ExecContext(c, cfg.TaskID, tableName, CheckpointStatusLoaded, table.ID)
if err != nil {
return errors.Trace(err)
}
}
}
return nil
})
if err != nil {
return errors.Trace(err)
}
return nil
}
func (cpdb *MySQLCheckpointsDB) TaskCheckpoint(ctx context.Context) (*TaskCheckpoint, error) {
s := common.SQLWithRetry{
DB: cpdb.db,
Logger: log.FromContext(ctx),
}
taskQuery := fmt.Sprintf(ReadTaskTemplate, cpdb.schema, CheckpointTableNameTask)
taskCp := &TaskCheckpoint{}
err := s.QueryRow(ctx, "fetch task checkpoint", taskQuery, &taskCp.TaskID, &taskCp.SourceDir, &taskCp.Backend,
&taskCp.ImporterAddr, &taskCp.TiDBHost, &taskCp.TiDBPort, &taskCp.PdAddr, &taskCp.SortedKVDir, &taskCp.LightningVer)
if err != nil {
// if task checkpoint is empty, return nil
if errors.Cause(err) == sql.ErrNoRows {
return nil, nil
}
return nil, errors.Trace(err)
}
return taskCp, nil
}
func (cpdb *MySQLCheckpointsDB) Close() error {
return errors.Trace(cpdb.db.Close())
}
func (cpdb *MySQLCheckpointsDB) Get(ctx context.Context, tableName string) (*TableCheckpoint, error) {
cp := &TableCheckpoint{
Engines: map[int32]*EngineCheckpoint{},
}
s := common.SQLWithRetry{
DB: cpdb.db,
Logger: log.FromContext(ctx).With(zap.String("table", tableName)),
}
err := s.Transact(ctx, "read checkpoint", func(c context.Context, tx *sql.Tx) error {
// 1. Populate the engines.
engineQuery := fmt.Sprintf(ReadEngineTemplate, cpdb.schema, CheckpointTableNameEngine)
engineRows, err := tx.QueryContext(c, engineQuery, tableName)
if err != nil {
return errors.Trace(err)
}
//nolint: errcheck
defer engineRows.Close()
for engineRows.Next() {
var (
engineID int32
status uint8
)
if err := engineRows.Scan(&engineID, &status); err != nil {
return errors.Trace(err)
}
cp.Engines[engineID] = &EngineCheckpoint{
Status: CheckpointStatus(status),
}
}
if err := engineRows.Err(); err != nil {
return errors.Trace(err)
}
// 2. Populate the chunks.
chunkQuery := fmt.Sprintf(ReadChunkTemplate, cpdb.schema, CheckpointTableNameChunk)
chunkRows, err := tx.QueryContext(c, chunkQuery, tableName)
if err != nil {
return errors.Trace(err)
}
//nolint: errcheck
defer chunkRows.Close()
for chunkRows.Next() {
var (
value = &ChunkCheckpoint{}
colPerm []byte
engineID int32
kvcBytes uint64
kvcKVs uint64
kvcChecksum uint64
)
if err := chunkRows.Scan(
&engineID, &value.Key.Path, &value.Key.Offset, &value.FileMeta.Type, &value.FileMeta.Compression,
&value.FileMeta.SortKey, &value.FileMeta.FileSize, &colPerm, &value.Chunk.Offset, &value.Chunk.EndOffset,
&value.Chunk.PrevRowIDMax, &value.Chunk.RowIDMax, &kvcBytes, &kvcKVs, &kvcChecksum,
&value.Timestamp,
); err != nil {
return errors.Trace(err)
}
value.FileMeta.Path = value.Key.Path
value.Checksum = verify.MakeKVChecksum(kvcBytes, kvcKVs, kvcChecksum)
if err := json.Unmarshal(colPerm, &value.ColumnPermutation); err != nil {
return errors.Trace(err)
}
cp.Engines[engineID].Chunks = append(cp.Engines[engineID].Chunks, value)
}
if err := chunkRows.Err(); err != nil {
return errors.Trace(err)
}
// 3. Fill in the remaining table info
tableQuery := fmt.Sprintf(ReadTableRemainTemplate, cpdb.schema, CheckpointTableNameTable)
tableRow := tx.QueryRowContext(c, tableQuery, tableName)
var status uint8
var kvs, bytes, checksum uint64
if err := tableRow.Scan(&status, &cp.AllocBase, &cp.TableID, &bytes, &kvs, &checksum); err != nil {
if err == sql.ErrNoRows {
return errors.NotFoundf("checkpoint for table %s", tableName)
}
}
cp.Checksum = verify.MakeKVChecksum(bytes, kvs, checksum)
cp.Status = CheckpointStatus(status)
return nil
})
if err != nil {
return nil, errors.Trace(err)
}
return cp, nil
}
func (cpdb *MySQLCheckpointsDB) InsertEngineCheckpoints(ctx context.Context, tableName string, checkpoints map[int32]*EngineCheckpoint) error {
s := common.SQLWithRetry{
DB: cpdb.db,
Logger: log.FromContext(ctx).With(zap.String("table", tableName)),
}
err := s.Transact(ctx, "update engine checkpoints", func(c context.Context, tx *sql.Tx) error {
engineStmt, err := tx.PrepareContext(c, fmt.Sprintf(ReplaceEngineTemplate, cpdb.schema, CheckpointTableNameEngine))
if err != nil {
return errors.Trace(err)
}
//nolint: errcheck
defer engineStmt.Close()
chunkStmt, err := tx.PrepareContext(c, fmt.Sprintf(ReplaceChunkTemplate, cpdb.schema, CheckpointTableNameChunk))
if err != nil {
return errors.Trace(err)
}
//nolint: errcheck
defer chunkStmt.Close()
for engineID, engine := range checkpoints {
_, err = engineStmt.ExecContext(c, tableName, engineID, engine.Status)
if err != nil {
return errors.Trace(err)
}
for _, value := range engine.Chunks {
columnPerm, err := json.Marshal(value.ColumnPermutation)
if err != nil {
return errors.Trace(err)
}
_, err = chunkStmt.ExecContext(
c, tableName, engineID,
value.Key.Path, value.Key.Offset, value.FileMeta.Type, value.FileMeta.Compression,
value.FileMeta.SortKey, value.FileMeta.FileSize, columnPerm, value.Chunk.Offset, value.Chunk.EndOffset,
value.Chunk.PrevRowIDMax, value.Chunk.RowIDMax, value.Timestamp,
)
if err != nil {
return errors.Trace(err)
}
}
}
return nil
})
if err != nil {
return errors.Trace(err)
}
return nil
}
func (cpdb *MySQLCheckpointsDB) Update(taskCtx context.Context, checkpointDiffs map[string]*TableCheckpointDiff) error {
chunkQuery := fmt.Sprintf(UpdateChunkTemplate, cpdb.schema, CheckpointTableNameChunk)
rebaseQuery := fmt.Sprintf(UpdateTableRebaseTemplate, cpdb.schema, CheckpointTableNameTable)
tableStatusQuery := fmt.Sprintf(UpdateTableStatusTemplate, cpdb.schema, CheckpointTableNameTable)
tableChecksumQuery := fmt.Sprintf(UpdateTableChecksumTemplate, cpdb.schema, CheckpointTableNameTable)
engineStatusQuery := fmt.Sprintf(UpdateEngineTemplate, cpdb.schema, CheckpointTableNameEngine)
s := common.SQLWithRetry{DB: cpdb.db, Logger: log.FromContext(taskCtx)}
return s.Transact(taskCtx, "update checkpoints", func(c context.Context, tx *sql.Tx) error {
chunkStmt, e := tx.PrepareContext(c, chunkQuery)
if e != nil {
return errors.Trace(e)
}
//nolint: errcheck
defer chunkStmt.Close()
rebaseStmt, e := tx.PrepareContext(c, rebaseQuery)
if e != nil {
return errors.Trace(e)
}
//nolint: errcheck
defer rebaseStmt.Close()
tableStatusStmt, e := tx.PrepareContext(c, tableStatusQuery)
if e != nil {
return errors.Trace(e)
}
//nolint: errcheck
defer tableStatusStmt.Close()
tableChecksumStmt, e := tx.PrepareContext(c, tableChecksumQuery)
if e != nil {
return errors.Trace(e)
}
//nolint: errcheck
defer tableChecksumStmt.Close()
engineStatusStmt, e := tx.PrepareContext(c, engineStatusQuery)
if e != nil {
return errors.Trace(e)
}
//nolint: errcheck
defer engineStatusStmt.Close()
for tableName, cpd := range checkpointDiffs {
if cpd.hasStatus {
if _, e := tableStatusStmt.ExecContext(c, cpd.status, tableName); e != nil {
return errors.Trace(e)
}
}
if cpd.hasRebase {
if _, e := rebaseStmt.ExecContext(c, cpd.allocBase, tableName); e != nil {
return errors.Trace(e)
}
}
if cpd.hasChecksum {
if _, e := tableChecksumStmt.ExecContext(c, cpd.checksum.SumSize(), cpd.checksum.SumKVS(), cpd.checksum.Sum(), tableName); e != nil {
return errors.Trace(e)
}
}
for engineID, engineDiff := range cpd.engines {
if engineDiff.hasStatus {
if _, e := engineStatusStmt.ExecContext(c, engineDiff.status, tableName, engineID); e != nil {
return errors.Trace(e)
}
}
for key, diff := range engineDiff.chunks {
columnPerm, err := json.Marshal(diff.columnPermutation)
if err != nil {
return errors.Trace(err)
}
if _, e := chunkStmt.ExecContext(
c,
diff.pos, diff.rowID, diff.checksum.SumSize(), diff.checksum.SumKVS(), diff.checksum.Sum(),
columnPerm, tableName, engineID, key.Path, key.Offset,
); e != nil {
return errors.Trace(e)
}
}
}
}
return nil
})
}
type FileCheckpointsDB struct {
lock sync.Mutex // we need to ensure only a thread can access to `checkpoints` at a time
checkpoints checkpointspb.CheckpointsModel
ctx context.Context
path string
fileName string
exStorage storage.ExternalStorage
}
func newFileCheckpointsDB(
ctx context.Context,
path string,
exStorage storage.ExternalStorage,
fileName string,
) (*FileCheckpointsDB, error) {
cpdb := &FileCheckpointsDB{
checkpoints: checkpointspb.CheckpointsModel{
TaskCheckpoint: &checkpointspb.TaskCheckpointModel{},
Checkpoints: map[string]*checkpointspb.TableCheckpointModel{},
},
ctx: ctx,
path: path,