-
Notifications
You must be signed in to change notification settings - Fork 5.9k
/
Copy pathlocal.go
1778 lines (1602 loc) · 55.3 KB
/
local.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 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,
// 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 local
import (
"bytes"
"context"
"fmt"
"io"
"math"
"net"
"os"
"path/filepath"
"strings"
"sync"
"time"
"github.com/cockroachdb/pebble"
"github.com/coreos/go-semver/semver"
"github.com/docker/go-units"
"github.com/google/uuid"
"github.com/pingcap/errors"
"github.com/pingcap/failpoint"
sst "github.com/pingcap/kvproto/pkg/import_sstpb"
"github.com/pingcap/kvproto/pkg/metapb"
"github.com/pingcap/tidb/br/pkg/lightning/backend"
"github.com/pingcap/tidb/br/pkg/lightning/backend/encode"
"github.com/pingcap/tidb/br/pkg/lightning/backend/kv"
"github.com/pingcap/tidb/br/pkg/lightning/common"
"github.com/pingcap/tidb/br/pkg/lightning/config"
"github.com/pingcap/tidb/br/pkg/lightning/errormanager"
"github.com/pingcap/tidb/br/pkg/lightning/glue"
"github.com/pingcap/tidb/br/pkg/lightning/log"
"github.com/pingcap/tidb/br/pkg/lightning/manual"
"github.com/pingcap/tidb/br/pkg/lightning/metric"
"github.com/pingcap/tidb/br/pkg/lightning/tikv"
"github.com/pingcap/tidb/br/pkg/logutil"
"github.com/pingcap/tidb/br/pkg/membuf"
"github.com/pingcap/tidb/br/pkg/pdutil"
"github.com/pingcap/tidb/br/pkg/restore/split"
"github.com/pingcap/tidb/br/pkg/utils"
"github.com/pingcap/tidb/br/pkg/version"
"github.com/pingcap/tidb/infoschema"
"github.com/pingcap/tidb/parser/model"
"github.com/pingcap/tidb/parser/mysql"
"github.com/pingcap/tidb/store/pdtypes"
"github.com/pingcap/tidb/table"
"github.com/pingcap/tidb/tablecodec"
"github.com/pingcap/tidb/types"
"github.com/pingcap/tidb/util/codec"
"github.com/pingcap/tidb/util/engine"
"github.com/pingcap/tidb/util/mathutil"
tikverror "github.com/tikv/client-go/v2/error"
"github.com/tikv/client-go/v2/oracle"
tikvclient "github.com/tikv/client-go/v2/tikv"
pd "github.com/tikv/pd/client"
"go.uber.org/atomic"
"go.uber.org/zap"
"golang.org/x/exp/slices"
"golang.org/x/sync/errgroup"
"golang.org/x/time/rate"
"google.golang.org/grpc"
"google.golang.org/grpc/backoff"
"google.golang.org/grpc/codes"
"google.golang.org/grpc/credentials"
"google.golang.org/grpc/credentials/insecure"
"google.golang.org/grpc/keepalive"
"google.golang.org/grpc/status"
)
const (
dialTimeout = 5 * time.Minute
maxRetryTimes = 5
defaultRetryBackoffTime = 3 * time.Second
// maxWriteAndIngestRetryTimes is the max retry times for write and ingest.
// A large retry times is for tolerating tikv cluster failures.
maxWriteAndIngestRetryTimes = 30
gRPCKeepAliveTime = 10 * time.Minute
gRPCKeepAliveTimeout = 5 * time.Minute
gRPCBackOffMaxDelay = 10 * time.Minute
// The max ranges count in a batch to split and scatter.
maxBatchSplitRanges = 4096
propRangeIndex = "tikv.range_index"
defaultPropSizeIndexDistance = 4 * units.MiB
defaultPropKeysIndexDistance = 40 * 1024
// the lower threshold of max open files for pebble db.
openFilesLowerThreshold = 128
duplicateDBName = "duplicates"
scanRegionLimit = 128
)
var (
// Local backend is compatible with TiDB [4.0.0, NextMajorVersion).
localMinTiDBVersion = *semver.New("4.0.0")
localMinTiKVVersion = *semver.New("4.0.0")
localMinPDVersion = *semver.New("4.0.0")
localMaxTiDBVersion = version.NextMajorVersion()
localMaxTiKVVersion = version.NextMajorVersion()
localMaxPDVersion = version.NextMajorVersion()
tiFlashMinVersion = *semver.New("4.0.5")
errorEngineClosed = errors.New("engine is closed")
maxRetryBackoffSecond = 30
)
// ImportClientFactory is factory to create new import client for specific store.
type ImportClientFactory interface {
Create(ctx context.Context, storeID uint64) (sst.ImportSSTClient, error)
Close()
}
type importClientFactoryImpl struct {
conns *common.GRPCConns
splitCli split.SplitClient
tls *common.TLS
tcpConcurrency int
compressionType config.CompressionType
}
func newImportClientFactoryImpl(
splitCli split.SplitClient,
tls *common.TLS,
tcpConcurrency int,
compressionType config.CompressionType,
) *importClientFactoryImpl {
return &importClientFactoryImpl{
conns: common.NewGRPCConns(),
splitCli: splitCli,
tls: tls,
tcpConcurrency: tcpConcurrency,
compressionType: compressionType,
}
}
func (f *importClientFactoryImpl) makeConn(ctx context.Context, storeID uint64) (*grpc.ClientConn, error) {
store, err := f.splitCli.GetStore(ctx, storeID)
if err != nil {
return nil, errors.Trace(err)
}
var opts []grpc.DialOption
if f.tls.TLSConfig() != nil {
opts = append(opts, grpc.WithTransportCredentials(credentials.NewTLS(f.tls.TLSConfig())))
} else {
opts = append(opts, grpc.WithTransportCredentials(insecure.NewCredentials()))
}
ctx, cancel := context.WithTimeout(ctx, dialTimeout)
defer cancel()
bfConf := backoff.DefaultConfig
bfConf.MaxDelay = gRPCBackOffMaxDelay
// we should use peer address for tiflash. for tikv, peer address is empty
addr := store.GetPeerAddress()
if addr == "" {
addr = store.GetAddress()
}
opts = append(opts,
grpc.WithConnectParams(grpc.ConnectParams{Backoff: bfConf}),
grpc.WithKeepaliveParams(keepalive.ClientParameters{
Time: gRPCKeepAliveTime,
Timeout: gRPCKeepAliveTimeout,
PermitWithoutStream: true,
}),
)
switch f.compressionType {
case config.CompressionNone:
// do nothing
case config.CompressionGzip:
// Use custom compressor/decompressor to speed up compression/decompression.
// Note that here we don't use grpc.UseCompressor option although it's the recommended way.
// Because gprc-go uses a global registry to store compressor/decompressor, we can't make sure
// the compressor/decompressor is not registered by other components.
opts = append(opts, grpc.WithCompressor(&gzipCompressor{}), grpc.WithDecompressor(&gzipDecompressor{}))
default:
return nil, common.ErrInvalidConfig.GenWithStack("unsupported compression type %s", f.compressionType)
}
failpoint.Inject("LoggingImportBytes", func() {
opts = append(opts, grpc.WithContextDialer(func(ctx context.Context, target string) (net.Conn, error) {
conn, err := (&net.Dialer{}).DialContext(ctx, "tcp", target)
if err != nil {
return nil, err
}
return &loggingConn{Conn: conn}, nil
}))
})
conn, err := grpc.DialContext(ctx, addr, opts...)
if err != nil {
return nil, errors.Trace(err)
}
return conn, nil
}
func (f *importClientFactoryImpl) getGrpcConn(ctx context.Context, storeID uint64) (*grpc.ClientConn, error) {
return f.conns.GetGrpcConn(ctx, storeID, f.tcpConcurrency,
func(ctx context.Context) (*grpc.ClientConn, error) {
return f.makeConn(ctx, storeID)
})
}
func (f *importClientFactoryImpl) Create(ctx context.Context, storeID uint64) (sst.ImportSSTClient, error) {
conn, err := f.getGrpcConn(ctx, storeID)
if err != nil {
return nil, err
}
return sst.NewImportSSTClient(conn), nil
}
func (f *importClientFactoryImpl) Close() {
f.conns.Close()
}
type loggingConn struct {
net.Conn
}
func (c loggingConn) Write(b []byte) (int, error) {
log.L().Debug("import write", zap.Int("bytes", len(b)))
return c.Conn.Write(b)
}
// Range record start and end key for localStoreDir.DB
// so we can write it to tikv in streaming
type Range struct {
start []byte
end []byte // end is always exclusive except import_sstpb.SSTMeta
}
type encodingBuilder struct {
metrics *metric.Metrics
}
// NewEncodingBuilder creates an KVEncodingBuilder with local backend implementation.
func NewEncodingBuilder(ctx context.Context) encode.EncodingBuilder {
result := new(encodingBuilder)
if m, ok := metric.FromContext(ctx); ok {
result.metrics = m
}
return result
}
// NewEncoder creates a KV encoder.
// It implements the `backend.EncodingBuilder` interface.
func (b *encodingBuilder) NewEncoder(ctx context.Context, config *encode.EncodingConfig) (encode.Encoder, error) {
return kv.NewTableKVEncoder(config, b.metrics)
}
// MakeEmptyRows creates an empty KV rows.
// It implements the `backend.EncodingBuilder` interface.
func (b *encodingBuilder) MakeEmptyRows() encode.Rows {
return kv.MakeRowsFromKvPairs(nil)
}
type targetInfoGetter struct {
tls *common.TLS
targetDBGlue glue.Glue
pdAddr string
}
// NewTargetInfoGetter creates an TargetInfoGetter with local backend implementation.
func NewTargetInfoGetter(tls *common.TLS, g glue.Glue, pdAddr string) backend.TargetInfoGetter {
return &targetInfoGetter{
tls: tls,
targetDBGlue: g,
pdAddr: pdAddr,
}
}
// FetchRemoteTableModels obtains the models of all tables given the schema name.
// It implements the `TargetInfoGetter` interface.
func (g *targetInfoGetter) FetchRemoteTableModels(ctx context.Context, schemaName string) ([]*model.TableInfo, error) {
return tikv.FetchRemoteTableModelsFromTLS(ctx, g.tls, schemaName)
}
// CheckRequirements performs the check whether the backend satisfies the version requirements.
// It implements the `TargetInfoGetter` interface.
func (g *targetInfoGetter) CheckRequirements(ctx context.Context, checkCtx *backend.CheckCtx) error {
// TODO: support lightning via SQL
db, _ := g.targetDBGlue.GetDB()
versionStr, err := version.FetchVersion(ctx, db)
if err != nil {
return errors.Trace(err)
}
if err := checkTiDBVersion(ctx, versionStr, localMinTiDBVersion, localMaxTiDBVersion); err != nil {
return err
}
if err := tikv.CheckPDVersion(ctx, g.tls, g.pdAddr, localMinPDVersion, localMaxPDVersion); err != nil {
return err
}
if err := tikv.CheckTiKVVersion(ctx, g.tls, g.pdAddr, localMinTiKVVersion, localMaxTiKVVersion); err != nil {
return err
}
serverInfo := version.ParseServerInfo(versionStr)
return checkTiFlashVersion(ctx, g.targetDBGlue, checkCtx, *serverInfo.ServerVersion)
}
func checkTiDBVersion(_ context.Context, versionStr string, requiredMinVersion, requiredMaxVersion semver.Version) error {
return version.CheckTiDBVersion(versionStr, requiredMinVersion, requiredMaxVersion)
}
var tiFlashReplicaQuery = "SELECT TABLE_SCHEMA, TABLE_NAME FROM information_schema.TIFLASH_REPLICA WHERE REPLICA_COUNT > 0;"
// TiFlashReplicaQueryForTest is only used for tests.
var TiFlashReplicaQueryForTest = tiFlashReplicaQuery
type tblName struct {
schema string
name string
}
type tblNames []tblName
func (t tblNames) String() string {
var b strings.Builder
b.WriteByte('[')
for i, n := range t {
if i > 0 {
b.WriteString(", ")
}
b.WriteString(common.UniqueTable(n.schema, n.name))
}
b.WriteByte(']')
return b.String()
}
// CheckTiFlashVersionForTest is only used for tests.
var CheckTiFlashVersionForTest = checkTiFlashVersion
// check TiFlash replicas.
// local backend doesn't support TiFlash before tidb v4.0.5
func checkTiFlashVersion(ctx context.Context, g glue.Glue, checkCtx *backend.CheckCtx, tidbVersion semver.Version) error {
if tidbVersion.Compare(tiFlashMinVersion) >= 0 {
return nil
}
res, err := g.GetSQLExecutor().QueryStringsWithLog(ctx, tiFlashReplicaQuery, "fetch tiflash replica info", log.FromContext(ctx))
if err != nil {
return errors.Annotate(err, "fetch tiflash replica info failed")
}
tiFlashTablesMap := make(map[tblName]struct{}, len(res))
for _, tblInfo := range res {
name := tblName{schema: tblInfo[0], name: tblInfo[1]}
tiFlashTablesMap[name] = struct{}{}
}
tiFlashTables := make(tblNames, 0)
for _, dbMeta := range checkCtx.DBMetas {
for _, tblMeta := range dbMeta.Tables {
if len(tblMeta.DataFiles) == 0 {
continue
}
name := tblName{schema: tblMeta.DB, name: tblMeta.Name}
if _, ok := tiFlashTablesMap[name]; ok {
tiFlashTables = append(tiFlashTables, name)
}
}
}
if len(tiFlashTables) > 0 {
helpInfo := "Please either upgrade TiDB to version >= 4.0.5 or add TiFlash replica after load data."
return errors.Errorf("lightning local backend doesn't support TiFlash in this TiDB version. conflict tables: %s. "+helpInfo, tiFlashTables)
}
return nil
}
type local struct {
engines sync.Map // sync version of map[uuid.UUID]*Engine
pdCtl *pdutil.PdController
splitCli split.SplitClient
tikvCli *tikvclient.KVStore
tls *common.TLS
pdAddr string
regionSizeGetter TableRegionSizeGetter
tikvCodec tikvclient.Codec
localStoreDir string
workerConcurrency int
kvWriteBatchSize int
checkpointEnabled bool
dupeConcurrency int
maxOpenFiles int
engineMemCacheSize int
localWriterMemCacheSize int64
supportMultiIngest bool
shouldCheckTiKV bool
duplicateDetection bool
duplicateDetectOpt dupDetectOpt
duplicateDB *pebble.DB
keyAdapter KeyAdapter
errorMgr *errormanager.ErrorManager
importClientFactory ImportClientFactory
bufferPool *membuf.Pool
metrics *metric.Metrics
writeLimiter StoreWriteLimiter
logger log.Logger
// When TiKV is in normal mode, ingesting too many SSTs will cause TiKV write stall.
// To avoid this, we should check write stall before ingesting SSTs. Note that, we
// must check both leader node and followers in client side, because followers will
// not check write stall as long as ingest command is accepted by leader.
shouldCheckWriteStall bool
}
func openDuplicateDB(storeDir string) (*pebble.DB, error) {
dbPath := filepath.Join(storeDir, duplicateDBName)
// TODO: Optimize the opts for better write.
opts := &pebble.Options{
TablePropertyCollectors: []func() pebble.TablePropertyCollector{
newRangePropertiesCollector,
},
}
return pebble.Open(dbPath, opts)
}
var (
// RunInTest indicates whether the current process is running in test.
RunInTest bool
// LastAlloc is the last ID allocator.
LastAlloc manual.Allocator
)
// NewLocalBackend creates new connections to tikv.
func NewLocalBackend(
ctx context.Context,
tls *common.TLS,
cfg *config.Config,
regionSizeGetter TableRegionSizeGetter,
maxOpenFiles int,
errorMgr *errormanager.ErrorManager,
keyspaceName string,
) (backend.Backend, error) {
localFile := cfg.TikvImporter.SortedKVDir
rangeConcurrency := cfg.TikvImporter.RangeConcurrency
pdCtl, err := pdutil.NewPdController(ctx, cfg.TiDB.PdAddr, tls.TLSConfig(), tls.ToPDSecurityOption())
if err != nil {
return backend.MakeBackend(nil), common.NormalizeOrWrapErr(common.ErrCreatePDClient, err)
}
splitCli := split.NewSplitClient(pdCtl.GetPDClient(), tls.TLSConfig(), false)
shouldCreate := true
if cfg.Checkpoint.Enable {
if info, err := os.Stat(localFile); err != nil {
if !os.IsNotExist(err) {
return backend.MakeBackend(nil), err
}
} else if info.IsDir() {
shouldCreate = false
}
}
if shouldCreate {
err = os.Mkdir(localFile, 0o700)
if err != nil {
return backend.MakeBackend(nil), common.ErrInvalidSortedKVDir.Wrap(err).GenWithStackByArgs(localFile)
}
}
var duplicateDB *pebble.DB
if cfg.TikvImporter.DuplicateResolution != config.DupeResAlgNone {
duplicateDB, err = openDuplicateDB(localFile)
if err != nil {
return backend.MakeBackend(nil), common.ErrOpenDuplicateDB.Wrap(err).GenWithStackByArgs()
}
}
// The following copies tikv.NewTxnClient without creating yet another pdClient.
spkv, err := tikvclient.NewEtcdSafePointKV(strings.Split(cfg.TiDB.PdAddr, ","), tls.TLSConfig())
if err != nil {
return backend.MakeBackend(nil), common.ErrCreateKVClient.Wrap(err).GenWithStackByArgs()
}
var pdCliForTiKV *tikvclient.CodecPDClient
if keyspaceName == "" {
pdCliForTiKV = tikvclient.NewCodecPDClient(tikvclient.ModeTxn, pdCtl.GetPDClient())
} else {
pdCliForTiKV, err = tikvclient.NewCodecPDClientWithKeyspace(tikvclient.ModeTxn, pdCtl.GetPDClient(), keyspaceName)
if err != nil {
return backend.MakeBackend(nil), common.ErrCreatePDClient.Wrap(err).GenWithStackByArgs()
}
}
tikvCodec := pdCliForTiKV.GetCodec()
rpcCli := tikvclient.NewRPCClient(tikvclient.WithSecurity(tls.ToTiKVSecurityConfig()), tikvclient.WithCodec(tikvCodec))
tikvCli, err := tikvclient.NewKVStore("lightning-local-backend", pdCliForTiKV, spkv, rpcCli)
if err != nil {
return backend.MakeBackend(nil), common.ErrCreateKVClient.Wrap(err).GenWithStackByArgs()
}
importClientFactory := newImportClientFactoryImpl(splitCli, tls, rangeConcurrency, cfg.TikvImporter.CompressKVPairs)
duplicateDetection := cfg.TikvImporter.DuplicateResolution != config.DupeResAlgNone
keyAdapter := KeyAdapter(noopKeyAdapter{})
if duplicateDetection {
keyAdapter = dupDetectKeyAdapter{}
}
var writeLimiter StoreWriteLimiter
if cfg.TikvImporter.StoreWriteBWLimit > 0 {
writeLimiter = newStoreWriteLimiter(int(cfg.TikvImporter.StoreWriteBWLimit))
} else {
writeLimiter = noopStoreWriteLimiter{}
}
alloc := manual.Allocator{}
if RunInTest {
alloc.RefCnt = new(atomic.Int64)
LastAlloc = alloc
}
local := &local{
engines: sync.Map{},
pdCtl: pdCtl,
splitCli: splitCli,
tikvCli: tikvCli,
tls: tls,
pdAddr: cfg.TiDB.PdAddr,
regionSizeGetter: regionSizeGetter,
tikvCodec: tikvCodec,
localStoreDir: localFile,
workerConcurrency: rangeConcurrency * 2,
dupeConcurrency: rangeConcurrency * 2,
kvWriteBatchSize: cfg.TikvImporter.SendKVPairs,
checkpointEnabled: cfg.Checkpoint.Enable,
maxOpenFiles: mathutil.Max(maxOpenFiles, openFilesLowerThreshold),
engineMemCacheSize: int(cfg.TikvImporter.EngineMemCacheSize),
localWriterMemCacheSize: int64(cfg.TikvImporter.LocalWriterMemCacheSize),
duplicateDetection: duplicateDetection,
duplicateDetectOpt: dupDetectOpt{duplicateDetection && cfg.TikvImporter.DuplicateResolution == config.DupeResAlgErr},
shouldCheckTiKV: cfg.App.CheckRequirements,
duplicateDB: duplicateDB,
keyAdapter: keyAdapter,
errorMgr: errorMgr,
importClientFactory: importClientFactory,
bufferPool: membuf.NewPool(membuf.WithAllocator(alloc)),
writeLimiter: writeLimiter,
logger: log.FromContext(ctx),
shouldCheckWriteStall: cfg.Cron.SwitchMode.Duration == 0,
}
if m, ok := metric.FromContext(ctx); ok {
local.metrics = m
}
if err = local.checkMultiIngestSupport(ctx); err != nil {
return backend.MakeBackend(nil), common.ErrCheckMultiIngest.Wrap(err).GenWithStackByArgs()
}
return backend.MakeBackend(local), nil
}
func (local *local) TotalMemoryConsume() int64 {
var memConsume int64 = 0
local.engines.Range(func(k, v interface{}) bool {
e := v.(*Engine)
if e != nil {
memConsume += e.TotalMemorySize()
}
return true
})
return memConsume + local.bufferPool.TotalSize()
}
func (local *local) checkMultiIngestSupport(ctx context.Context) error {
stores, err := local.pdCtl.GetPDClient().GetAllStores(ctx, pd.WithExcludeTombstone())
if err != nil {
return errors.Trace(err)
}
hasTiFlash := false
for _, s := range stores {
if s.State == metapb.StoreState_Up && engine.IsTiFlash(s) {
hasTiFlash = true
break
}
}
for _, s := range stores {
// skip stores that are not online
if s.State != metapb.StoreState_Up || engine.IsTiFlash(s) {
continue
}
var err error
for i := 0; i < maxRetryTimes; i++ {
if i > 0 {
select {
case <-time.After(100 * time.Millisecond):
case <-ctx.Done():
return ctx.Err()
}
}
client, err1 := local.getImportClient(ctx, s.Id)
if err1 != nil {
err = err1
log.FromContext(ctx).Warn("get import client failed", zap.Error(err), zap.String("store", s.Address))
continue
}
_, err = client.MultiIngest(ctx, &sst.MultiIngestRequest{})
if err == nil {
break
}
if st, ok := status.FromError(err); ok {
if st.Code() == codes.Unimplemented {
log.FromContext(ctx).Info("multi ingest not support", zap.Any("unsupported store", s))
local.supportMultiIngest = false
return nil
}
}
log.FromContext(ctx).Warn("check multi ingest support failed", zap.Error(err), zap.String("store", s.Address),
zap.Int("retry", i))
}
if err != nil {
// if the cluster contains no TiFlash store, we don't need the multi-ingest feature,
// so in this condition, downgrade the logic instead of return an error.
if hasTiFlash {
return errors.Trace(err)
}
log.FromContext(ctx).Warn("check multi failed all retry, fallback to false", log.ShortError(err))
local.supportMultiIngest = false
return nil
}
}
local.supportMultiIngest = true
log.FromContext(ctx).Info("multi ingest support")
return nil
}
// rlock read locks a local file and returns the Engine instance if it exists.
func (local *local) rLockEngine(engineId uuid.UUID) *Engine {
if e, ok := local.engines.Load(engineId); ok {
engine := e.(*Engine)
engine.rLock()
return engine
}
return nil
}
// lock locks a local file and returns the Engine instance if it exists.
func (local *local) lockEngine(engineID uuid.UUID, state importMutexState) *Engine {
if e, ok := local.engines.Load(engineID); ok {
engine := e.(*Engine)
engine.lock(state)
return engine
}
return nil
}
// tryRLockAllEngines tries to read lock all engines, return all `Engine`s that are successfully locked.
func (local *local) tryRLockAllEngines() []*Engine {
var allEngines []*Engine
local.engines.Range(func(k, v interface{}) bool {
engine := v.(*Engine)
// skip closed engine
if engine.tryRLock() {
if !engine.closed.Load() {
allEngines = append(allEngines, engine)
} else {
engine.rUnlock()
}
}
return true
})
return allEngines
}
// lockAllEnginesUnless tries to lock all engines, unless those which are already locked in the
// state given by ignoreStateMask. Returns the list of locked engines.
func (local *local) lockAllEnginesUnless(newState, ignoreStateMask importMutexState) []*Engine {
var allEngines []*Engine
local.engines.Range(func(k, v interface{}) bool {
engine := v.(*Engine)
if engine.lockUnless(newState, ignoreStateMask) {
allEngines = append(allEngines, engine)
}
return true
})
return allEngines
}
// Close the local backend.
func (local *local) Close() {
allEngines := local.lockAllEnginesUnless(importMutexStateClose, 0)
local.engines = sync.Map{}
for _, engine := range allEngines {
_ = engine.Close()
engine.unlock()
}
local.importClientFactory.Close()
local.bufferPool.Destroy()
if local.duplicateDB != nil {
// Check if there are duplicates that are not collected.
iter := local.duplicateDB.NewIter(&pebble.IterOptions{})
hasDuplicates := iter.First()
allIsWell := true
if err := iter.Error(); err != nil {
local.logger.Warn("iterate duplicate db failed", zap.Error(err))
allIsWell = false
}
if err := iter.Close(); err != nil {
local.logger.Warn("close duplicate db iter failed", zap.Error(err))
allIsWell = false
}
if err := local.duplicateDB.Close(); err != nil {
local.logger.Warn("close duplicate db failed", zap.Error(err))
allIsWell = false
}
// If checkpoint is disabled, or we don't detect any duplicate, then this duplicate
// db dir will be useless, so we clean up this dir.
if allIsWell && (!local.checkpointEnabled || !hasDuplicates) {
if err := os.RemoveAll(filepath.Join(local.localStoreDir, duplicateDBName)); err != nil {
local.logger.Warn("remove duplicate db file failed", zap.Error(err))
}
}
local.duplicateDB = nil
}
// if checkpoint is disable or we finish load all data successfully, then files in this
// dir will be useless, so we clean up this dir and all files in it.
if !local.checkpointEnabled || common.IsEmptyDir(local.localStoreDir) {
err := os.RemoveAll(local.localStoreDir)
if err != nil {
local.logger.Warn("remove local db file failed", zap.Error(err))
}
}
_ = local.tikvCli.Close()
local.pdCtl.Close()
}
// FlushEngine ensure the written data is saved successfully, to make sure no data lose after restart
func (local *local) FlushEngine(ctx context.Context, engineID uuid.UUID) error {
engine := local.rLockEngine(engineID)
// the engine cannot be deleted after while we've acquired the lock identified by UUID.
if engine == nil {
return errors.Errorf("engine '%s' not found", engineID)
}
defer engine.rUnlock()
if engine.closed.Load() {
return nil
}
return engine.flushEngineWithoutLock(ctx)
}
func (local *local) FlushAllEngines(parentCtx context.Context) (err error) {
allEngines := local.tryRLockAllEngines()
defer func() {
for _, engine := range allEngines {
engine.rUnlock()
}
}()
eg, ctx := errgroup.WithContext(parentCtx)
for _, engine := range allEngines {
e := engine
eg.Go(func() error {
return e.flushEngineWithoutLock(ctx)
})
}
return eg.Wait()
}
func (local *local) RetryImportDelay() time.Duration {
return defaultRetryBackoffTime
}
func (local *local) ShouldPostProcess() bool {
return true
}
func (local *local) openEngineDB(engineUUID uuid.UUID, readOnly bool) (*pebble.DB, error) {
opt := &pebble.Options{
MemTableSize: local.engineMemCacheSize,
// the default threshold value may cause write stall.
MemTableStopWritesThreshold: 8,
MaxConcurrentCompactions: 16,
// set threshold to half of the max open files to avoid trigger compaction
L0CompactionThreshold: math.MaxInt32,
L0StopWritesThreshold: math.MaxInt32,
LBaseMaxBytes: 16 * units.TiB,
MaxOpenFiles: local.maxOpenFiles,
DisableWAL: true,
ReadOnly: readOnly,
TablePropertyCollectors: []func() pebble.TablePropertyCollector{
newRangePropertiesCollector,
},
}
// set level target file size to avoid pebble auto triggering compaction that split ingest SST files into small SST.
opt.Levels = []pebble.LevelOptions{
{
TargetFileSize: 16 * units.GiB,
},
}
dbPath := filepath.Join(local.localStoreDir, engineUUID.String())
db, err := pebble.Open(dbPath, opt)
return db, errors.Trace(err)
}
// OpenEngine must be called with holding mutex of Engine.
func (local *local) OpenEngine(ctx context.Context, cfg *backend.EngineConfig, engineUUID uuid.UUID) error {
db, err := local.openEngineDB(engineUUID, false)
if err != nil {
return err
}
sstDir := engineSSTDir(local.localStoreDir, engineUUID)
if err := os.RemoveAll(sstDir); err != nil {
return errors.Trace(err)
}
if !common.IsDirExists(sstDir) {
if err := os.Mkdir(sstDir, 0o750); err != nil {
return errors.Trace(err)
}
}
engineCtx, cancel := context.WithCancel(ctx)
e, _ := local.engines.LoadOrStore(engineUUID, &Engine{
UUID: engineUUID,
sstDir: sstDir,
sstMetasChan: make(chan metaOrFlush, 64),
ctx: engineCtx,
cancel: cancel,
config: cfg.Local,
tableInfo: cfg.TableInfo,
duplicateDetection: local.duplicateDetection,
dupDetectOpt: local.duplicateDetectOpt,
duplicateDB: local.duplicateDB,
errorMgr: local.errorMgr,
keyAdapter: local.keyAdapter,
logger: log.FromContext(ctx),
})
engine := e.(*Engine)
engine.db = db
engine.sstIngester = dbSSTIngester{e: engine}
if err = engine.loadEngineMeta(); err != nil {
return errors.Trace(err)
}
if err = local.allocateTSIfNotExists(ctx, engine); err != nil {
return errors.Trace(err)
}
engine.wg.Add(1)
go engine.ingestSSTLoop()
return nil
}
func (local *local) allocateTSIfNotExists(ctx context.Context, engine *Engine) error {
if engine.TS > 0 {
return nil
}
physical, logical, err := local.pdCtl.GetPDClient().GetTS(ctx)
if err != nil {
return err
}
ts := oracle.ComposeTS(physical, logical)
engine.TS = ts
return engine.saveEngineMeta()
}
// CloseEngine closes backend engine by uuid.
func (local *local) CloseEngine(ctx context.Context, cfg *backend.EngineConfig, engineUUID uuid.UUID) error {
// flush mem table to storage, to free memory,
// ask others' advise, looks like unnecessary, but with this we can control memory precisely.
engineI, ok := local.engines.Load(engineUUID)
if !ok {
// recovery mode, we should reopen this engine file
db, err := local.openEngineDB(engineUUID, true)
if err != nil {
return err
}
engine := &Engine{
UUID: engineUUID,
db: db,
sstMetasChan: make(chan metaOrFlush),
tableInfo: cfg.TableInfo,
keyAdapter: local.keyAdapter,
duplicateDetection: local.duplicateDetection,
dupDetectOpt: local.duplicateDetectOpt,
duplicateDB: local.duplicateDB,
errorMgr: local.errorMgr,
logger: log.FromContext(ctx),
}
engine.sstIngester = dbSSTIngester{e: engine}
if err = engine.loadEngineMeta(); err != nil {
return err
}
local.engines.Store(engineUUID, engine)
return nil
}
engine := engineI.(*Engine)
engine.rLock()
if engine.closed.Load() {
engine.rUnlock()
return nil
}
err := engine.flushEngineWithoutLock(ctx)
engine.rUnlock()
// use mutex to make sure we won't close sstMetasChan while other routines
// trying to do flush.
engine.lock(importMutexStateClose)
engine.closed.Store(true)
close(engine.sstMetasChan)
engine.unlock()
if err != nil {
return errors.Trace(err)
}
engine.wg.Wait()
return engine.ingestErr.Get()
}
func (local *local) getImportClient(ctx context.Context, storeID uint64) (sst.ImportSSTClient, error) {
return local.importClientFactory.Create(ctx, storeID)
}
func splitRangeBySizeProps(fullRange Range, sizeProps *sizeProperties, sizeLimit int64, keysLimit int64) []Range {
ranges := make([]Range, 0, sizeProps.totalSize/uint64(sizeLimit))
curSize := uint64(0)
curKeys := uint64(0)
curKey := fullRange.start
sizeProps.iter(func(p *rangeProperty) bool {
if bytes.Compare(p.Key, curKey) <= 0 {
return true
}
if bytes.Compare(p.Key, fullRange.end) > 0 {
return false
}
curSize += p.Size
curKeys += p.Keys
if int64(curSize) >= sizeLimit || int64(curKeys) >= keysLimit {
ranges = append(ranges, Range{start: curKey, end: p.Key})
curKey = p.Key
curSize = 0
curKeys = 0
}
return true
})
if bytes.Compare(curKey, fullRange.end) < 0 {
// If the remaining range is too small, append it to last range.
if len(ranges) > 0 && curKeys == 0 {
ranges[len(ranges)-1].end = fullRange.end
} else {
ranges = append(ranges, Range{start: curKey, end: fullRange.end})
}
}
return ranges
}
func (local *local) readAndSplitIntoRange(
ctx context.Context,
engine *Engine,
sizeLimit int64,
keysLimit int64,
) ([]Range, error) {
firstKey, lastKey, err := engine.getFirstAndLastKey(nil, nil)
if err != nil {
return nil, err
}
if firstKey == nil {
return nil, errors.New("could not find first pair")
}
endKey := nextKey(lastKey)
engineFileTotalSize := engine.TotalSize.Load()
engineFileLength := engine.Length.Load()
if engineFileTotalSize <= sizeLimit && engineFileLength <= keysLimit {
ranges := []Range{{start: firstKey, end: endKey}}
return ranges, nil
}