github.com/pingcap/tidb-lightning@v5.0.0-rc.0.20210428090220-84b649866577+incompatible/lightning/backend/local.go (about) 1 // Copyright 2020 PingCAP, Inc. 2 // 3 // Licensed under the Apache License, Version 2.0 (the "License"); 4 // you may not use this file except in compliance with the License. 5 // You may obtain a copy of the License at 6 // 7 // http://www.apache.org/licenses/LICENSE-2.0 8 // 9 // Unless required by applicable law or agreed to in writing, software 10 // distributed under the License is distributed on an "AS IS" BASIS, 11 // See the License for the specific language governing permissions and 12 // limitations under the License. 13 14 package backend 15 16 import ( 17 "bytes" 18 "context" 19 "encoding/binary" 20 "encoding/json" 21 "io" 22 "os" 23 "path/filepath" 24 "sort" 25 "strings" 26 "sync" 27 "time" 28 29 "github.com/cockroachdb/pebble" 30 "github.com/cockroachdb/pebble/sstable" 31 "github.com/coreos/go-semver/semver" 32 "github.com/google/btree" 33 "github.com/google/uuid" 34 split "github.com/pingcap/br/pkg/restore" 35 "github.com/pingcap/br/pkg/utils" 36 "github.com/pingcap/errors" 37 "github.com/pingcap/failpoint" 38 "github.com/pingcap/kvproto/pkg/errorpb" 39 sst "github.com/pingcap/kvproto/pkg/import_sstpb" 40 "github.com/pingcap/kvproto/pkg/kvrpcpb" 41 "github.com/pingcap/kvproto/pkg/metapb" 42 "github.com/pingcap/parser/model" 43 "github.com/pingcap/tidb/table" 44 "github.com/pingcap/tidb/tablecodec" 45 "github.com/pingcap/tidb/util/codec" 46 "github.com/pingcap/tidb/util/hack" 47 pd "github.com/tikv/pd/client" 48 "go.uber.org/atomic" 49 "go.uber.org/multierr" 50 "go.uber.org/zap" 51 "golang.org/x/sync/errgroup" 52 "google.golang.org/grpc" 53 "google.golang.org/grpc/backoff" 54 "google.golang.org/grpc/credentials" 55 "google.golang.org/grpc/keepalive" 56 57 "github.com/pingcap/tidb-lightning/lightning/common" 58 "github.com/pingcap/tidb-lightning/lightning/config" 59 "github.com/pingcap/tidb-lightning/lightning/glue" 60 "github.com/pingcap/tidb-lightning/lightning/log" 61 "github.com/pingcap/tidb-lightning/lightning/manual" 62 "github.com/pingcap/tidb-lightning/lightning/metric" 63 "github.com/pingcap/tidb-lightning/lightning/worker" 64 ) 65 66 const ( 67 dialTimeout = 5 * time.Second 68 bigValueSize = 1 << 16 // 64K 69 70 gRPCKeepAliveTime = 10 * time.Second 71 gRPCKeepAliveTimeout = 3 * time.Second 72 gRPCBackOffMaxDelay = 3 * time.Second 73 74 LocalMemoryTableSize = config.LocalMemoryTableSize 75 76 // See: https://github.com/tikv/tikv/blob/e030a0aae9622f3774df89c62f21b2171a72a69e/etc/config-template.toml#L360 77 regionMaxKeyCount = 1_440_000 78 79 propRangeIndex = "tikv.range_index" 80 81 defaultPropSizeIndexDistance = 4 * 1024 * 1024 // 4MB 82 defaultPropKeysIndexDistance = 40 * 1024 83 84 // the lower threshold of max open files for pebble db. 85 openFilesLowerThreshold = 128 86 ) 87 88 var ( 89 localMinTiDBVersion = *semver.New("4.0.0") 90 localMinTiKVVersion = *semver.New("4.0.0") 91 localMinPDVersion = *semver.New("4.0.0") 92 ) 93 94 var ( 95 engineMetaKey = []byte{0, 'm', 'e', 't', 'a'} 96 normalIterStartKey = []byte{1} 97 ) 98 99 // Range record start and end key for localStoreDir.DB 100 // so we can write it to tikv in streaming 101 type Range struct { 102 start []byte 103 end []byte 104 length int 105 } 106 107 // localFileMeta contains some field that is necessary to continue the engine restore/import process. 108 // These field should be written to disk when we update chunk checkpoint 109 type localFileMeta struct { 110 Ts uint64 `json:"ts"` 111 // Length is the number of KV pairs stored by the engine. 112 Length atomic.Int64 `json:"length"` 113 // TotalSize is the total pre-compressed KV byte size stored by engine. 114 TotalSize atomic.Int64 `json:"total_size"` 115 } 116 117 type importMutexState uint32 118 119 const ( 120 importMutexStateImport importMutexState = 1 << iota 121 importMutexStateFlush 122 importMutexStateClose 123 importMutexStateLocalIngest 124 ) 125 126 type LocalFile struct { 127 localFileMeta 128 db *pebble.DB 129 Uuid uuid.UUID 130 localWriters sync.Map 131 132 // isImportingAtomic is an atomic variable indicating whether the importMutex has been locked. 133 // This should not be used as a "spin lock" indicator. 134 isImportingAtomic atomic.Uint32 135 mutex sync.Mutex 136 } 137 138 func (e *LocalFile) Close() error { 139 log.L().Debug("closing local engine", zap.Stringer("engine", e.Uuid), zap.Stack("stack")) 140 if e.db == nil { 141 return nil 142 } 143 err := errors.Trace(e.db.Close()) 144 e.db = nil 145 return err 146 } 147 148 // Cleanup remove meta and db files 149 func (e *LocalFile) Cleanup(dataDir string) error { 150 dbPath := filepath.Join(dataDir, e.Uuid.String()) 151 return os.RemoveAll(dbPath) 152 } 153 154 // Exist checks if db folder existing (meta sometimes won't flush before lightning exit) 155 func (e *LocalFile) Exist(dataDir string) error { 156 dbPath := filepath.Join(dataDir, e.Uuid.String()) 157 if _, err := os.Stat(dbPath); err != nil { 158 return err 159 } 160 return nil 161 } 162 163 func (e *LocalFile) getSizeProperties() (*sizeProperties, error) { 164 sstables, err := e.db.SSTables(pebble.WithProperties()) 165 if err != nil { 166 log.L().Warn("get table properties failed", zap.Stringer("engine", e.Uuid), log.ShortError(err)) 167 return nil, errors.Trace(err) 168 } 169 170 sizeProps := newSizeProperties() 171 for _, level := range sstables { 172 for _, info := range level { 173 if prop, ok := info.Properties.UserProperties[propRangeIndex]; ok { 174 data := hack.Slice(prop) 175 rangeProps, err := decodeRangeProperties(data) 176 if err != nil { 177 log.L().Warn("decodeRangeProperties failed", zap.Stringer("engine", e.Uuid), 178 zap.Stringer("fileNum", info.FileNum), log.ShortError(err)) 179 return nil, errors.Trace(err) 180 } 181 182 sizeProps.addAll(rangeProps) 183 } 184 } 185 } 186 187 return sizeProps, nil 188 } 189 190 func (e *LocalFile) isLocked() bool { 191 return e.isImportingAtomic.Load() != 0 192 } 193 194 func (e *LocalFile) getEngineFileSize() EngineFileSize { 195 metrics := e.db.Metrics() 196 total := metrics.Total() 197 var memSize int64 198 e.localWriters.Range(func(k, v interface{}) bool { 199 w := k.(*LocalWriter) 200 memSize += w.writeBatch.totalSize 201 if w.writer != nil { 202 total.Size += int64(w.writer.writer.EstimatedSize()) 203 } 204 return true 205 }) 206 207 return EngineFileSize{ 208 UUID: e.Uuid, 209 DiskSize: total.Size, 210 MemSize: memSize, 211 IsImporting: e.isLocked(), 212 } 213 } 214 215 // lock locks the local file for importing. 216 func (e *LocalFile) lock(state importMutexState) { 217 e.mutex.Lock() 218 e.isImportingAtomic.Store(uint32(state)) 219 } 220 221 // lockUnless tries to lock the local file unless it is already locked into the state given by 222 // ignoreStateMask. Returns whether the lock is successful. 223 func (e *LocalFile) lockUnless(newState, ignoreStateMask importMutexState) bool { 224 curState := e.isImportingAtomic.Load() 225 if curState&uint32(ignoreStateMask) != 0 { 226 return false 227 } 228 e.lock(newState) 229 return true 230 } 231 232 func (e *LocalFile) unlock() { 233 if e == nil { 234 return 235 } 236 e.isImportingAtomic.Store(0) 237 e.mutex.Unlock() 238 } 239 240 func (e *LocalFile) flushLocalWriters(parentCtx context.Context) error { 241 eg, ctx := errgroup.WithContext(parentCtx) 242 e.localWriters.Range(func(k, v interface{}) bool { 243 eg.Go(func() error { 244 w := k.(*LocalWriter) 245 replyErrCh := make(chan error, 1) 246 w.flushChMutex.RLock() 247 if w.flushCh != nil { 248 w.flushCh <- replyErrCh 249 } else { 250 replyErrCh <- nil 251 } 252 w.flushChMutex.RUnlock() 253 select { 254 case <-ctx.Done(): 255 return ctx.Err() 256 case err := <-replyErrCh: 257 return err 258 } 259 }) 260 return true 261 }) 262 return eg.Wait() 263 } 264 265 func (e *LocalFile) flushEngineWithoutLock(ctx context.Context) error { 266 if err := e.flushLocalWriters(ctx); err != nil { 267 return err 268 } 269 if err := e.saveEngineMeta(); err != nil { 270 return err 271 } 272 flushFinishedCh, err := e.db.AsyncFlush() 273 if err != nil { 274 return err 275 } 276 select { 277 case <-flushFinishedCh: 278 return nil 279 case <-ctx.Done(): 280 return ctx.Err() 281 } 282 } 283 284 // saveEngineMeta saves the metadata about the DB into the DB itself. 285 // This method should be followed by a Flush to ensure the data is actually synchronized 286 func (e *LocalFile) saveEngineMeta() error { 287 jsonBytes, err := json.Marshal(&e.localFileMeta) 288 if err != nil { 289 return errors.Trace(err) 290 } 291 // note: we can't set Sync to true since we disabled WAL. 292 return errors.Trace(e.db.Set(engineMetaKey, jsonBytes, &pebble.WriteOptions{Sync: false})) 293 } 294 295 func (e *LocalFile) loadEngineMeta() { 296 jsonBytes, closer, err := e.db.Get(engineMetaKey) 297 if err != nil { 298 log.L().Debug("local db missing engine meta", zap.Stringer("uuid", e.Uuid), zap.Error(err)) 299 return 300 } 301 defer closer.Close() 302 303 err = json.Unmarshal(jsonBytes, &e.localFileMeta) 304 if err != nil { 305 log.L().Warn("local db failed to deserialize meta", zap.Stringer("uuid", e.Uuid), zap.ByteString("content", jsonBytes), zap.Error(err)) 306 } 307 } 308 309 type gRPCConns struct { 310 mu sync.Mutex 311 conns map[uint64]*connPool 312 } 313 314 func (conns *gRPCConns) Close() { 315 conns.mu.Lock() 316 defer conns.mu.Unlock() 317 318 for _, cp := range conns.conns { 319 cp.Close() 320 } 321 } 322 323 type local struct { 324 engines sync.Map // sync version of map[uuid.UUID]*LocalFile 325 326 conns gRPCConns 327 splitCli split.SplitClient 328 tls *common.TLS 329 pdAddr string 330 g glue.Glue 331 332 localStoreDir string 333 regionSplitSize int64 334 335 rangeConcurrency *worker.Pool 336 ingestConcurrency *worker.Pool 337 batchWriteKVPairs int 338 checkpointEnabled bool 339 340 tcpConcurrency int 341 maxOpenFiles int 342 } 343 344 // connPool is a lazy pool of gRPC channels. 345 // When `Get` called, it lazily allocates new connection if connection not full. 346 // If it's full, then it will return allocated channels round-robin. 347 type connPool struct { 348 mu sync.Mutex 349 350 conns []*grpc.ClientConn 351 name string 352 next int 353 cap int 354 newConn func(ctx context.Context) (*grpc.ClientConn, error) 355 } 356 357 func (p *connPool) takeConns() (conns []*grpc.ClientConn) { 358 p.mu.Lock() 359 defer p.mu.Unlock() 360 p.conns, conns = nil, p.conns 361 p.next = 0 362 return conns 363 } 364 365 // Close closes the conn pool. 366 func (p *connPool) Close() { 367 for _, c := range p.takeConns() { 368 if err := c.Close(); err != nil { 369 log.L().Warn("failed to close clientConn", zap.String("target", c.Target()), log.ShortError(err)) 370 } 371 } 372 } 373 374 // get tries to get an existing connection from the pool, or make a new one if the pool not full. 375 func (p *connPool) get(ctx context.Context) (*grpc.ClientConn, error) { 376 p.mu.Lock() 377 defer p.mu.Unlock() 378 if len(p.conns) < p.cap { 379 c, err := p.newConn(ctx) 380 if err != nil { 381 return nil, err 382 } 383 p.conns = append(p.conns, c) 384 return c, nil 385 } 386 387 conn := p.conns[p.next] 388 p.next = (p.next + 1) % p.cap 389 return conn, nil 390 } 391 392 // newConnPool creates a new connPool by the specified conn factory function and capacity. 393 func newConnPool(cap int, newConn func(ctx context.Context) (*grpc.ClientConn, error)) *connPool { 394 return &connPool{ 395 cap: cap, 396 conns: make([]*grpc.ClientConn, 0, cap), 397 newConn: newConn, 398 399 mu: sync.Mutex{}, 400 } 401 } 402 403 // NewLocalBackend creates new connections to tikv. 404 func NewLocalBackend( 405 ctx context.Context, 406 tls *common.TLS, 407 pdAddr string, 408 regionSplitSize int64, 409 localFile string, 410 rangeConcurrency int, 411 sendKVPairs int, 412 enableCheckpoint bool, 413 g glue.Glue, 414 maxOpenFiles int, 415 ) (Backend, error) { 416 pdCli, err := pd.NewClientWithContext(ctx, []string{pdAddr}, tls.ToPDSecurityOption()) 417 if err != nil { 418 return MakeBackend(nil), errors.Annotate(err, "construct pd client failed") 419 } 420 splitCli := split.NewSplitClient(pdCli, tls.TLSConfig()) 421 422 shouldCreate := true 423 if enableCheckpoint { 424 if info, err := os.Stat(localFile); err != nil { 425 if !os.IsNotExist(err) { 426 return MakeBackend(nil), err 427 } 428 } else if info.IsDir() { 429 shouldCreate = false 430 } 431 } 432 433 if shouldCreate { 434 err = os.Mkdir(localFile, 0700) 435 if err != nil { 436 return MakeBackend(nil), errors.Annotate(err, "invalid sorted-kv-dir for local backend, please change the config or delete the path") 437 } 438 } 439 440 local := &local{ 441 engines: sync.Map{}, 442 splitCli: splitCli, 443 tls: tls, 444 pdAddr: pdAddr, 445 g: g, 446 447 localStoreDir: localFile, 448 regionSplitSize: regionSplitSize, 449 450 rangeConcurrency: worker.NewPool(ctx, rangeConcurrency, "range"), 451 ingestConcurrency: worker.NewPool(ctx, rangeConcurrency*2, "ingest"), 452 tcpConcurrency: rangeConcurrency, 453 batchWriteKVPairs: sendKVPairs, 454 checkpointEnabled: enableCheckpoint, 455 maxOpenFiles: utils.MaxInt(maxOpenFiles, openFilesLowerThreshold), 456 } 457 local.conns.conns = make(map[uint64]*connPool) 458 return MakeBackend(local), nil 459 } 460 461 // lock locks a local file and returns the LocalFile instance if it exists. 462 func (local *local) lockEngine(engineId uuid.UUID, state importMutexState) *LocalFile { 463 if e, ok := local.engines.Load(engineId); ok { 464 engine := e.(*LocalFile) 465 engine.lock(state) 466 return engine 467 } 468 return nil 469 } 470 471 // lockAllEnginesUnless tries to lock all engines, unless those which are already locked in the 472 // state given by ignoreStateMask. Returns the list of locked engines. 473 func (local *local) lockAllEnginesUnless(newState, ignoreStateMask importMutexState) []*LocalFile { 474 var allEngines []*LocalFile 475 local.engines.Range(func(k, v interface{}) bool { 476 engine := v.(*LocalFile) 477 if engine.lockUnless(newState, ignoreStateMask) { 478 allEngines = append(allEngines, engine) 479 } 480 return true 481 }) 482 return allEngines 483 } 484 485 func (local *local) makeConn(ctx context.Context, storeID uint64) (*grpc.ClientConn, error) { 486 store, err := local.splitCli.GetStore(ctx, storeID) 487 if err != nil { 488 return nil, errors.Trace(err) 489 } 490 opt := grpc.WithInsecure() 491 if local.tls.TLSConfig() != nil { 492 opt = grpc.WithTransportCredentials(credentials.NewTLS(local.tls.TLSConfig())) 493 } 494 ctx, cancel := context.WithTimeout(ctx, dialTimeout) 495 496 bfConf := backoff.DefaultConfig 497 bfConf.MaxDelay = gRPCBackOffMaxDelay 498 // we should use peer address for tiflash. for tikv, peer address is empty 499 addr := store.GetPeerAddress() 500 if addr == "" { 501 addr = store.GetAddress() 502 } 503 conn, err := grpc.DialContext( 504 ctx, 505 addr, 506 opt, 507 grpc.WithConnectParams(grpc.ConnectParams{Backoff: bfConf}), 508 grpc.WithKeepaliveParams(keepalive.ClientParameters{ 509 Time: gRPCKeepAliveTime, 510 Timeout: gRPCKeepAliveTimeout, 511 PermitWithoutStream: true, 512 }), 513 ) 514 cancel() 515 if err != nil { 516 return nil, errors.Trace(err) 517 } 518 return conn, nil 519 } 520 521 func (local *local) getGrpcConnLocked(ctx context.Context, storeID uint64) (*grpc.ClientConn, error) { 522 if _, ok := local.conns.conns[storeID]; !ok { 523 local.conns.conns[storeID] = newConnPool(local.tcpConcurrency, func(ctx context.Context) (*grpc.ClientConn, error) { 524 return local.makeConn(ctx, storeID) 525 }) 526 } 527 return local.conns.conns[storeID].get(ctx) 528 } 529 530 // Close the local backend. 531 func (local *local) Close() { 532 allEngines := local.lockAllEnginesUnless(importMutexStateClose, 0) 533 local.engines = sync.Map{} 534 535 for _, engine := range allEngines { 536 engine.Close() 537 engine.unlock() 538 } 539 local.conns.Close() 540 541 // if checkpoint is disable or we finish load all data successfully, then files in this 542 // dir will be useless, so we clean up this dir and all files in it. 543 if !local.checkpointEnabled || common.IsEmptyDir(local.localStoreDir) { 544 err := os.RemoveAll(local.localStoreDir) 545 if err != nil { 546 log.L().Warn("remove local db file failed", zap.Error(err)) 547 } 548 } 549 } 550 551 // FlushEngine ensure the written data is saved successfully, to make sure no data lose after restart 552 func (local *local) FlushEngine(ctx context.Context, engineId uuid.UUID) error { 553 engineFile := local.lockEngine(engineId, importMutexStateFlush) 554 555 // the engine cannot be deleted after while we've acquired the lock identified by UUID. 556 557 if engineFile == nil { 558 return errors.Errorf("engine '%s' not found", engineId) 559 } 560 defer engineFile.unlock() 561 return engineFile.flushEngineWithoutLock(ctx) 562 } 563 564 func (local *local) FlushAllEngines(parentCtx context.Context) (err error) { 565 allEngines := local.lockAllEnginesUnless(importMutexStateFlush, ^importMutexStateLocalIngest) 566 defer func() { 567 for _, engine := range allEngines { 568 engine.unlock() 569 } 570 }() 571 572 eg, ctx := errgroup.WithContext(parentCtx) 573 for _, engineFile := range allEngines { 574 ef := engineFile 575 eg.Go(func() error { 576 return ef.flushEngineWithoutLock(ctx) 577 }) 578 } 579 return eg.Wait() 580 } 581 582 func (local *local) RetryImportDelay() time.Duration { 583 return defaultRetryBackoffTime 584 } 585 586 func (local *local) MaxChunkSize() int { 587 // a batch size write to leveldb 588 return int(local.regionSplitSize) 589 } 590 591 func (local *local) ShouldPostProcess() bool { 592 return true 593 } 594 595 func (local *local) openEngineDB(engineUUID uuid.UUID, readOnly bool) (*pebble.DB, error) { 596 opt := &pebble.Options{ 597 MemTableSize: LocalMemoryTableSize, 598 // the default threshold value may cause write stall. 599 MemTableStopWritesThreshold: 8, 600 MaxConcurrentCompactions: 16, 601 // set to half of the max open files so that if open files is more that estimation, trigger compaction 602 // to avoid failure due to open files exceeded limit 603 L0CompactionThreshold: local.maxOpenFiles / 2, 604 L0StopWritesThreshold: local.maxOpenFiles / 2, 605 MaxOpenFiles: local.maxOpenFiles, 606 DisableWAL: true, 607 ReadOnly: readOnly, 608 TablePropertyCollectors: []func() pebble.TablePropertyCollector{ 609 newRangePropertiesCollector, 610 }, 611 } 612 dbPath := filepath.Join(local.localStoreDir, engineUUID.String()) 613 return pebble.Open(dbPath, opt) 614 } 615 616 // This method must be called with holding mutex of LocalFile 617 func (local *local) OpenEngine(ctx context.Context, engineUUID uuid.UUID) error { 618 db, err := local.openEngineDB(engineUUID, false) 619 if err != nil { 620 return err 621 } 622 e, _ := local.engines.LoadOrStore(engineUUID, &LocalFile{Uuid: engineUUID}) 623 engine := e.(*LocalFile) 624 engine.db = db 625 engine.loadEngineMeta() 626 return nil 627 } 628 629 // Close backend engine by uuid 630 // NOTE: we will return nil if engine is not exist. This will happen if engine import&cleanup successfully 631 // but exit before update checkpoint. Thus after restart, we will try to import this engine again. 632 func (local *local) CloseEngine(ctx context.Context, engineUUID uuid.UUID) error { 633 // flush mem table to storage, to free memory, 634 // ask others' advise, looks like unnecessary, but with this we can control memory precisely. 635 engine, ok := local.engines.Load(engineUUID) 636 if !ok { 637 // recovery mode, we should reopen this engine file 638 db, err := local.openEngineDB(engineUUID, true) 639 if err != nil { 640 // if engine db does not exist, just skip 641 if os.IsNotExist(errors.Cause(err)) { 642 return nil 643 } 644 return err 645 } 646 engineFile := &LocalFile{ 647 Uuid: engineUUID, 648 db: db, 649 } 650 engineFile.loadEngineMeta() 651 local.engines.Store(engineUUID, engineFile) 652 return nil 653 } 654 engineFile := engine.(*LocalFile) 655 engineFile.lock(importMutexStateFlush) 656 defer engineFile.unlock() 657 return engineFile.flushEngineWithoutLock(ctx) 658 } 659 660 func (local *local) getImportClient(ctx context.Context, peer *metapb.Peer) (sst.ImportSSTClient, error) { 661 local.conns.mu.Lock() 662 defer local.conns.mu.Unlock() 663 664 conn, err := local.getGrpcConnLocked(ctx, peer.GetStoreId()) 665 if err != nil { 666 return nil, err 667 } 668 return sst.NewImportSSTClient(conn), nil 669 } 670 671 type rangeStats struct { 672 count int64 673 totalBytes int64 674 } 675 676 // WriteToTiKV writer engine key-value pairs to tikv and return the sst meta generated by tikv. 677 // we don't need to do cleanup for the pairs written to tikv if encounters an error, 678 // tikv will takes the responsibility to do so. 679 func (local *local) WriteToTiKV( 680 ctx context.Context, 681 engineFile *LocalFile, 682 region *split.RegionInfo, 683 start, end []byte, 684 ) ([]*sst.SSTMeta, *Range, rangeStats, error) { 685 begin := time.Now() 686 regionRange := intersectRange(region.Region, Range{start: start, end: end}) 687 opt := &pebble.IterOptions{LowerBound: regionRange.start, UpperBound: regionRange.end} 688 iter := engineFile.db.NewIter(opt) 689 defer iter.Close() 690 691 stats := rangeStats{} 692 693 iter.First() 694 if iter.Error() != nil { 695 return nil, nil, stats, errors.Annotate(iter.Error(), "failed to read the first key") 696 } 697 if !iter.Valid() { 698 log.L().Info("keys within region is empty, skip ingest", log.ZapRedactBinary("start", start), 699 log.ZapRedactBinary("regionStart", region.Region.StartKey), log.ZapRedactBinary("end", end), 700 log.ZapRedactBinary("regionEnd", region.Region.EndKey)) 701 return nil, nil, stats, nil 702 } 703 704 firstKey := codec.EncodeBytes([]byte{}, iter.Key()) 705 iter.Last() 706 if iter.Error() != nil { 707 return nil, nil, stats, errors.Annotate(iter.Error(), "failed to seek to the last key") 708 } 709 lastKey := codec.EncodeBytes([]byte{}, iter.Key()) 710 711 u := uuid.New() 712 meta := &sst.SSTMeta{ 713 Uuid: u[:], 714 RegionId: region.Region.GetId(), 715 RegionEpoch: region.Region.GetRegionEpoch(), 716 Range: &sst.Range{ 717 Start: firstKey, 718 End: lastKey, 719 }, 720 } 721 722 leaderID := region.Leader.GetId() 723 clients := make([]sst.ImportSST_WriteClient, 0, len(region.Region.GetPeers())) 724 requests := make([]*sst.WriteRequest, 0, len(region.Region.GetPeers())) 725 for _, peer := range region.Region.GetPeers() { 726 cli, err := local.getImportClient(ctx, peer) 727 if err != nil { 728 return nil, nil, stats, err 729 } 730 731 wstream, err := cli.Write(ctx) 732 if err != nil { 733 return nil, nil, stats, errors.Trace(err) 734 } 735 736 // Bind uuid for this write request 737 req := &sst.WriteRequest{ 738 Chunk: &sst.WriteRequest_Meta{ 739 Meta: meta, 740 }, 741 } 742 if err = wstream.Send(req); err != nil { 743 return nil, nil, stats, errors.Trace(err) 744 } 745 req.Chunk = &sst.WriteRequest_Batch{ 746 Batch: &sst.WriteBatch{ 747 CommitTs: engineFile.Ts, 748 }, 749 } 750 clients = append(clients, wstream) 751 requests = append(requests, req) 752 } 753 754 bytesBuf := newBytesBuffer() 755 defer bytesBuf.destroy() 756 pairs := make([]*sst.Pair, 0, local.batchWriteKVPairs) 757 count := 0 758 size := int64(0) 759 totalCount := int64(0) 760 firstLoop := true 761 regionMaxSize := local.regionSplitSize * 4 / 3 762 763 for iter.First(); iter.Valid(); iter.Next() { 764 size += int64(len(iter.Key()) + len(iter.Value())) 765 // here we reuse the `*sst.Pair`s to optimize object allocation 766 if firstLoop { 767 pair := &sst.Pair{ 768 Key: bytesBuf.addBytes(iter.Key()), 769 Value: bytesBuf.addBytes(iter.Value()), 770 } 771 pairs = append(pairs, pair) 772 } else { 773 pairs[count].Key = bytesBuf.addBytes(iter.Key()) 774 pairs[count].Value = bytesBuf.addBytes(iter.Value()) 775 } 776 count++ 777 totalCount++ 778 779 if count >= local.batchWriteKVPairs { 780 for i := range clients { 781 requests[i].Chunk.(*sst.WriteRequest_Batch).Batch.Pairs = pairs[:count] 782 if err := clients[i].Send(requests[i]); err != nil { 783 return nil, nil, stats, err 784 } 785 } 786 count = 0 787 bytesBuf.reset() 788 firstLoop = false 789 } 790 if size >= regionMaxSize || totalCount >= regionMaxKeyCount { 791 break 792 } 793 } 794 795 if iter.Error() != nil { 796 return nil, nil, stats, errors.Trace(iter.Error()) 797 } 798 799 if count > 0 { 800 for i := range clients { 801 requests[i].Chunk.(*sst.WriteRequest_Batch).Batch.Pairs = pairs[:count] 802 if err := clients[i].Send(requests[i]); err != nil { 803 return nil, nil, stats, err 804 } 805 } 806 } 807 808 var leaderPeerMetas []*sst.SSTMeta 809 for i, wStream := range clients { 810 if resp, closeErr := wStream.CloseAndRecv(); closeErr != nil { 811 return nil, nil, stats, closeErr 812 } else { 813 if leaderID == region.Region.Peers[i].GetId() { 814 leaderPeerMetas = resp.Metas 815 log.L().Debug("get metas after write kv stream to tikv", zap.Reflect("metas", leaderPeerMetas)) 816 } 817 } 818 } 819 820 // if there is not leader currently, we should directly return an error 821 if leaderPeerMetas == nil { 822 log.L().Warn("write to tikv no leader", log.ZapRedactReflect("region", region), 823 zap.Uint64("leader_id", leaderID), log.ZapRedactReflect("meta", meta), 824 zap.Int64("kv_pairs", totalCount), zap.Int64("total_bytes", size)) 825 return nil, nil, stats, errors.Errorf("write to tikv with no leader returned, region '%d', leader: %d", 826 region.Region.Id, leaderID) 827 } 828 829 log.L().Debug("write to kv", zap.Reflect("region", region), zap.Uint64("leader", leaderID), 830 zap.Reflect("meta", meta), zap.Reflect("return metas", leaderPeerMetas), 831 zap.Int64("kv_pairs", totalCount), zap.Int64("total_bytes", size), 832 zap.Int64("buf_size", bytesBuf.totalSize()), 833 zap.Stringer("takeTime", time.Since(begin))) 834 835 var remainRange *Range 836 if iter.Valid() && iter.Next() { 837 firstKey := append([]byte{}, iter.Key()...) 838 remainRange = &Range{start: firstKey, end: regionRange.end} 839 log.L().Info("write to tikv partial finish", zap.Int64("count", totalCount), 840 zap.Int64("size", size), log.ZapRedactBinary("startKey", regionRange.start), log.ZapRedactBinary("endKey", regionRange.end), 841 log.ZapRedactBinary("remainStart", remainRange.start), log.ZapRedactBinary("remainEnd", remainRange.end), 842 log.ZapRedactReflect("region", region)) 843 } 844 stats.count = totalCount 845 stats.totalBytes = size 846 847 return leaderPeerMetas, remainRange, stats, nil 848 } 849 850 func (local *local) Ingest(ctx context.Context, meta *sst.SSTMeta, region *split.RegionInfo) (*sst.IngestResponse, error) { 851 leader := region.Leader 852 if leader == nil { 853 leader = region.Region.GetPeers()[0] 854 } 855 856 cli, err := local.getImportClient(ctx, leader) 857 if err != nil { 858 return nil, err 859 } 860 reqCtx := &kvrpcpb.Context{ 861 RegionId: region.Region.GetId(), 862 RegionEpoch: region.Region.GetRegionEpoch(), 863 Peer: leader, 864 } 865 866 req := &sst.IngestRequest{ 867 Context: reqCtx, 868 Sst: meta, 869 } 870 resp, err := cli.Ingest(ctx, req) 871 if err != nil { 872 return nil, err 873 } 874 return resp, nil 875 } 876 877 func splitRangeBySizeProps(fullRange Range, sizeProps *sizeProperties, sizeLimit int64, keysLimit int64) []Range { 878 ranges := make([]Range, 0, sizeProps.totalSize/uint64(sizeLimit)) 879 curSize := uint64(0) 880 curKeys := uint64(0) 881 curKey := fullRange.start 882 sizeProps.iter(func(p *rangeProperty) bool { 883 curSize += p.Size 884 curKeys += p.Keys 885 if int64(curSize) >= sizeLimit || int64(curKeys) >= keysLimit { 886 ranges = append(ranges, Range{start: curKey, end: p.Key}) 887 curKey = p.Key 888 curSize = 0 889 curKeys = 0 890 } 891 return true 892 }) 893 894 if curKeys > 0 { 895 ranges = append(ranges, Range{start: curKey, end: fullRange.end}) 896 } else { 897 ranges[len(ranges)-1].end = fullRange.end 898 } 899 return ranges 900 } 901 902 func (local *local) readAndSplitIntoRange(engineFile *LocalFile) ([]Range, error) { 903 iter := engineFile.db.NewIter(&pebble.IterOptions{LowerBound: normalIterStartKey}) 904 defer iter.Close() 905 906 iterError := func(e string) error { 907 err := iter.Error() 908 if err != nil { 909 return errors.Annotate(err, e) 910 } 911 return errors.New(e) 912 } 913 914 var firstKey, lastKey []byte 915 if iter.First() { 916 firstKey = append([]byte{}, iter.Key()...) 917 } else { 918 return nil, iterError("could not find first pair") 919 } 920 if iter.Last() { 921 lastKey = append([]byte{}, iter.Key()...) 922 } else { 923 return nil, iterError("could not find last pair") 924 } 925 endKey := nextKey(lastKey) 926 927 engineFileTotalSize := engineFile.TotalSize.Load() 928 engineFileLength := engineFile.Length.Load() 929 930 // <= 96MB no need to split into range 931 if engineFileTotalSize <= local.regionSplitSize && engineFileLength <= regionMaxKeyCount { 932 ranges := []Range{{start: firstKey, end: endKey, length: int(engineFileLength)}} 933 return ranges, nil 934 } 935 936 sizeProps, err := engineFile.getSizeProperties() 937 if err != nil { 938 return nil, errors.Trace(err) 939 } 940 941 ranges := splitRangeBySizeProps(Range{start: firstKey, end: endKey}, sizeProps, 942 local.regionSplitSize, regionMaxKeyCount*2/3) 943 944 log.L().Info("split engine key ranges", zap.Stringer("engine", engineFile.Uuid), 945 zap.Int64("totalSize", engineFileTotalSize), zap.Int64("totalCount", engineFileLength), 946 log.ZapRedactBinary("firstKey", firstKey), log.ZapRedactBinary("lastKey", lastKey), 947 zap.Int("ranges", len(ranges))) 948 949 return ranges, nil 950 } 951 952 type bytesRecycleChan struct { 953 ch chan []byte 954 } 955 956 // recycleChan is used for reusing allocated []byte so we can use memory more efficiently 957 // 958 // NOTE: we don't used a `sync.Pool` because when will sync.Pool release is depending on the 959 // garbage collector which always release the memory so late. Use a fixed size chan to reuse 960 // can decrease the memory usage to 1/3 compare with sync.Pool. 961 var recycleChan *bytesRecycleChan 962 963 func init() { 964 recycleChan = &bytesRecycleChan{ 965 ch: make(chan []byte, 1024), 966 } 967 } 968 969 func (c *bytesRecycleChan) Acquire() []byte { 970 select { 971 case b := <-c.ch: 972 return b 973 default: 974 return manual.New(1 << 20) // 1M 975 } 976 } 977 978 func (c *bytesRecycleChan) Release(w []byte) { 979 select { 980 case c.ch <- w: 981 return 982 default: 983 manual.Free(w) 984 } 985 } 986 987 type bytesBuffer struct { 988 bufs [][]byte 989 curBuf []byte 990 curIdx int 991 curBufIdx int 992 curBufLen int 993 } 994 995 func newBytesBuffer() *bytesBuffer { 996 return &bytesBuffer{bufs: make([][]byte, 0, 128), curBufIdx: -1} 997 } 998 999 func (b *bytesBuffer) addBuf() { 1000 if b.curBufIdx < len(b.bufs)-1 { 1001 b.curBufIdx += 1 1002 b.curBuf = b.bufs[b.curBufIdx] 1003 } else { 1004 buf := recycleChan.Acquire() 1005 b.bufs = append(b.bufs, buf) 1006 b.curBuf = buf 1007 b.curBufIdx = len(b.bufs) - 1 1008 } 1009 1010 b.curBufLen = len(b.curBuf) 1011 b.curIdx = 0 1012 } 1013 1014 func (b *bytesBuffer) reset() { 1015 if len(b.bufs) > 0 { 1016 b.curBuf = b.bufs[0] 1017 b.curBufLen = len(b.bufs[0]) 1018 b.curBufIdx = 0 1019 b.curIdx = 0 1020 } 1021 } 1022 1023 func (b *bytesBuffer) destroy() { 1024 for _, buf := range b.bufs { 1025 recycleChan.Release(buf) 1026 } 1027 b.bufs = b.bufs[:0] 1028 } 1029 1030 func (b *bytesBuffer) totalSize() int64 { 1031 return int64(len(b.bufs)) * int64(1<<20) 1032 } 1033 1034 func (b *bytesBuffer) addBytes(bytes []byte) []byte { 1035 if len(bytes) > bigValueSize { 1036 return append([]byte{}, bytes...) 1037 } 1038 1039 if b.curIdx+len(bytes) > b.curBufLen { 1040 b.addBuf() 1041 } 1042 idx := b.curIdx 1043 copy(b.curBuf[idx:], bytes) 1044 b.curIdx += len(bytes) 1045 return b.curBuf[idx:b.curIdx] 1046 } 1047 1048 func (local *local) writeAndIngestByRange( 1049 ctxt context.Context, 1050 engineFile *LocalFile, 1051 start, end []byte, 1052 remainRanges *syncdRanges, 1053 ) error { 1054 ito := &pebble.IterOptions{ 1055 LowerBound: start, 1056 UpperBound: end, 1057 } 1058 1059 iter := engineFile.db.NewIter(ito) 1060 defer iter.Close() 1061 // Needs seek to first because NewIter returns an iterator that is unpositioned 1062 hasKey := iter.First() 1063 if iter.Error() != nil { 1064 return errors.Annotate(iter.Error(), "failed to read the first key") 1065 } 1066 if !hasKey { 1067 log.L().Info("There is no pairs in iterator", 1068 log.ZapRedactBinary("start", start), 1069 log.ZapRedactBinary("end", end), 1070 log.ZapRedactBinary("next end", nextKey(end))) 1071 return nil 1072 } 1073 pairStart := append([]byte{}, iter.Key()...) 1074 iter.Last() 1075 if iter.Error() != nil { 1076 return errors.Annotate(iter.Error(), "failed to seek to the last key") 1077 } 1078 pairEnd := append([]byte{}, iter.Key()...) 1079 1080 var regions []*split.RegionInfo 1081 var err error 1082 ctx, cancel := context.WithCancel(ctxt) 1083 defer cancel() 1084 1085 WriteAndIngest: 1086 for retry := 0; retry < maxRetryTimes; { 1087 if retry != 0 { 1088 select { 1089 case <-time.After(time.Second): 1090 case <-ctx.Done(): 1091 return ctx.Err() 1092 } 1093 } 1094 startKey := codec.EncodeBytes([]byte{}, pairStart) 1095 endKey := codec.EncodeBytes([]byte{}, nextKey(pairEnd)) 1096 regions, err = paginateScanRegion(ctx, local.splitCli, startKey, endKey, 128) 1097 if err != nil || len(regions) == 0 { 1098 log.L().Warn("scan region failed", log.ShortError(err), zap.Int("region_len", len(regions)), 1099 log.ZapRedactBinary("startKey", startKey), log.ZapRedactBinary("endKey", endKey), zap.Int("retry", retry)) 1100 retry++ 1101 continue WriteAndIngest 1102 } 1103 1104 for _, region := range regions { 1105 log.L().Debug("get region", zap.Int("retry", retry), zap.Binary("startKey", startKey), 1106 zap.Binary("endKey", endKey), zap.Uint64("id", region.Region.GetId()), 1107 zap.Stringer("epoch", region.Region.GetRegionEpoch()), zap.Binary("start", region.Region.GetStartKey()), 1108 zap.Binary("end", region.Region.GetEndKey()), zap.Reflect("peers", region.Region.GetPeers())) 1109 1110 w := local.ingestConcurrency.Apply() 1111 var rg *Range 1112 rg, err = local.writeAndIngestPairs(ctx, engineFile, region, pairStart, end) 1113 local.ingestConcurrency.Recycle(w) 1114 if err != nil { 1115 _, regionStart, _ := codec.DecodeBytes(region.Region.StartKey, []byte{}) 1116 // if we have at least succeeded one region, retry without increasing the retry count 1117 if bytes.Compare(regionStart, pairStart) > 0 { 1118 pairStart = regionStart 1119 } else { 1120 retry++ 1121 } 1122 log.L().Info("retry write and ingest kv pairs", log.ZapRedactBinary("startKey", pairStart), 1123 log.ZapRedactBinary("endKey", end), log.ShortError(err), zap.Int("retry", retry)) 1124 continue WriteAndIngest 1125 } 1126 if rg != nil { 1127 remainRanges.add(*rg) 1128 } 1129 } 1130 1131 return err 1132 } 1133 1134 return err 1135 } 1136 1137 type retryType int 1138 1139 const ( 1140 retryNone retryType = iota 1141 retryWrite 1142 retryIngest 1143 ) 1144 1145 func (local *local) writeAndIngestPairs( 1146 ctx context.Context, 1147 engineFile *LocalFile, 1148 region *split.RegionInfo, 1149 start, end []byte, 1150 ) (*Range, error) { 1151 var err error 1152 var remainRange *Range 1153 var rangeStats rangeStats 1154 loopWrite: 1155 for i := 0; i < maxRetryTimes; i++ { 1156 var metas []*sst.SSTMeta 1157 metas, remainRange, rangeStats, err = local.WriteToTiKV(ctx, engineFile, region, start, end) 1158 if err != nil { 1159 log.L().Warn("write to tikv failed", log.ShortError(err)) 1160 return nil, err 1161 } 1162 1163 for _, meta := range metas { 1164 errCnt := 0 1165 for errCnt < maxRetryTimes { 1166 log.L().Debug("ingest meta", zap.Reflect("meta", meta)) 1167 var resp *sst.IngestResponse 1168 failpoint.Inject("FailIngestMeta", func(val failpoint.Value) { 1169 // only inject the error once 1170 switch val.(string) { 1171 case "notleader": 1172 resp = &sst.IngestResponse{ 1173 Error: &errorpb.Error{ 1174 NotLeader: &errorpb.NotLeader{ 1175 RegionId: region.Region.Id, 1176 Leader: region.Leader, 1177 }, 1178 }, 1179 } 1180 case "epochnotmatch": 1181 resp = &sst.IngestResponse{ 1182 Error: &errorpb.Error{ 1183 EpochNotMatch: &errorpb.EpochNotMatch{ 1184 CurrentRegions: []*metapb.Region{region.Region}, 1185 }, 1186 }, 1187 } 1188 } 1189 if resp != nil { 1190 err = nil 1191 } 1192 }) 1193 if resp == nil { 1194 resp, err = local.Ingest(ctx, meta, region) 1195 } 1196 if err != nil { 1197 if errors.Cause(err) == context.Canceled { 1198 return nil, err 1199 } 1200 log.L().Warn("ingest failed", log.ShortError(err), log.ZapRedactReflect("meta", meta), 1201 log.ZapRedactReflect("region", region)) 1202 errCnt++ 1203 continue 1204 } 1205 1206 var retryTy retryType 1207 var newRegion *split.RegionInfo 1208 retryTy, newRegion, err = local.isIngestRetryable(ctx, resp, region, meta) 1209 if err == nil { 1210 // ingest next meta 1211 break 1212 } 1213 switch retryTy { 1214 case retryNone: 1215 log.L().Warn("ingest failed noretry", log.ShortError(err), log.ZapRedactReflect("meta", meta), 1216 log.ZapRedactReflect("region", region)) 1217 // met non-retryable error retry whole Write procedure 1218 return remainRange, err 1219 case retryWrite: 1220 region = newRegion 1221 continue loopWrite 1222 case retryIngest: 1223 region = newRegion 1224 continue 1225 } 1226 } 1227 } 1228 1229 if err != nil { 1230 log.L().Warn("write and ingest region, will retry import full range", log.ShortError(err), 1231 log.ZapRedactStringer("region", region.Region), log.ZapRedactBinary("start", start), 1232 log.ZapRedactBinary("end", end)) 1233 } else { 1234 metric.BytesCounter.WithLabelValues(metric.TableStateImported).Add(float64(rangeStats.totalBytes)) 1235 } 1236 return remainRange, errors.Trace(err) 1237 } 1238 1239 return remainRange, errors.Trace(err) 1240 } 1241 1242 func (local *local) writeAndIngestByRanges(ctx context.Context, engineFile *LocalFile, ranges []Range, remainRanges *syncdRanges) error { 1243 if engineFile.Length.Load() == 0 { 1244 // engine is empty, this is likes because it's a index engine but the table contains no index 1245 log.L().Info("engine contains no data", zap.Stringer("uuid", engineFile.Uuid)) 1246 return nil 1247 } 1248 log.L().Debug("the ranges Length write to tikv", zap.Int("Length", len(ranges))) 1249 1250 var allErrLock sync.Mutex 1251 var allErr error 1252 var wg sync.WaitGroup 1253 1254 wg.Add(len(ranges)) 1255 1256 for _, r := range ranges { 1257 startKey := r.start 1258 endKey := r.end 1259 w := local.rangeConcurrency.Apply() 1260 go func(w *worker.Worker) { 1261 defer func() { 1262 local.rangeConcurrency.Recycle(w) 1263 wg.Done() 1264 }() 1265 var err error 1266 for i := 0; i < maxRetryTimes; i++ { 1267 err = local.writeAndIngestByRange(ctx, engineFile, startKey, endKey, remainRanges) 1268 if err == nil || errors.Cause(err) == context.Canceled { 1269 return 1270 } 1271 log.L().Warn("write and ingest by range failed", 1272 zap.Int("retry time", i+1), log.ShortError(err)) 1273 } 1274 1275 allErrLock.Lock() 1276 allErr = multierr.Append(allErr, err) 1277 allErrLock.Unlock() 1278 }(w) 1279 } 1280 1281 // wait for all sub tasks finish to avoid panic. if we return on the first error, 1282 // the outer tasks may close the pebble db but some sub tasks still read from the db 1283 wg.Wait() 1284 return allErr 1285 } 1286 1287 type syncdRanges struct { 1288 sync.Mutex 1289 ranges []Range 1290 } 1291 1292 func (r *syncdRanges) add(g Range) { 1293 r.Lock() 1294 r.ranges = append(r.ranges, g) 1295 r.Unlock() 1296 } 1297 1298 func (r *syncdRanges) take() []Range { 1299 r.Lock() 1300 rg := r.ranges 1301 r.ranges = []Range{} 1302 r.Unlock() 1303 if len(rg) > 0 { 1304 sort.Slice(rg, func(i, j int) bool { 1305 return bytes.Compare(rg[i].start, rg[j].start) < 0 1306 }) 1307 } 1308 return rg 1309 } 1310 1311 func (local *local) ImportEngine(ctx context.Context, engineUUID uuid.UUID) error { 1312 lf := local.lockEngine(engineUUID, importMutexStateImport) 1313 if lf == nil { 1314 // skip if engine not exist. See the comment of `CloseEngine` for more detail. 1315 return nil 1316 } 1317 defer lf.unlock() 1318 1319 lfTotalSize := lf.TotalSize.Load() 1320 lfLength := lf.Length.Load() 1321 1322 if lfTotalSize == 0 { 1323 log.L().Info("engine contains no kv, skip import", zap.Stringer("engine", engineUUID)) 1324 return nil 1325 } 1326 1327 // split sorted file into range by 96MB size per file 1328 ranges, err := local.readAndSplitIntoRange(lf) 1329 if err != nil { 1330 return err 1331 } 1332 remains := &syncdRanges{} 1333 1334 for { 1335 log.L().Info("start import engine", zap.Stringer("uuid", engineUUID), 1336 zap.Int("ranges", len(ranges))) 1337 1338 // if all the kv can fit in one region, skip split regions. TiDB will split one region for 1339 // the table when table is created. 1340 needSplit := len(ranges) > 1 || lfTotalSize > local.regionSplitSize || lfLength > regionMaxKeyCount 1341 1342 // split region by given ranges 1343 for i := 0; i < maxRetryTimes; i++ { 1344 err = local.SplitAndScatterRegionByRanges(ctx, ranges, needSplit) 1345 if err == nil || common.IsContextCanceledError(err) { 1346 break 1347 } 1348 1349 log.L().Warn("split and scatter failed in retry", zap.Stringer("uuid", engineUUID), 1350 log.ShortError(err), zap.Int("retry", i)) 1351 } 1352 if err != nil { 1353 log.L().Error("split & scatter ranges failed", zap.Stringer("uuid", engineUUID), log.ShortError(err)) 1354 return err 1355 } 1356 1357 // start to write to kv and ingest 1358 err = local.writeAndIngestByRanges(ctx, lf, ranges, remains) 1359 if err != nil { 1360 log.L().Error("write and ingest engine failed", log.ShortError(err)) 1361 return err 1362 } 1363 1364 unfinishedRanges := remains.take() 1365 if len(unfinishedRanges) == 0 { 1366 break 1367 } 1368 log.L().Info("ingest ranges unfinished", zap.Int("remain ranges", len(unfinishedRanges))) 1369 ranges = unfinishedRanges 1370 } 1371 1372 log.L().Info("import engine success", zap.Stringer("uuid", engineUUID), 1373 zap.Int64("size", lfTotalSize), zap.Int64("kvs", lfLength)) 1374 return nil 1375 } 1376 1377 func (local *local) ResetEngine(ctx context.Context, engineUUID uuid.UUID) error { 1378 // the only way to reset the engine + reclaim the space is to delete and reopen it 🤷 1379 localEngine := local.lockEngine(engineUUID, importMutexStateClose) 1380 if localEngine == nil { 1381 log.L().Warn("could not find engine in cleanupEngine", zap.Stringer("uuid", engineUUID)) 1382 return nil 1383 } 1384 defer localEngine.unlock() 1385 if err := localEngine.Close(); err != nil { 1386 return err 1387 } 1388 if err := localEngine.Cleanup(local.localStoreDir); err != nil { 1389 return err 1390 } 1391 db, err := local.openEngineDB(engineUUID, false) 1392 if err == nil { 1393 localEngine.db = db 1394 localEngine.localFileMeta = localFileMeta{} 1395 } 1396 return err 1397 } 1398 1399 func (local *local) CleanupEngine(ctx context.Context, engineUUID uuid.UUID) error { 1400 localEngine := local.lockEngine(engineUUID, importMutexStateClose) 1401 // release this engine after import success 1402 if localEngine == nil { 1403 log.L().Warn("could not find engine in cleanupEngine", zap.Stringer("uuid", engineUUID)) 1404 return nil 1405 } 1406 defer localEngine.unlock() 1407 1408 // since closing the engine causes all subsequent operations on it panic, 1409 // we make sure to delete it from the engine map before calling Close(). 1410 // (note that Close() returning error does _not_ mean the pebble DB 1411 // remains open/usable.) 1412 local.engines.Delete(engineUUID) 1413 err := localEngine.Close() 1414 if err != nil { 1415 return err 1416 } 1417 err = localEngine.Cleanup(local.localStoreDir) 1418 if err != nil { 1419 return err 1420 } 1421 localEngine.TotalSize.Store(0) 1422 localEngine.Length.Store(0) 1423 return nil 1424 } 1425 1426 func (local *local) CheckRequirements(ctx context.Context) error { 1427 if err := checkTiDBVersionBySQL(ctx, local.g, localMinTiDBVersion); err != nil { 1428 return err 1429 } 1430 if err := checkPDVersion(ctx, local.tls, local.pdAddr, localMinPDVersion); err != nil { 1431 return err 1432 } 1433 if err := checkTiKVVersion(ctx, local.tls, local.pdAddr, localMinTiKVVersion); err != nil { 1434 return err 1435 } 1436 return nil 1437 } 1438 1439 func (local *local) FetchRemoteTableModels(ctx context.Context, schemaName string) ([]*model.TableInfo, error) { 1440 return fetchRemoteTableModelsFromTLS(ctx, local.tls, schemaName) 1441 } 1442 1443 func (local *local) MakeEmptyRows() Rows { 1444 return kvPairs(nil) 1445 } 1446 1447 func (local *local) NewEncoder(tbl table.Table, options *SessionOptions) (Encoder, error) { 1448 return NewTableKVEncoder(tbl, options) 1449 } 1450 1451 func (local *local) LocalWriter(ctx context.Context, engineUUID uuid.UUID, maxCacheSize int64) (EngineWriter, error) { 1452 e, ok := local.engines.Load(engineUUID) 1453 if !ok { 1454 return nil, errors.Errorf("could not find engine for %s", engineUUID.String()) 1455 } 1456 engineFile := e.(*LocalFile) 1457 return openLocalWriter(engineFile, local.localStoreDir, maxCacheSize), nil 1458 } 1459 1460 func openLocalWriter(f *LocalFile, sstDir string, memtableSizeLimit int64) *LocalWriter { 1461 w := &LocalWriter{ 1462 sstDir: sstDir, 1463 kvsChan: make(chan []common.KvPair, 1024), 1464 flushCh: make(chan chan error), 1465 consumeCh: make(chan struct{}, 1), 1466 local: f, 1467 memtableSizeLimit: memtableSizeLimit, 1468 } 1469 f.localWriters.Store(w, nil) 1470 go w.writeRowsLoop() 1471 return w 1472 } 1473 1474 func (local *local) isIngestRetryable( 1475 ctx context.Context, 1476 resp *sst.IngestResponse, 1477 region *split.RegionInfo, 1478 meta *sst.SSTMeta, 1479 ) (retryType, *split.RegionInfo, error) { 1480 if resp.GetError() == nil { 1481 return retryNone, nil, nil 1482 } 1483 1484 getRegion := func() (*split.RegionInfo, error) { 1485 for i := 0; ; i++ { 1486 newRegion, err := local.splitCli.GetRegion(ctx, region.Region.GetStartKey()) 1487 if err != nil { 1488 return nil, errors.Trace(err) 1489 } 1490 if newRegion != nil { 1491 return newRegion, nil 1492 } 1493 log.L().Warn("get region by key return nil, will retry", log.ZapRedactReflect("region", region), 1494 zap.Int("retry", i)) 1495 select { 1496 case <-ctx.Done(): 1497 return nil, ctx.Err() 1498 case <-time.After(time.Second): 1499 } 1500 } 1501 } 1502 1503 var newRegion *split.RegionInfo 1504 var err error 1505 switch errPb := resp.GetError(); { 1506 case errPb.NotLeader != nil: 1507 if newLeader := errPb.GetNotLeader().GetLeader(); newLeader != nil { 1508 newRegion = &split.RegionInfo{ 1509 Leader: newLeader, 1510 Region: region.Region, 1511 } 1512 } else { 1513 newRegion, err = getRegion() 1514 if err != nil { 1515 return retryNone, nil, errors.Trace(err) 1516 } 1517 } 1518 // TODO: because in some case, TiKV may return retryable error while the ingest is succeeded. 1519 // Thus directly retry ingest may cause TiKV panic. So always return retryWrite here to avoid 1520 // this issue. 1521 // See: https://github.com/tikv/tikv/issues/9496 1522 return retryWrite, newRegion, errors.Errorf("not leader: %s", errPb.GetMessage()) 1523 case errPb.EpochNotMatch != nil: 1524 if currentRegions := errPb.GetEpochNotMatch().GetCurrentRegions(); currentRegions != nil { 1525 var currentRegion *metapb.Region 1526 for _, r := range currentRegions { 1527 if insideRegion(r, meta) { 1528 currentRegion = r 1529 break 1530 } 1531 } 1532 if currentRegion != nil { 1533 var newLeader *metapb.Peer 1534 for _, p := range currentRegion.Peers { 1535 if p.GetStoreId() == region.Leader.GetStoreId() { 1536 newLeader = p 1537 break 1538 } 1539 } 1540 if newLeader != nil { 1541 newRegion = &split.RegionInfo{ 1542 Leader: newLeader, 1543 Region: currentRegion, 1544 } 1545 } 1546 } 1547 } 1548 retryTy := retryNone 1549 if newRegion != nil { 1550 retryTy = retryWrite 1551 } 1552 return retryTy, newRegion, errors.Errorf("epoch not match: %s", errPb.GetMessage()) 1553 case strings.Contains(errPb.Message, "raft: proposal dropped"): 1554 // TODO: we should change 'Raft raft: proposal dropped' to a error type like 'NotLeader' 1555 newRegion, err = getRegion() 1556 if err != nil { 1557 return retryNone, nil, errors.Trace(err) 1558 } 1559 return retryWrite, newRegion, errors.New(errPb.GetMessage()) 1560 } 1561 return retryNone, nil, errors.Errorf("non-retryable error: %s", resp.GetError().GetMessage()) 1562 } 1563 1564 // return the smallest []byte that is bigger than current bytes. 1565 // special case when key is empty, empty bytes means infinity in our context, so directly return itself. 1566 func nextKey(key []byte) []byte { 1567 if len(key) == 0 { 1568 return []byte{} 1569 } 1570 1571 // in tikv <= 4.x, tikv will truncate the row key, so we should fetch the next valid row key 1572 // See: https://github.com/tikv/tikv/blob/f7f22f70e1585d7ca38a59ea30e774949160c3e8/components/raftstore/src/coprocessor/split_observer.rs#L36-L41 1573 if tablecodec.IsRecordKey(key) { 1574 tableId, handle, _ := tablecodec.DecodeRecordKey(key) 1575 return tablecodec.EncodeRowKeyWithHandle(tableId, handle.Next()) 1576 } 1577 1578 // if key is an index, directly append a 0x00 to the key. 1579 res := make([]byte, 0, len(key)+1) 1580 res = append(res, key...) 1581 res = append(res, 0) 1582 return res 1583 } 1584 1585 type rangeOffsets struct { 1586 Size uint64 1587 Keys uint64 1588 } 1589 1590 type rangeProperty struct { 1591 Key []byte 1592 rangeOffsets 1593 } 1594 1595 func (r *rangeProperty) Less(than btree.Item) bool { 1596 ta := than.(*rangeProperty) 1597 return bytes.Compare(r.Key, ta.Key) < 0 1598 } 1599 1600 var _ btree.Item = &rangeProperty{} 1601 1602 type rangeProperties []rangeProperty 1603 1604 func decodeRangeProperties(data []byte) (rangeProperties, error) { 1605 r := make(rangeProperties, 0, 16) 1606 for len(data) > 0 { 1607 if len(data) < 4 { 1608 return nil, io.ErrUnexpectedEOF 1609 } 1610 keyLen := int(binary.BigEndian.Uint32(data[:4])) 1611 data = data[4:] 1612 if len(data) < keyLen+8*2 { 1613 return nil, io.ErrUnexpectedEOF 1614 } 1615 key := data[:keyLen] 1616 data = data[keyLen:] 1617 size := binary.BigEndian.Uint64(data[:8]) 1618 keys := binary.BigEndian.Uint64(data[8:]) 1619 data = data[16:] 1620 r = append(r, rangeProperty{Key: key, rangeOffsets: rangeOffsets{Size: size, Keys: keys}}) 1621 } 1622 1623 return r, nil 1624 } 1625 1626 func (r rangeProperties) Encode() []byte { 1627 b := make([]byte, 0, 1024) 1628 idx := 0 1629 for _, p := range r { 1630 b = append(b, 0, 0, 0, 0) 1631 binary.BigEndian.PutUint32(b[idx:], uint32(len(p.Key))) 1632 idx += 4 1633 b = append(b, p.Key...) 1634 idx += len(p.Key) 1635 1636 b = append(b, 0, 0, 0, 0, 0, 0, 0, 0) 1637 binary.BigEndian.PutUint64(b[idx:], p.Size) 1638 idx += 8 1639 1640 b = append(b, 0, 0, 0, 0, 0, 0, 0, 0) 1641 binary.BigEndian.PutUint64(b[idx:], p.Keys) 1642 idx += 8 1643 } 1644 return b 1645 } 1646 1647 func (r rangeProperties) get(key []byte) rangeOffsets { 1648 idx := sort.Search(len(r), func(i int) bool { 1649 return bytes.Compare(r[i].Key, key) >= 0 1650 }) 1651 return r[idx].rangeOffsets 1652 } 1653 1654 type RangePropertiesCollector struct { 1655 props rangeProperties 1656 lastOffsets rangeOffsets 1657 lastKey []byte 1658 currentOffsets rangeOffsets 1659 propSizeIdxDistance uint64 1660 propKeysIdxDistance uint64 1661 } 1662 1663 func newRangePropertiesCollector() pebble.TablePropertyCollector { 1664 return &RangePropertiesCollector{ 1665 props: make([]rangeProperty, 0, 1024), 1666 propSizeIdxDistance: defaultPropSizeIndexDistance, 1667 propKeysIdxDistance: defaultPropKeysIndexDistance, 1668 } 1669 } 1670 1671 func (c *RangePropertiesCollector) sizeInLastRange() uint64 { 1672 return c.currentOffsets.Size - c.lastOffsets.Size 1673 } 1674 1675 func (c *RangePropertiesCollector) keysInLastRange() uint64 { 1676 return c.currentOffsets.Keys - c.lastOffsets.Keys 1677 } 1678 1679 func (c *RangePropertiesCollector) insertNewPoint(key []byte) { 1680 c.lastOffsets = c.currentOffsets 1681 c.props = append(c.props, rangeProperty{Key: append([]byte{}, key...), rangeOffsets: c.currentOffsets}) 1682 } 1683 1684 // implement `pebble.TablePropertyCollector` 1685 // implement `TablePropertyCollector.Add` 1686 func (c *RangePropertiesCollector) Add(key pebble.InternalKey, value []byte) error { 1687 c.currentOffsets.Size += uint64(len(value)) + uint64(len(key.UserKey)) 1688 c.currentOffsets.Keys += 1 1689 if len(c.lastKey) == 0 || c.sizeInLastRange() >= c.propSizeIdxDistance || 1690 c.keysInLastRange() >= c.propKeysIdxDistance { 1691 c.insertNewPoint(key.UserKey) 1692 } 1693 c.lastKey = append(c.lastKey[:0], key.UserKey...) 1694 return nil 1695 } 1696 1697 func (c *RangePropertiesCollector) Finish(userProps map[string]string) error { 1698 if c.sizeInLastRange() > 0 || c.keysInLastRange() > 0 { 1699 c.insertNewPoint(c.lastKey) 1700 } 1701 1702 userProps[propRangeIndex] = string(c.props.Encode()) 1703 return nil 1704 } 1705 1706 // The name of the property collector. 1707 func (c *RangePropertiesCollector) Name() string { 1708 return propRangeIndex 1709 } 1710 1711 type sizeProperties struct { 1712 totalSize uint64 1713 indexHandles *btree.BTree 1714 } 1715 1716 func newSizeProperties() *sizeProperties { 1717 return &sizeProperties{indexHandles: btree.New(32)} 1718 } 1719 1720 func (s *sizeProperties) add(item *rangeProperty) { 1721 if old := s.indexHandles.ReplaceOrInsert(item); old != nil { 1722 o := old.(*rangeProperty) 1723 item.Keys += o.Keys 1724 item.Size += o.Size 1725 } 1726 } 1727 1728 func (s *sizeProperties) addAll(props rangeProperties) { 1729 prevRange := rangeOffsets{} 1730 for _, r := range props { 1731 s.add(&rangeProperty{ 1732 Key: r.Key, 1733 rangeOffsets: rangeOffsets{Keys: r.Keys - prevRange.Keys, Size: r.Size - prevRange.Size}, 1734 }) 1735 prevRange = r.rangeOffsets 1736 } 1737 if len(props) > 0 { 1738 s.totalSize = props[len(props)-1].Size 1739 } 1740 } 1741 1742 // iter the tree until f return false 1743 func (s *sizeProperties) iter(f func(p *rangeProperty) bool) { 1744 s.indexHandles.Ascend(func(i btree.Item) bool { 1745 prop := i.(*rangeProperty) 1746 return f(prop) 1747 }) 1748 } 1749 1750 func (local *local) EngineFileSizes() (res []EngineFileSize) { 1751 local.engines.Range(func(k, v interface{}) bool { 1752 engine := v.(*LocalFile) 1753 res = append(res, engine.getEngineFileSize()) 1754 return true 1755 }) 1756 return 1757 } 1758 1759 type LocalWriter struct { 1760 writeErr common.OnceError 1761 local *LocalFile 1762 consumeCh chan struct{} 1763 kvsChan chan []common.KvPair 1764 flushChMutex sync.RWMutex 1765 flushCh chan chan error 1766 sstDir string 1767 memtableSizeLimit int64 1768 writeBatch kvMemCache 1769 writer *sstWriter 1770 } 1771 1772 func (w *LocalWriter) AppendRows(ctx context.Context, tableName string, columnNames []string, ts uint64, rows Rows) error { 1773 kvs := rows.(kvPairs) 1774 if len(kvs) == 0 { 1775 return nil 1776 } 1777 if err := w.writeErr.Get(); err != nil { 1778 return err 1779 } 1780 w.kvsChan <- kvs 1781 w.local.Ts = ts 1782 return nil 1783 } 1784 1785 func (w *LocalWriter) Close() error { 1786 w.local.localWriters.Delete(w) 1787 close(w.kvsChan) 1788 1789 w.flushChMutex.Lock() 1790 flushCh := w.flushCh 1791 w.flushCh = nil 1792 w.flushChMutex.Unlock() 1793 1794 // after closing kvsChan, the writeRowsLoop will ingest all cached KVs. 1795 // during this time, the flushCh might still be receiving data. 1796 // so we have this extra loop to immediately consume them to avoid AsyncFlush 1797 for { 1798 select { 1799 case <-w.consumeCh: 1800 return w.writeErr.Get() 1801 case replyErrCh := <-flushCh: 1802 replyErrCh <- nil 1803 } 1804 } 1805 } 1806 1807 func (w *LocalWriter) genSSTPath() string { 1808 return filepath.Join(w.sstDir, uuid.New().String()+".sst") 1809 } 1810 1811 func (w *LocalWriter) writeRowsLoop() { 1812 defer func() { 1813 if w.writer != nil { 1814 w.writer.cleanUp() 1815 w.writer = nil 1816 } 1817 w.consumeCh <- struct{}{} 1818 }() 1819 var err error 1820 outside: 1821 for { 1822 w.flushChMutex.RLock() 1823 flushCh := w.flushCh 1824 w.flushChMutex.RUnlock() 1825 1826 select { 1827 case kvs, ok := <-w.kvsChan: 1828 if !ok { 1829 break outside 1830 } 1831 1832 w.writeBatch.append(kvs) 1833 if w.writeBatch.totalSize <= w.memtableSizeLimit { 1834 break 1835 } 1836 if w.writer == nil { 1837 w.writer, err = newSSTWriter(w.genSSTPath()) 1838 if err != nil { 1839 w.writeErr.Set(err) 1840 return 1841 } 1842 } 1843 1844 if err = w.writeKVsOrIngest(0); err != nil { 1845 w.writeErr.Set(err) 1846 return 1847 } 1848 1849 case replyErrCh := <-flushCh: 1850 err = w.writeKVsOrIngest(localIngestDescriptionFlushed) 1851 if w.writer != nil { 1852 err = w.writer.ingestInto(w.local, localIngestDescriptionFlushed) 1853 if err == nil { 1854 err = w.writer.reopen() 1855 } 1856 } 1857 replyErrCh <- err 1858 if err != nil { 1859 w.writeErr.Set(err) 1860 return 1861 } 1862 } 1863 } 1864 1865 if err = w.writeKVsOrIngest(0); err != nil { 1866 w.writeErr.Set(err) 1867 return 1868 } 1869 if w.writer != nil { 1870 if err := w.writer.ingestInto(w.local, 0); err != nil { 1871 w.writeErr.Set(err) 1872 } 1873 } 1874 } 1875 1876 func (w *LocalWriter) writeKVsOrIngest(desc localIngestDescription) error { 1877 if w.writer != nil { 1878 if err := w.writer.writeKVs(&w.writeBatch); err != errorUnorderedSSTInsertion { 1879 return err 1880 } 1881 } 1882 1883 // if write failed only because of unorderedness, we immediately ingest the memcache. 1884 immWriter, err := newSSTWriter(w.genSSTPath()) 1885 if err != nil { 1886 return err 1887 } 1888 defer immWriter.cleanUp() 1889 1890 if err = immWriter.writeKVs(&w.writeBatch); err != nil { 1891 return err 1892 } 1893 1894 return immWriter.ingestInto(w.local, desc|localIngestDescriptionImmediate) 1895 } 1896 1897 var errorUnorderedSSTInsertion = errors.New("inserting KVs into SST without order") 1898 1899 type localIngestDescription uint8 1900 1901 const ( 1902 localIngestDescriptionFlushed localIngestDescription = 1 << iota 1903 localIngestDescriptionImmediate 1904 ) 1905 1906 type sstWriter struct { 1907 path string 1908 writer *sstable.Writer 1909 lastKey []byte 1910 totalSize int64 1911 totalCount int64 1912 } 1913 1914 func newSSTWriter(path string) (*sstWriter, error) { 1915 sw := &sstWriter{path: path} 1916 if err := sw.reopen(); err != nil { 1917 return nil, err 1918 } 1919 return sw, nil 1920 } 1921 1922 // writeKVs moves the KV pairs in the cache into the SST writer. 1923 // On success, the cache will be cleared. 1924 func (sw *sstWriter) writeKVs(m *kvMemCache) error { 1925 if len(m.kvs) == 0 { 1926 return nil 1927 } 1928 m.sort() 1929 if bytes.Compare(m.kvs[0].Key, sw.lastKey) <= 0 { 1930 return errorUnorderedSSTInsertion 1931 } 1932 1933 internalKey := sstable.InternalKey{ 1934 Trailer: uint64(sstable.InternalKeyKindSet), 1935 } 1936 for _, p := range m.kvs { 1937 internalKey.UserKey = p.Key 1938 if err := sw.writer.Add(internalKey, p.Val); err != nil { 1939 return errors.Trace(err) 1940 } 1941 } 1942 sw.totalSize += m.totalSize 1943 sw.totalCount += int64(len(m.kvs)) 1944 sw.lastKey = m.kvs[len(m.kvs)-1].Key 1945 m.clear() 1946 return nil 1947 } 1948 1949 // ingestInto finishes the SST file, and ingests itself into the target LocalFile database. 1950 // On success, the entire writer will be reset as empty. 1951 func (sw *sstWriter) ingestInto(e *LocalFile, desc localIngestDescription) error { 1952 if sw.totalCount > 0 { 1953 if err := sw.writer.Close(); err != nil { 1954 return errors.Trace(err) 1955 } 1956 if desc&localIngestDescriptionFlushed == 0 { 1957 // No need to acquire lock around ingestion when flushing. 1958 // we already held the lock before flushing. 1959 e.lock(importMutexStateLocalIngest) 1960 defer e.unlock() 1961 } 1962 meta, _ := sw.writer.Metadata() // this method returns error only if it has not been closed yet. 1963 log.L().Info("write data to local DB", 1964 zap.Int64("size", sw.totalSize), 1965 zap.Int64("kvs", sw.totalCount), 1966 zap.Uint8("description", uint8(desc)), 1967 zap.Uint64("sstFileSize", meta.Size), 1968 log.ZapRedactBinary("firstKey", meta.SmallestPoint.UserKey), 1969 log.ZapRedactBinary("lastKey", meta.LargestPoint.UserKey)) 1970 1971 if err := e.db.Ingest([]string{sw.path}); err != nil { 1972 return errors.Trace(err) 1973 } 1974 e.TotalSize.Add(sw.totalSize) 1975 e.Length.Add(sw.totalCount) 1976 sw.totalSize = 0 1977 sw.totalCount = 0 1978 sw.lastKey = nil 1979 } 1980 sw.writer = nil 1981 return nil 1982 } 1983 1984 // reopen creates a new SST file after ingestInto is successful. 1985 // Returns error if the SST file was not ingested. 1986 func (sw *sstWriter) reopen() error { 1987 if sw.writer != nil { 1988 return errors.New("cannot reopen an SST writer without ingesting it first") 1989 } 1990 f, err := os.Create(sw.path) 1991 if err != nil { 1992 return errors.Trace(err) 1993 } 1994 sw.writer = sstable.NewWriter(f, sstable.WriterOptions{ 1995 TablePropertyCollectors: []func() pebble.TablePropertyCollector{ 1996 newRangePropertiesCollector, 1997 }, 1998 BlockSize: 16 * 1024, 1999 }) 2000 return nil 2001 } 2002 2003 // cleanUp removes any un-ingested SST file. 2004 func (sw *sstWriter) cleanUp() { 2005 if sw.writer != nil { 2006 sw.writer.Close() 2007 os.Remove(sw.path) 2008 } 2009 } 2010 2011 // kvMemCache is an array of KV pairs. It also keep tracks of the total KV size and whether the array is already sorted. 2012 type kvMemCache struct { 2013 kvs []common.KvPair 2014 totalSize int64 2015 notSorted bool // record "not sorted" instead of "sorted" so that the zero value is correct. 2016 } 2017 2018 // append more KV pairs to the kvMemCache. 2019 func (m *kvMemCache) append(kvs []common.KvPair) { 2020 if !m.notSorted { 2021 var lastKey []byte 2022 if len(m.kvs) > 0 { 2023 lastKey = m.kvs[len(m.kvs)-1].Key 2024 } 2025 for _, kv := range kvs { 2026 if bytes.Compare(kv.Key, lastKey) <= 0 { 2027 m.notSorted = true 2028 break 2029 } 2030 lastKey = kv.Key 2031 } 2032 } 2033 2034 m.kvs = append(m.kvs, kvs...) 2035 for _, kv := range kvs { 2036 m.totalSize += int64(len(kv.Key)) + int64(len(kv.Val)) 2037 } 2038 } 2039 2040 // sort ensures the content is actually sorted. 2041 func (m *kvMemCache) sort() { 2042 if m.notSorted { 2043 sort.Slice(m.kvs, func(i, j int) bool { return bytes.Compare(m.kvs[i].Key, m.kvs[j].Key) < 0 }) 2044 m.notSorted = false 2045 } 2046 } 2047 2048 // clear resets the cache to contain nothing. 2049 func (m *kvMemCache) clear() { 2050 m.kvs = m.kvs[:0] 2051 m.totalSize = 0 2052 m.notSorted = false 2053 }