github.com/pingcap/br@v5.3.0-alpha.0.20220125034240-ec59c7b6ce30+incompatible/pkg/lightning/backend/local/duplicate.go (about) 1 // Copyright 2021 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 local 15 16 import ( 17 "bytes" 18 "context" 19 "io" 20 "sort" 21 "time" 22 23 "golang.org/x/sync/errgroup" 24 25 "github.com/cockroachdb/pebble" 26 "github.com/pingcap/errors" 27 "github.com/pingcap/kvproto/pkg/import_sstpb" 28 "github.com/pingcap/kvproto/pkg/kvrpcpb" 29 "github.com/pingcap/kvproto/pkg/metapb" 30 "github.com/pingcap/kvproto/pkg/tikvpb" 31 "github.com/pingcap/parser/model" 32 "github.com/pingcap/parser/mysql" 33 "github.com/pingcap/tidb/distsql" 34 tidbkv "github.com/pingcap/tidb/kv" 35 "github.com/pingcap/tidb/table" 36 "github.com/pingcap/tidb/util/codec" 37 "github.com/pingcap/tidb/util/ranger" 38 "go.uber.org/zap" 39 "google.golang.org/grpc" 40 "google.golang.org/grpc/backoff" 41 "google.golang.org/grpc/credentials" 42 "google.golang.org/grpc/keepalive" 43 44 "github.com/pingcap/br/pkg/lightning/backend/kv" 45 "github.com/pingcap/br/pkg/lightning/common" 46 "github.com/pingcap/br/pkg/lightning/log" 47 "github.com/pingcap/br/pkg/logutil" 48 "github.com/pingcap/br/pkg/restore" 49 ) 50 51 const ( 52 maxWriteBatchCount = 128 53 maxGetRequestKeyCount = 1024 54 ) 55 56 type DuplicateRequest struct { 57 tableID int64 58 start tidbkv.Key 59 end tidbkv.Key 60 indexInfo *model.IndexInfo 61 } 62 63 type DuplicateManager struct { 64 // TODO: Remote the member `db` and store the result in another place. 65 db *pebble.DB 66 splitCli restore.SplitClient 67 regionConcurrency int 68 connPool common.GRPCConns 69 tls *common.TLS 70 ts uint64 71 keyAdapter KeyAdapter 72 } 73 74 func NewDuplicateManager( 75 db *pebble.DB, 76 splitCli restore.SplitClient, 77 ts uint64, 78 tls *common.TLS, 79 regionConcurrency int) (*DuplicateManager, error) { 80 return &DuplicateManager{ 81 db: db, 82 tls: tls, 83 regionConcurrency: regionConcurrency, 84 splitCli: splitCli, 85 keyAdapter: duplicateKeyAdapter{}, 86 ts: ts, 87 connPool: common.NewGRPCConns(), 88 }, nil 89 } 90 91 func (manager *DuplicateManager) CollectDuplicateRowsFromTiKV(ctx context.Context, tbl table.Table) error { 92 log.L().Info("Begin collect duplicate data from remote TiKV") 93 reqs, err := buildDuplicateRequests(tbl.Meta()) 94 if err != nil { 95 return err 96 } 97 98 decoder, err := kv.NewTableKVDecoder(tbl, &kv.SessionOptions{ 99 SQLMode: mysql.ModeStrictAllTables, 100 }) 101 if err != nil { 102 return err 103 } 104 g, rpcctx := errgroup.WithContext(ctx) 105 for _, r := range reqs { 106 req := r 107 g.Go(func() error { 108 err := manager.sendRequestToTiKV(rpcctx, decoder, req) 109 if err != nil { 110 log.L().Error("error occur when collect duplicate data from TiKV", zap.Error(err)) 111 } 112 return err 113 }) 114 } 115 err = g.Wait() 116 log.L().Info("End collect duplicate data from remote TiKV") 117 return err 118 } 119 120 func (manager *DuplicateManager) sendRequestToTiKV(ctx context.Context, 121 decoder *kv.TableKVDecoder, 122 req *DuplicateRequest) error { 123 startKey := codec.EncodeBytes([]byte{}, req.start) 124 endKey := codec.EncodeBytes([]byte{}, req.end) 125 126 regions, err := paginateScanRegion(ctx, manager.splitCli, startKey, endKey, scanRegionLimit) 127 if err != nil { 128 return err 129 } 130 tryTimes := 0 131 indexHandles := make([][]byte, 0) 132 for { 133 if len(regions) == 0 { 134 break 135 } 136 if tryTimes > maxRetryTimes { 137 return errors.Errorf("retry time exceed limit") 138 } 139 unfinishedRegions := make([]*restore.RegionInfo, 0) 140 waitingClients := make([]import_sstpb.ImportSST_DuplicateDetectClient, 0) 141 watingRegions := make([]*restore.RegionInfo, 0) 142 for idx, region := range regions { 143 if len(waitingClients) > manager.regionConcurrency { 144 r := regions[idx:] 145 unfinishedRegions = append(unfinishedRegions, r...) 146 break 147 } 148 _, start, _ := codec.DecodeBytes(region.Region.StartKey, []byte{}) 149 _, end, _ := codec.DecodeBytes(region.Region.EndKey, []byte{}) 150 if bytes.Compare(startKey, region.Region.StartKey) > 0 { 151 start = req.start 152 } 153 if region.Region.EndKey == nil || len(region.Region.EndKey) == 0 || bytes.Compare(endKey, region.Region.EndKey) < 0 { 154 end = req.end 155 } 156 157 cli, err := manager.getDuplicateStream(ctx, region, start, end) 158 if err != nil { 159 r, err := manager.splitCli.GetRegionByID(ctx, region.Region.GetId()) 160 if err != nil { 161 unfinishedRegions = append(unfinishedRegions, region) 162 } else { 163 unfinishedRegions = append(unfinishedRegions, r) 164 } 165 } else { 166 waitingClients = append(waitingClients, cli) 167 watingRegions = append(watingRegions, region) 168 } 169 } 170 171 if len(indexHandles) > 0 { 172 handles := manager.getValues(ctx, indexHandles) 173 if len(handles) > 0 { 174 indexHandles = handles 175 } else { 176 indexHandles = indexHandles[:0] 177 } 178 } 179 180 for idx, cli := range waitingClients { 181 region := watingRegions[idx] 182 for { 183 resp, reqErr := cli.Recv() 184 hasErr := false 185 if reqErr != nil { 186 if errors.Cause(reqErr) == io.EOF { 187 break 188 } 189 hasErr = true 190 } 191 192 if hasErr || resp.GetKeyError() != nil { 193 r, err := manager.splitCli.GetRegionByID(ctx, region.Region.GetId()) 194 if err != nil { 195 unfinishedRegions = append(unfinishedRegions, region) 196 } else { 197 unfinishedRegions = append(unfinishedRegions, r) 198 } 199 } 200 if hasErr { 201 log.L().Warn("meet error when recving duplicate detect response from TiKV, retry again", 202 logutil.Region(region.Region), logutil.Leader(region.Leader), zap.Error(reqErr)) 203 break 204 } 205 if resp.GetKeyError() != nil { 206 log.L().Warn("meet key error in duplicate detect response from TiKV, retry again ", 207 logutil.Region(region.Region), logutil.Leader(region.Leader), 208 zap.String("KeyError", resp.GetKeyError().GetMessage())) 209 break 210 } 211 212 if resp.GetRegionError() != nil { 213 log.L().Warn("meet key error in duplicate detect response from TiKV, retry again ", 214 logutil.Region(region.Region), logutil.Leader(region.Leader), 215 zap.String("RegionError", resp.GetRegionError().GetMessage())) 216 217 r, err := paginateScanRegion(ctx, manager.splitCli, watingRegions[idx].Region.GetStartKey(), watingRegions[idx].Region.GetEndKey(), scanRegionLimit) 218 if err != nil { 219 unfinishedRegions = append(unfinishedRegions, watingRegions[idx]) 220 } else { 221 unfinishedRegions = append(unfinishedRegions, r...) 222 } 223 break 224 } 225 226 handles, err := manager.storeDuplicateData(ctx, resp, decoder, req) 227 if err != nil { 228 return err 229 } 230 if handles != nil && len(handles) > 0 { 231 indexHandles = append(indexHandles, handles...) 232 } 233 } 234 } 235 236 // it means that all of region send to TiKV fail, so we must sleep some time to avoid retry too frequency 237 if len(unfinishedRegions) == len(regions) { 238 tryTimes += 1 239 time.Sleep(defaultRetryBackoffTime) 240 } 241 regions = unfinishedRegions 242 } 243 return nil 244 } 245 246 func (manager *DuplicateManager) storeDuplicateData( 247 ctx context.Context, 248 resp *import_sstpb.DuplicateDetectResponse, 249 decoder *kv.TableKVDecoder, 250 req *DuplicateRequest, 251 ) ([][]byte, error) { 252 opts := &pebble.WriteOptions{Sync: false} 253 var err error 254 maxKeyLen := 0 255 for _, kv := range resp.Pairs { 256 l := manager.keyAdapter.EncodedLen(kv.Key) 257 if l > maxKeyLen { 258 maxKeyLen = l 259 } 260 } 261 buf := make([]byte, maxKeyLen) 262 for i := 0; i < maxRetryTimes; i++ { 263 b := manager.db.NewBatch() 264 handles := make([][]byte, 0) 265 for _, kv := range resp.Pairs { 266 if req.indexInfo != nil { 267 h, err := decoder.DecodeHandleFromIndex(req.indexInfo, kv.Key, kv.Value) 268 if err != nil { 269 log.L().Error("decode handle error from index", 270 zap.Error(err), logutil.Key("key", kv.Key), 271 logutil.Key("value", kv.Value), zap.Uint64("commit-ts", kv.CommitTs)) 272 continue 273 } 274 key := decoder.EncodeHandleKey(h) 275 handles = append(handles, key) 276 } else { 277 encodedKey := manager.keyAdapter.Encode(buf, kv.Key, 0, int64(kv.CommitTs)) 278 b.Set(encodedKey, kv.Value, opts) 279 } 280 } 281 err = b.Commit(opts) 282 if err != nil { 283 continue 284 } 285 b.Close() 286 if len(handles) == 0 { 287 return handles, nil 288 } 289 return manager.getValues(ctx, handles), nil 290 } 291 return nil, err 292 } 293 294 func (manager *DuplicateManager) ReportDuplicateData() error { 295 return nil 296 } 297 298 func (manager *DuplicateManager) RepairDuplicateData() error { 299 // TODO 300 return nil 301 } 302 303 // Collect rows by read the index in db. 304 func (manager *DuplicateManager) CollectDuplicateRowsFromLocalIndex( 305 ctx context.Context, 306 tbl table.Table, 307 db *pebble.DB, 308 ) error { 309 decoder, err := kv.NewTableKVDecoder(tbl, &kv.SessionOptions{ 310 SQLMode: mysql.ModeStrictAllTables, 311 }) 312 if err != nil { 313 return err 314 } 315 handles := make([][]byte, 0) 316 allRanges := make([]tidbkv.KeyRange, 0) 317 for _, indexInfo := range tbl.Meta().Indices { 318 if indexInfo.State != model.StatePublic { 319 continue 320 } 321 ranges := ranger.FullRange() 322 keysRanges, err := distsql.IndexRangesToKVRanges(nil, tbl.Meta().ID, indexInfo.ID, ranges, nil) 323 if err != nil { 324 return err 325 } 326 allRanges = append(allRanges, keysRanges...) 327 for _, r := range keysRanges { 328 startKey := codec.EncodeBytes([]byte{}, r.StartKey) 329 endKey := codec.EncodeBytes([]byte{}, r.EndKey) 330 opts := &pebble.IterOptions{ 331 LowerBound: startKey, 332 UpperBound: endKey, 333 } 334 log.L().Warn("collect index from db", 335 logutil.Key("start", startKey), 336 logutil.Key("end", endKey), 337 ) 338 339 iter := db.NewIter(opts) 340 for iter.SeekGE(startKey); iter.Valid(); iter.Next() { 341 rawKey, _, _, err := manager.keyAdapter.Decode(nil, iter.Key()) 342 if err != nil { 343 log.L().Warn( 344 "decode key error when query handle for duplicate index", 345 zap.Binary("key", iter.Key()), 346 ) 347 continue 348 } 349 value := iter.Value() 350 h, err := decoder.DecodeHandleFromIndex(indexInfo, rawKey, value) 351 if err != nil { 352 log.L().Error("decode handle error from index for duplicatedb", 353 zap.Error(err), logutil.Key("rawKey", rawKey), 354 logutil.Key("value", value)) 355 continue 356 } 357 key := decoder.EncodeHandleKey(h) 358 handles = append(handles, key) 359 if len(handles) > maxGetRequestKeyCount { 360 handles = manager.getValues(ctx, handles) 361 } 362 } 363 if len(handles) > 0 { 364 handles = manager.getValues(ctx, handles) 365 } 366 if len(handles) == 0 { 367 db.DeleteRange(r.StartKey, r.EndKey, &pebble.WriteOptions{Sync: false}) 368 } 369 iter.Close() 370 } 371 } 372 if len(handles) == 0 { 373 return nil 374 } 375 376 for i := 0; i < maxRetryTimes; i++ { 377 handles = manager.getValues(ctx, handles) 378 if len(handles) == 0 { 379 for _, r := range allRanges { 380 db.DeleteRange(r.StartKey, r.EndKey, &pebble.WriteOptions{Sync: false}) 381 } 382 } 383 } 384 return errors.Errorf("retry getValues time exceed limit") 385 } 386 387 func (manager *DuplicateManager) getValues( 388 ctx context.Context, 389 handles [][]byte, 390 ) [][]byte { 391 retryHandles := make([][]byte, 0) 392 sort.Slice(handles, func(i, j int) bool { 393 return bytes.Compare(handles[i], handles[j]) < 0 394 }) 395 l := len(handles) 396 startKey := codec.EncodeBytes([]byte{}, handles[0]) 397 endKey := codec.EncodeBytes([]byte{}, nextKey(handles[l-1])) 398 regions, err := paginateScanRegion(ctx, manager.splitCli, startKey, endKey, scanRegionLimit) 399 if err != nil { 400 log.L().Error("scan regions errors", zap.Error(err)) 401 return handles 402 } 403 startIdx := 0 404 endIdx := 0 405 batch := make([][]byte, 0) 406 for _, region := range regions { 407 if startIdx >= l { 408 break 409 } 410 handleKey := codec.EncodeBytes([]byte{}, handles[startIdx]) 411 if bytes.Compare(handleKey, region.Region.EndKey) >= 0 { 412 continue 413 } 414 endIdx = startIdx 415 for endIdx < l { 416 handleKey := codec.EncodeBytes([]byte{}, handles[endIdx]) 417 if bytes.Compare(handleKey, region.Region.EndKey) < 0 { 418 batch = append(batch, handles[endIdx]) 419 endIdx++ 420 } else { 421 break 422 } 423 } 424 if err := manager.getValuesFromRegion(ctx, region, batch); err != nil { 425 log.L().Error("failed to collect values from TiKV by handle, we will retry it again", zap.Error(err)) 426 retryHandles = append(retryHandles, batch...) 427 } 428 startIdx = endIdx 429 } 430 return retryHandles 431 } 432 433 func (manager *DuplicateManager) getValuesFromRegion( 434 ctx context.Context, 435 region *restore.RegionInfo, 436 handles [][]byte, 437 ) error { 438 kvclient, err := manager.getKvClient(ctx, region.Leader) 439 if err != nil { 440 return err 441 } 442 reqCtx := &kvrpcpb.Context{ 443 RegionId: region.Region.GetId(), 444 RegionEpoch: region.Region.GetRegionEpoch(), 445 Peer: region.Leader, 446 } 447 448 req := &kvrpcpb.BatchGetRequest{ 449 Context: reqCtx, 450 Keys: handles, 451 Version: manager.ts, 452 } 453 resp, err := kvclient.KvBatchGet(ctx, req) 454 if err != nil { 455 return err 456 } 457 if resp.GetRegionError() != nil { 458 return errors.Errorf("region error because of %v", resp.GetRegionError().GetMessage()) 459 } 460 if resp.Error != nil { 461 return errors.Errorf("key error") 462 } 463 464 maxKeyLen := 0 465 for _, kv := range resp.Pairs { 466 l := manager.keyAdapter.EncodedLen(kv.Key) 467 if l > maxKeyLen { 468 maxKeyLen = l 469 } 470 } 471 buf := make([]byte, maxKeyLen) 472 473 log.L().Error("get keys", zap.Int("key size", len(resp.Pairs))) 474 for i := 0; i < maxRetryTimes; i++ { 475 b := manager.db.NewBatch() 476 opts := &pebble.WriteOptions{Sync: false} 477 for _, kv := range resp.Pairs { 478 encodedKey := manager.keyAdapter.Encode(buf, kv.Key, 0, 0) 479 b.Set(encodedKey, kv.Value, opts) 480 if b.Count() > maxWriteBatchCount { 481 err = b.Commit(opts) 482 if err != nil { 483 break 484 } else { 485 b.Reset() 486 } 487 } 488 } 489 if err == nil { 490 err = b.Commit(opts) 491 } 492 if err == nil { 493 return nil 494 } 495 } 496 return err 497 } 498 499 func (manager *DuplicateManager) getDuplicateStream(ctx context.Context, 500 region *restore.RegionInfo, 501 start []byte, end []byte) (import_sstpb.ImportSST_DuplicateDetectClient, error) { 502 leader := region.Leader 503 if leader == nil { 504 leader = region.Region.GetPeers()[0] 505 } 506 507 cli, err := manager.getImportClient(ctx, leader) 508 if err != nil { 509 return nil, err 510 } 511 512 reqCtx := &kvrpcpb.Context{ 513 RegionId: region.Region.GetId(), 514 RegionEpoch: region.Region.GetRegionEpoch(), 515 Peer: leader, 516 } 517 req := &import_sstpb.DuplicateDetectRequest{ 518 Context: reqCtx, 519 StartKey: start, 520 EndKey: end, 521 KeyOnly: false, 522 } 523 stream, err := cli.DuplicateDetect(ctx, req) 524 return stream, err 525 } 526 527 func (manager *DuplicateManager) getKvClient(ctx context.Context, peer *metapb.Peer) (tikvpb.TikvClient, error) { 528 conn, err := manager.connPool.GetGrpcConn(ctx, peer.GetStoreId(), 1, func(ctx context.Context) (*grpc.ClientConn, error) { 529 return manager.makeConn(ctx, peer.GetStoreId()) 530 }) 531 if err != nil { 532 return nil, err 533 } 534 return tikvpb.NewTikvClient(conn), nil 535 } 536 537 func (manager *DuplicateManager) getImportClient(ctx context.Context, peer *metapb.Peer) (import_sstpb.ImportSSTClient, error) { 538 conn, err := manager.connPool.GetGrpcConn(ctx, peer.GetStoreId(), 1, func(ctx context.Context) (*grpc.ClientConn, error) { 539 return manager.makeConn(ctx, peer.GetStoreId()) 540 }) 541 if err != nil { 542 return nil, err 543 } 544 return import_sstpb.NewImportSSTClient(conn), nil 545 } 546 547 func (manager *DuplicateManager) makeConn(ctx context.Context, storeID uint64) (*grpc.ClientConn, error) { 548 store, err := manager.splitCli.GetStore(ctx, storeID) 549 if err != nil { 550 return nil, errors.Trace(err) 551 } 552 opt := grpc.WithInsecure() 553 if manager.tls.TLSConfig() != nil { 554 opt = grpc.WithTransportCredentials(credentials.NewTLS(manager.tls.TLSConfig())) 555 } 556 ctx, cancel := context.WithTimeout(ctx, dialTimeout) 557 558 bfConf := backoff.DefaultConfig 559 bfConf.MaxDelay = gRPCBackOffMaxDelay 560 // we should use peer address for tiflash. for tikv, peer address is empty 561 addr := store.GetPeerAddress() 562 if addr == "" { 563 addr = store.GetAddress() 564 } 565 conn, err := grpc.DialContext( 566 ctx, 567 addr, 568 opt, 569 grpc.WithConnectParams(grpc.ConnectParams{Backoff: bfConf}), 570 grpc.WithKeepaliveParams(keepalive.ClientParameters{ 571 Time: gRPCKeepAliveTime, 572 Timeout: gRPCKeepAliveTimeout, 573 PermitWithoutStream: true, 574 }), 575 ) 576 cancel() 577 if err != nil { 578 return nil, errors.Trace(err) 579 } 580 return conn, nil 581 } 582 583 func buildDuplicateRequests(tableInfo *model.TableInfo) ([]*DuplicateRequest, error) { 584 reqs := make([]*DuplicateRequest, 0) 585 req := buildTableRequest(tableInfo.ID) 586 reqs = append(reqs, req...) 587 for _, indexInfo := range tableInfo.Indices { 588 if indexInfo.State != model.StatePublic { 589 continue 590 } 591 req, err := buildIndexRequest(tableInfo.ID, indexInfo) 592 if err != nil { 593 return nil, err 594 } 595 reqs = append(reqs, req...) 596 } 597 return reqs, nil 598 } 599 600 func buildTableRequest(tableID int64) []*DuplicateRequest { 601 ranges := ranger.FullIntRange(false) 602 keysRanges := distsql.TableRangesToKVRanges(tableID, ranges, nil) 603 reqs := make([]*DuplicateRequest, 0) 604 for _, r := range keysRanges { 605 r := &DuplicateRequest{ 606 start: r.StartKey, 607 end: r.EndKey, 608 tableID: tableID, 609 indexInfo: nil, 610 } 611 reqs = append(reqs, r) 612 } 613 return reqs 614 } 615 616 func buildIndexRequest(tableID int64, indexInfo *model.IndexInfo) ([]*DuplicateRequest, error) { 617 ranges := ranger.FullRange() 618 keysRanges, err := distsql.IndexRangesToKVRanges(nil, tableID, indexInfo.ID, ranges, nil) 619 if err != nil { 620 return nil, err 621 } 622 reqs := make([]*DuplicateRequest, 0) 623 for _, r := range keysRanges { 624 r := &DuplicateRequest{ 625 start: r.StartKey, 626 end: r.EndKey, 627 tableID: tableID, 628 indexInfo: indexInfo, 629 } 630 reqs = append(reqs, r) 631 } 632 return reqs, nil 633 }