github.com/pingcap/ticdc@v0.0.0-20220526033649-485a10ef2652/cdc/kv/client_test.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 kv 15 16 import ( 17 "context" 18 "fmt" 19 "net" 20 "sync" 21 "sync/atomic" 22 "testing" 23 "time" 24 25 "github.com/golang/protobuf/proto" 26 "github.com/pingcap/check" 27 "github.com/pingcap/errors" 28 "github.com/pingcap/failpoint" 29 "github.com/pingcap/kvproto/pkg/cdcpb" 30 "github.com/pingcap/kvproto/pkg/errorpb" 31 "github.com/pingcap/kvproto/pkg/metapb" 32 "github.com/pingcap/log" 33 "github.com/pingcap/ticdc/cdc/model" 34 "github.com/pingcap/ticdc/pkg/config" 35 cerror "github.com/pingcap/ticdc/pkg/errors" 36 "github.com/pingcap/ticdc/pkg/regionspan" 37 "github.com/pingcap/ticdc/pkg/retry" 38 "github.com/pingcap/ticdc/pkg/security" 39 "github.com/pingcap/ticdc/pkg/txnutil" 40 "github.com/pingcap/ticdc/pkg/util" 41 "github.com/pingcap/ticdc/pkg/util/testleak" 42 "github.com/pingcap/tidb/kv" 43 "github.com/pingcap/tidb/store/mockstore/mocktikv" 44 "github.com/pingcap/tidb/store/tikv" 45 "github.com/pingcap/tidb/store/tikv/oracle" 46 "go.uber.org/zap" 47 "google.golang.org/grpc" 48 ) 49 50 func Test(t *testing.T) { 51 conf := config.GetDefaultServerConfig() 52 config.StoreGlobalServerConfig(conf) 53 InitWorkerPool() 54 go func() { 55 RunWorkerPool(context.Background()) //nolint:errcheck 56 }() 57 check.TestingT(t) 58 } 59 60 type clientSuite struct { 61 } 62 63 var _ = check.Suite(&clientSuite{}) 64 65 func (s *clientSuite) TestNewClose(c *check.C) { 66 defer testleak.AfterTest(c)() 67 store := mocktikv.MustNewMVCCStore() 68 defer store.Close() //nolint:errcheck 69 cluster := mocktikv.NewCluster(store) 70 pdCli := mocktikv.NewPDClient(cluster) 71 defer pdCli.Close() //nolint:errcheck 72 73 grpcPool := NewGrpcPoolImpl(context.Background(), &security.Credential{}) 74 defer grpcPool.Close() 75 cli := NewCDCClient(context.Background(), pdCli, nil, grpcPool) 76 err := cli.Close() 77 c.Assert(err, check.IsNil) 78 } 79 80 func (s *clientSuite) TestAssembleRowEvent(c *check.C) { 81 defer testleak.AfterTest(c)() 82 testCases := []struct { 83 regionID uint64 84 entry *cdcpb.Event_Row 85 enableOldValue bool 86 expected model.RegionFeedEvent 87 err string 88 }{{ 89 regionID: 1, 90 entry: &cdcpb.Event_Row{ 91 StartTs: 1, 92 CommitTs: 2, 93 Key: []byte("k1"), 94 Value: []byte("v1"), 95 OpType: cdcpb.Event_Row_PUT, 96 }, 97 enableOldValue: false, 98 expected: model.RegionFeedEvent{ 99 RegionID: 1, 100 Val: &model.RawKVEntry{ 101 OpType: model.OpTypePut, 102 StartTs: 1, 103 CRTs: 2, 104 Key: []byte("k1"), 105 Value: []byte("v1"), 106 RegionID: 1, 107 }, 108 }, 109 }, { 110 regionID: 2, 111 entry: &cdcpb.Event_Row{ 112 StartTs: 1, 113 CommitTs: 2, 114 Key: []byte("k2"), 115 Value: []byte("v2"), 116 OpType: cdcpb.Event_Row_DELETE, 117 }, 118 enableOldValue: false, 119 expected: model.RegionFeedEvent{ 120 RegionID: 2, 121 Val: &model.RawKVEntry{ 122 OpType: model.OpTypeDelete, 123 StartTs: 1, 124 CRTs: 2, 125 Key: []byte("k2"), 126 Value: []byte("v2"), 127 RegionID: 2, 128 }, 129 }, 130 }, { 131 regionID: 3, 132 entry: &cdcpb.Event_Row{ 133 StartTs: 1, 134 CommitTs: 2, 135 Key: []byte("k2"), 136 Value: []byte("v2"), 137 OldValue: []byte("ov2"), 138 OpType: cdcpb.Event_Row_PUT, 139 }, 140 enableOldValue: false, 141 expected: model.RegionFeedEvent{ 142 RegionID: 3, 143 Val: &model.RawKVEntry{ 144 OpType: model.OpTypePut, 145 StartTs: 1, 146 CRTs: 2, 147 Key: []byte("k2"), 148 Value: []byte("v2"), 149 RegionID: 3, 150 }, 151 }, 152 }, { 153 regionID: 4, 154 entry: &cdcpb.Event_Row{ 155 StartTs: 1, 156 CommitTs: 2, 157 Key: []byte("k3"), 158 Value: []byte("v3"), 159 OldValue: []byte("ov3"), 160 OpType: cdcpb.Event_Row_PUT, 161 }, 162 enableOldValue: true, 163 expected: model.RegionFeedEvent{ 164 RegionID: 4, 165 Val: &model.RawKVEntry{ 166 OpType: model.OpTypePut, 167 StartTs: 1, 168 CRTs: 2, 169 Key: []byte("k3"), 170 Value: []byte("v3"), 171 OldValue: []byte("ov3"), 172 RegionID: 4, 173 }, 174 }, 175 }, { 176 regionID: 2, 177 entry: &cdcpb.Event_Row{ 178 StartTs: 1, 179 CommitTs: 2, 180 Key: []byte("k2"), 181 Value: []byte("v2"), 182 OpType: cdcpb.Event_Row_UNKNOWN, 183 }, 184 enableOldValue: false, 185 err: "[CDC:ErrUnknownKVEventType]unknown kv event type: UNKNOWN, entry: start_ts:1 commit_ts:2 key:\"k2\" value:\"v2\" ", 186 }} 187 188 for _, tc := range testCases { 189 event, err := assembleRowEvent(tc.regionID, tc.entry, tc.enableOldValue) 190 c.Assert(event, check.DeepEquals, tc.expected) 191 if err != nil { 192 c.Assert(err.Error(), check.Equals, tc.err) 193 } 194 } 195 } 196 197 type mockChangeDataService struct { 198 c *check.C 199 ch chan *cdcpb.ChangeDataEvent 200 recvLoop func(server cdcpb.ChangeData_EventFeedServer) 201 exitNotify sync.Map 202 eventFeedID uint64 203 } 204 205 func newMockChangeDataService(c *check.C, ch chan *cdcpb.ChangeDataEvent) *mockChangeDataService { 206 s := &mockChangeDataService{ 207 c: c, 208 ch: ch, 209 } 210 return s 211 } 212 213 type notifyCh struct { 214 notify chan struct{} 215 callback chan struct{} 216 } 217 218 func (s *mockChangeDataService) registerExitNotify(id uint64, ch *notifyCh) { 219 s.exitNotify.Store(id, ch) 220 } 221 222 func (s *mockChangeDataService) notifyExit(id uint64) chan struct{} { 223 if ch, ok := s.exitNotify.Load(id); ok { 224 nch := ch.(*notifyCh) 225 nch.notify <- struct{}{} 226 return nch.callback 227 } 228 return nil 229 } 230 231 func (s *mockChangeDataService) EventFeed(server cdcpb.ChangeData_EventFeedServer) error { 232 if s.recvLoop != nil { 233 go func() { 234 s.recvLoop(server) 235 }() 236 } 237 notify := ¬ifyCh{ 238 notify: make(chan struct{}), 239 callback: make(chan struct{}, 1), // callback is not always retrieved 240 } 241 s.registerExitNotify(atomic.LoadUint64(&s.eventFeedID), notify) 242 atomic.AddUint64(&s.eventFeedID, 1) 243 loop: 244 for { 245 select { 246 case e := <-s.ch: 247 if e == nil { 248 break loop 249 } 250 _ = server.Send(e) 251 case <-notify.notify: 252 break loop 253 } 254 } 255 notify.callback <- struct{}{} 256 return nil 257 } 258 259 func newMockService( 260 ctx context.Context, 261 c *check.C, 262 srv cdcpb.ChangeDataServer, 263 wg *sync.WaitGroup, 264 ) (grpcServer *grpc.Server, addr string) { 265 return newMockServiceSpecificAddr(ctx, c, srv, "127.0.0.1:0", wg) 266 } 267 268 func newMockServiceSpecificAddr( 269 ctx context.Context, 270 c *check.C, 271 srv cdcpb.ChangeDataServer, 272 listenAddr string, 273 wg *sync.WaitGroup, 274 ) (grpcServer *grpc.Server, addr string) { 275 lc := &net.ListenConfig{} 276 lis, err := lc.Listen(ctx, "tcp", listenAddr) 277 c.Assert(err, check.IsNil) 278 addr = lis.Addr().String() 279 grpcServer = grpc.NewServer() 280 cdcpb.RegisterChangeDataServer(grpcServer, srv) 281 wg.Add(1) 282 go func() { 283 defer wg.Done() 284 err := grpcServer.Serve(lis) 285 c.Assert(err, check.IsNil) 286 }() 287 return 288 } 289 290 // waitRequestID waits request ID larger than the given allocated ID 291 func waitRequestID(c *check.C, allocatedID uint64) { 292 err := retry.Do(context.Background(), func() error { 293 if currentRequestID() > allocatedID { 294 return nil 295 } 296 return errors.Errorf("request id %d is not larger than %d", currentRequestID(), allocatedID) 297 }, retry.WithBackoffBaseDelay(10), retry.WithMaxTries(20)) 298 299 c.Assert(err, check.IsNil) 300 } 301 302 // Use etcdSuite to workaround the race. See comments of `TestConnArray`. 303 func (s *etcdSuite) TestConnectOfflineTiKV(c *check.C) { 304 defer testleak.AfterTest(c)() 305 defer s.TearDownTest(c) 306 ctx, cancel := context.WithCancel(context.Background()) 307 defer cancel() 308 wg := &sync.WaitGroup{} 309 ch2 := make(chan *cdcpb.ChangeDataEvent, 10) 310 srv := newMockChangeDataService(c, ch2) 311 server2, addr := newMockService(ctx, c, srv, wg) 312 defer func() { 313 close(ch2) 314 server2.Stop() 315 wg.Wait() 316 }() 317 318 rpcClient, cluster, pdClient, err := mocktikv.NewTiKVAndPDClient("") 319 c.Assert(err, check.IsNil) 320 pdClient = &mockPDClient{Client: pdClient, versionGen: defaultVersionGen} 321 tiStore, err := tikv.NewTestTiKVStore(rpcClient, pdClient, nil, nil, 0) 322 c.Assert(err, check.IsNil) 323 kvStorage := newStorageWithCurVersionCache(tiStore, addr) 324 defer kvStorage.Close() //nolint:errcheck 325 326 invalidStore := "localhost:1" 327 cluster.AddStore(1, invalidStore) 328 cluster.AddStore(2, addr) 329 cluster.Bootstrap(3, []uint64{1, 2}, []uint64{4, 5}, 4) 330 331 baseAllocatedID := currentRequestID() 332 lockresolver := txnutil.NewLockerResolver(kvStorage) 333 isPullInit := &mockPullerInit{} 334 grpcPool := NewGrpcPoolImpl(ctx, &security.Credential{}) 335 defer grpcPool.Close() 336 cdcClient := NewCDCClient(context.Background(), pdClient, kvStorage, grpcPool) 337 defer cdcClient.Close() //nolint:errcheck 338 eventCh := make(chan model.RegionFeedEvent, 10) 339 wg.Add(1) 340 go func() { 341 defer wg.Done() 342 err := cdcClient.EventFeed(ctx, regionspan.ComparableSpan{Start: []byte("a"), End: []byte("b")}, 1, false, lockresolver, isPullInit, eventCh) 343 c.Assert(errors.Cause(err), check.Equals, context.Canceled) 344 }() 345 346 // new session, request to store 1, request to store 2 347 waitRequestID(c, baseAllocatedID+2) 348 349 makeEvent := func(ts uint64) *cdcpb.ChangeDataEvent { 350 return &cdcpb.ChangeDataEvent{ 351 Events: []*cdcpb.Event{ 352 { 353 RegionId: 3, 354 RequestId: currentRequestID(), 355 Event: &cdcpb.Event_ResolvedTs{ 356 ResolvedTs: ts, 357 }, 358 }, 359 }, 360 } 361 } 362 363 checkEvent := func(event model.RegionFeedEvent, ts uint64) { 364 c.Assert(event.Resolved.ResolvedTs, check.Equals, ts) 365 } 366 367 initialized := mockInitializedEvent(3 /* regionID */, currentRequestID()) 368 ch2 <- initialized 369 370 cluster.ChangeLeader(3, 5) 371 372 ts, err := kvStorage.CurrentTimestamp(oracle.GlobalTxnScope) 373 ver := kv.NewVersion(ts) 374 c.Assert(err, check.IsNil) 375 ch2 <- makeEvent(ver.Ver) 376 var event model.RegionFeedEvent 377 // consume the first resolved ts event, which is sent before region starts 378 <-eventCh 379 select { 380 case event = <-eventCh: 381 case <-time.After(time.Second): 382 c.Fatalf("reconnection not succeed in 1 second") 383 } 384 checkEvent(event, 1) 385 386 select { 387 case event = <-eventCh: 388 case <-time.After(time.Second): 389 c.Fatalf("reconnection not succeed in 1 second") 390 } 391 checkEvent(event, ver.Ver) 392 393 // check gRPC connection active counter is updated correctly 394 bucket, ok := grpcPool.bucketConns[invalidStore] 395 c.Assert(ok, check.IsTrue) 396 empty := bucket.recycle() 397 c.Assert(empty, check.IsTrue) 398 399 cancel() 400 } 401 402 func (s *etcdSuite) TestRecvLargeMessageSize(c *check.C) { 403 defer testleak.AfterTest(c)() 404 defer s.TearDownTest(c) 405 ctx, cancel := context.WithCancel(context.Background()) 406 wg := &sync.WaitGroup{} 407 ch2 := make(chan *cdcpb.ChangeDataEvent, 10) 408 srv := newMockChangeDataService(c, ch2) 409 server2, addr := newMockService(ctx, c, srv, wg) 410 defer func() { 411 close(ch2) 412 server2.Stop() 413 wg.Wait() 414 }() 415 // Cancel first, and then close the server. 416 defer cancel() 417 418 rpcClient, cluster, pdClient, err := mocktikv.NewTiKVAndPDClient("") 419 c.Assert(err, check.IsNil) 420 pdClient = &mockPDClient{Client: pdClient, versionGen: defaultVersionGen} 421 defer pdClient.Close() //nolint:errcheck 422 tiStore, err := tikv.NewTestTiKVStore(rpcClient, pdClient, nil, nil, 0) 423 c.Assert(err, check.IsNil) 424 kvStorage := newStorageWithCurVersionCache(tiStore, addr) 425 defer kvStorage.Close() //nolint:errcheck 426 427 cluster.AddStore(2, addr) 428 cluster.Bootstrap(3, []uint64{2}, []uint64{4}, 4) 429 430 baseAllocatedID := currentRequestID() 431 lockresolver := txnutil.NewLockerResolver(kvStorage) 432 isPullInit := &mockPullerInit{} 433 grpcPool := NewGrpcPoolImpl(ctx, &security.Credential{}) 434 defer grpcPool.Close() 435 cdcClient := NewCDCClient(ctx, pdClient, kvStorage, grpcPool) 436 eventCh := make(chan model.RegionFeedEvent, 10) 437 wg.Add(1) 438 go func() { 439 defer wg.Done() 440 err := cdcClient.EventFeed(ctx, regionspan.ComparableSpan{Start: []byte("a"), End: []byte("b")}, 1, false, lockresolver, isPullInit, eventCh) 441 c.Assert(errors.Cause(err), check.Equals, context.Canceled) 442 cdcClient.Close() //nolint:errcheck 443 }() 444 445 // new session, new request 446 waitRequestID(c, baseAllocatedID+1) 447 448 initialized := mockInitializedEvent(3 /* regionID */, currentRequestID()) 449 ch2 <- initialized 450 451 var event model.RegionFeedEvent 452 select { 453 case event = <-eventCh: 454 case <-time.After(time.Second): 455 c.Fatalf("recving message takes too long") 456 } 457 c.Assert(event, check.NotNil) 458 459 largeValSize := 128*1024*1024 + 1 // 128MB + 1 460 largeMsg := &cdcpb.ChangeDataEvent{Events: []*cdcpb.Event{ 461 { 462 RegionId: 3, 463 RequestId: currentRequestID(), 464 Event: &cdcpb.Event_Entries_{ 465 Entries: &cdcpb.Event_Entries{ 466 Entries: []*cdcpb.Event_Row{{ 467 Type: cdcpb.Event_COMMITTED, 468 OpType: cdcpb.Event_Row_PUT, 469 Key: []byte("a"), 470 Value: make([]byte, largeValSize), 471 CommitTs: 2, // ResolvedTs = 1 472 }}, 473 }, 474 }, 475 }, 476 }} 477 ch2 <- largeMsg 478 select { 479 case event = <-eventCh: 480 case <-time.After(30 * time.Second): // Send 128MB object may costs lots of time. 481 c.Fatalf("recving message takes too long") 482 } 483 c.Assert(len(event.Val.Value), check.Equals, largeValSize) 484 cancel() 485 } 486 487 func (s *etcdSuite) TestHandleError(c *check.C) { 488 defer testleak.AfterTest(c)() 489 defer s.TearDownTest(c) 490 ctx, cancel := context.WithCancel(context.Background()) 491 wg := &sync.WaitGroup{} 492 493 ch1 := make(chan *cdcpb.ChangeDataEvent, 10) 494 srv1 := newMockChangeDataService(c, ch1) 495 server1, addr1 := newMockService(ctx, c, srv1, wg) 496 497 ch2 := make(chan *cdcpb.ChangeDataEvent, 10) 498 srv2 := newMockChangeDataService(c, ch2) 499 server2, addr2 := newMockService(ctx, c, srv2, wg) 500 501 defer func() { 502 close(ch1) 503 server1.Stop() 504 close(ch2) 505 server2.Stop() 506 wg.Wait() 507 }() 508 509 rpcClient, cluster, pdClient, err := mocktikv.NewTiKVAndPDClient("") 510 c.Assert(err, check.IsNil) 511 pdClient = &mockPDClient{Client: pdClient, versionGen: defaultVersionGen} 512 tiStore, err := tikv.NewTestTiKVStore(rpcClient, pdClient, nil, nil, 0) 513 c.Assert(err, check.IsNil) 514 kvStorage := newStorageWithCurVersionCache(tiStore, addr1) 515 defer kvStorage.Close() //nolint:errcheck 516 517 // TODO: remove the following three lines after kv client v2 is enabled 518 cluster.AddStore(1, addr1) 519 cluster.AddStore(2, addr2) 520 cluster.Bootstrap(3, []uint64{1, 2}, []uint64{4, 5}, 4) 521 522 /* TODO: after switch to kv client v2, enable the following codes 523 region3 := uint64(3) 524 region4 := uint64(4) 525 region5 := uint64(5) 526 cluster.AddStore(1, addr1) 527 cluster.AddStore(2, addr2) 528 cluster.Bootstrap(region3, []uint64{1, 2}, []uint64{4, 5}, 4) 529 // split two regions with leader on different TiKV nodes to avoid region 530 // worker exits because of empty maintained region 531 cluster.SplitRaw(region3, region4, []byte("b"), []uint64{6, 7}, 6) 532 cluster.SplitRaw(region4, region5, []byte("c"), []uint64{8, 9}, 9) 533 */ 534 535 baseAllocatedID := currentRequestID() 536 lockresolver := txnutil.NewLockerResolver(kvStorage) 537 isPullInit := &mockPullerInit{} 538 grpcPool := NewGrpcPoolImpl(ctx, &security.Credential{}) 539 defer grpcPool.Close() 540 cdcClient := NewCDCClient(ctx, pdClient, kvStorage, grpcPool) 541 eventCh := make(chan model.RegionFeedEvent, 10) 542 wg.Add(1) 543 go func() { 544 defer wg.Done() 545 err := cdcClient.EventFeed(ctx, regionspan.ComparableSpan{Start: []byte("a"), End: []byte("d")}, 100, false, lockresolver, isPullInit, eventCh) 546 c.Assert(errors.Cause(err), check.Equals, context.Canceled) 547 cdcClient.Close() //nolint:errcheck 548 }() 549 550 // wait request id allocated with: new session, new request 551 waitRequestID(c, baseAllocatedID+1) 552 553 var event model.RegionFeedEvent 554 notLeader := &cdcpb.ChangeDataEvent{Events: []*cdcpb.Event{ 555 { 556 RegionId: 3, 557 RequestId: currentRequestID(), 558 Event: &cdcpb.Event_Error{ 559 Error: &cdcpb.Error{ 560 NotLeader: &errorpb.NotLeader{ 561 RegionId: 3, 562 Leader: &metapb.Peer{ 563 StoreId: 2, 564 }, 565 }, 566 }, 567 }, 568 }, 569 }} 570 ch1 <- notLeader 571 cluster.ChangeLeader(3, 5) 572 573 // wait request id allocated with: 574 // new session, no leader request, epoch not match request 575 waitRequestID(c, baseAllocatedID+2) 576 epochNotMatch := &cdcpb.ChangeDataEvent{Events: []*cdcpb.Event{ 577 { 578 RegionId: 3, 579 RequestId: currentRequestID(), 580 Event: &cdcpb.Event_Error{ 581 Error: &cdcpb.Error{ 582 EpochNotMatch: &errorpb.EpochNotMatch{}, 583 }, 584 }, 585 }, 586 }} 587 ch2 <- epochNotMatch 588 589 waitRequestID(c, baseAllocatedID+3) 590 regionNotFound := &cdcpb.ChangeDataEvent{Events: []*cdcpb.Event{ 591 { 592 RegionId: 3, 593 RequestId: currentRequestID(), 594 Event: &cdcpb.Event_Error{ 595 Error: &cdcpb.Error{ 596 RegionNotFound: &errorpb.RegionNotFound{}, 597 }, 598 }, 599 }, 600 }} 601 ch2 <- regionNotFound 602 603 waitRequestID(c, baseAllocatedID+4) 604 unknownErr := &cdcpb.ChangeDataEvent{Events: []*cdcpb.Event{ 605 { 606 RegionId: 3, 607 RequestId: currentRequestID(), 608 Event: &cdcpb.Event_Error{ 609 Error: &cdcpb.Error{}, 610 }, 611 }, 612 }} 613 ch2 <- unknownErr 614 615 // `singleEventFeed` always emits a resolved event with ResolvedTs == StartTs 616 // when it starts. 617 consumePreResolvedTs: 618 for { 619 select { 620 case event = <-eventCh: 621 c.Assert(event.Resolved, check.NotNil) 622 c.Assert(event.Resolved.ResolvedTs, check.Equals, uint64(100)) 623 case <-time.After(time.Second): 624 break consumePreResolvedTs 625 } 626 } 627 628 // wait request id allocated with: 629 // new session, no leader request, epoch not match request, 630 // region not found request, unknown error request, normal request 631 waitRequestID(c, baseAllocatedID+5) 632 initialized := mockInitializedEvent(3 /* regionID */, currentRequestID()) 633 ch2 <- initialized 634 635 makeEvent := func(ts uint64) *cdcpb.ChangeDataEvent { 636 return &cdcpb.ChangeDataEvent{ 637 Events: []*cdcpb.Event{ 638 { 639 RegionId: 3, 640 RequestId: currentRequestID(), 641 Event: &cdcpb.Event_ResolvedTs{ 642 ResolvedTs: ts, 643 }, 644 }, 645 }, 646 } 647 } 648 // fallback resolved ts event from TiKV 649 ch2 <- makeEvent(90) 650 // normal resolved ts evnet 651 ch2 <- makeEvent(120) 652 select { 653 case event = <-eventCh: 654 case <-time.After(3 * time.Second): 655 c.Fatalf("reconnection not succeed in 3 seconds") 656 } 657 c.Assert(event.Resolved, check.NotNil) 658 c.Assert(event.Resolved.ResolvedTs, check.Equals, uint64(120)) 659 660 cancel() 661 } 662 663 // TestCompatibilityWithSameConn tests kv client returns an error when TiKV returns 664 // the Compatibility error. This error only happens when the same connection to 665 // TiKV have different versions. 666 func (s *etcdSuite) TestCompatibilityWithSameConn(c *check.C) { 667 defer testleak.AfterTest(c)() 668 defer s.TearDownTest(c) 669 ctx, cancel := context.WithCancel(context.Background()) 670 wg := &sync.WaitGroup{} 671 672 ch1 := make(chan *cdcpb.ChangeDataEvent, 10) 673 srv1 := newMockChangeDataService(c, ch1) 674 server1, addr1 := newMockService(ctx, c, srv1, wg) 675 defer func() { 676 close(ch1) 677 server1.Stop() 678 wg.Wait() 679 }() 680 681 rpcClient, cluster, pdClient, err := mocktikv.NewTiKVAndPDClient("") 682 c.Assert(err, check.IsNil) 683 pdClient = &mockPDClient{Client: pdClient, versionGen: defaultVersionGen} 684 tiStore, err := tikv.NewTestTiKVStore(rpcClient, pdClient, nil, nil, 0) 685 c.Assert(err, check.IsNil) 686 kvStorage := newStorageWithCurVersionCache(tiStore, addr1) 687 defer kvStorage.Close() //nolint:errcheck 688 689 cluster.AddStore(1, addr1) 690 cluster.Bootstrap(3, []uint64{1}, []uint64{4}, 4) 691 692 baseAllocatedID := currentRequestID() 693 lockresolver := txnutil.NewLockerResolver(kvStorage) 694 isPullInit := &mockPullerInit{} 695 grpcPool := NewGrpcPoolImpl(ctx, &security.Credential{}) 696 defer grpcPool.Close() 697 cdcClient := NewCDCClient(ctx, pdClient, kvStorage, grpcPool) 698 eventCh := make(chan model.RegionFeedEvent, 10) 699 var wg2 sync.WaitGroup 700 wg2.Add(1) 701 go func() { 702 defer wg2.Done() 703 err := cdcClient.EventFeed(ctx, regionspan.ComparableSpan{Start: []byte("a"), End: []byte("b")}, 100, false, lockresolver, isPullInit, eventCh) 704 c.Assert(cerror.ErrVersionIncompatible.Equal(err), check.IsTrue) 705 cdcClient.Close() //nolint:errcheck 706 }() 707 708 // wait request id allocated with: new session, new request 709 waitRequestID(c, baseAllocatedID+1) 710 incompatibility := &cdcpb.ChangeDataEvent{Events: []*cdcpb.Event{ 711 { 712 RegionId: 3, 713 RequestId: currentRequestID(), 714 Event: &cdcpb.Event_Error{ 715 Error: &cdcpb.Error{ 716 Compatibility: &cdcpb.Compatibility{ 717 RequiredVersion: "v4.0.7", 718 }, 719 }, 720 }, 721 }, 722 }} 723 ch1 <- incompatibility 724 wg2.Wait() 725 cancel() 726 } 727 728 func (s *etcdSuite) testHandleFeedEvent(c *check.C) { 729 defer s.TearDownTest(c) 730 ctx, cancel := context.WithCancel(context.Background()) 731 wg := &sync.WaitGroup{} 732 733 ch1 := make(chan *cdcpb.ChangeDataEvent, 10) 734 srv1 := newMockChangeDataService(c, ch1) 735 server1, addr1 := newMockService(ctx, c, srv1, wg) 736 737 defer func() { 738 close(ch1) 739 server1.Stop() 740 wg.Wait() 741 }() 742 743 rpcClient, cluster, pdClient, err := mocktikv.NewTiKVAndPDClient("") 744 c.Assert(err, check.IsNil) 745 pdClient = &mockPDClient{Client: pdClient, versionGen: defaultVersionGen} 746 tiStore, err := tikv.NewTestTiKVStore(rpcClient, pdClient, nil, nil, 0) 747 c.Assert(err, check.IsNil) 748 kvStorage := newStorageWithCurVersionCache(tiStore, addr1) 749 defer kvStorage.Close() //nolint:errcheck 750 751 cluster.AddStore(1, addr1) 752 cluster.Bootstrap(3, []uint64{1}, []uint64{4}, 4) 753 754 baseAllocatedID := currentRequestID() 755 lockresolver := txnutil.NewLockerResolver(kvStorage) 756 isPullInit := &mockPullerInit{} 757 grpcPool := NewGrpcPoolImpl(ctx, &security.Credential{}) 758 defer grpcPool.Close() 759 cdcClient := NewCDCClient(ctx, pdClient, kvStorage, grpcPool) 760 eventCh := make(chan model.RegionFeedEvent, 10) 761 wg.Add(1) 762 go func() { 763 defer wg.Done() 764 err := cdcClient.EventFeed(ctx, regionspan.ComparableSpan{Start: []byte("a"), End: []byte("b")}, 100, false, lockresolver, isPullInit, eventCh) 765 c.Assert(errors.Cause(err), check.Equals, context.Canceled) 766 cdcClient.Close() //nolint:errcheck 767 }() 768 769 // wait request id allocated with: new session, new request 770 waitRequestID(c, baseAllocatedID+1) 771 772 eventsBeforeInit := &cdcpb.ChangeDataEvent{Events: []*cdcpb.Event{ 773 // before initialized, prewrite and commit could be in any sequence, 774 // simulate commit comes before prewrite 775 { 776 RegionId: 3, 777 RequestId: currentRequestID(), 778 Event: &cdcpb.Event_Entries_{ 779 Entries: &cdcpb.Event_Entries{ 780 Entries: []*cdcpb.Event_Row{{ 781 Type: cdcpb.Event_COMMIT, 782 OpType: cdcpb.Event_Row_PUT, 783 Key: []byte("aaa"), 784 StartTs: 112, 785 CommitTs: 122, 786 }}, 787 }, 788 }, 789 }, 790 { 791 RegionId: 3, 792 RequestId: currentRequestID(), 793 Event: &cdcpb.Event_Entries_{ 794 Entries: &cdcpb.Event_Entries{ 795 Entries: []*cdcpb.Event_Row{{ 796 Type: cdcpb.Event_PREWRITE, 797 OpType: cdcpb.Event_Row_PUT, 798 Key: []byte("aaa"), 799 Value: []byte("commit-prewrite-sequence-before-init"), 800 StartTs: 112, 801 }}, 802 }, 803 }, 804 }, 805 806 // prewrite and commit in the normal sequence 807 { 808 RegionId: 3, 809 RequestId: currentRequestID(), 810 Event: &cdcpb.Event_Entries_{ 811 Entries: &cdcpb.Event_Entries{ 812 Entries: []*cdcpb.Event_Row{{ 813 Type: cdcpb.Event_PREWRITE, 814 OpType: cdcpb.Event_Row_PUT, 815 Key: []byte("aaa"), 816 Value: []byte("prewrite-commit-sequence-before-init"), 817 StartTs: 110, // ResolvedTs = 100 818 }}, 819 }, 820 }, 821 }, 822 { 823 RegionId: 3, 824 RequestId: currentRequestID(), 825 Event: &cdcpb.Event_Entries_{ 826 Entries: &cdcpb.Event_Entries{ 827 Entries: []*cdcpb.Event_Row{{ 828 Type: cdcpb.Event_COMMIT, 829 OpType: cdcpb.Event_Row_PUT, 830 Key: []byte("aaa"), 831 StartTs: 110, // ResolvedTs = 100 832 CommitTs: 120, 833 }}, 834 }, 835 }, 836 }, 837 838 // commit event before initializtion without prewrite matched will be ignored 839 { 840 RegionId: 3, 841 RequestId: currentRequestID(), 842 Event: &cdcpb.Event_Entries_{ 843 Entries: &cdcpb.Event_Entries{ 844 Entries: []*cdcpb.Event_Row{{ 845 Type: cdcpb.Event_COMMIT, 846 OpType: cdcpb.Event_Row_PUT, 847 Key: []byte("aa"), 848 StartTs: 105, 849 CommitTs: 115, 850 }}, 851 }, 852 }, 853 }, 854 { 855 RegionId: 3, 856 RequestId: currentRequestID(), 857 Event: &cdcpb.Event_Entries_{ 858 Entries: &cdcpb.Event_Entries{ 859 Entries: []*cdcpb.Event_Row{{ 860 Type: cdcpb.Event_COMMITTED, 861 OpType: cdcpb.Event_Row_PUT, 862 Key: []byte("aaaa"), 863 Value: []byte("committed put event before init"), 864 StartTs: 105, 865 CommitTs: 115, 866 }}, 867 }, 868 }, 869 }, 870 { 871 RegionId: 3, 872 RequestId: currentRequestID(), 873 Event: &cdcpb.Event_Entries_{ 874 Entries: &cdcpb.Event_Entries{ 875 Entries: []*cdcpb.Event_Row{{ 876 Type: cdcpb.Event_COMMITTED, 877 OpType: cdcpb.Event_Row_DELETE, 878 Key: []byte("aaaa"), 879 Value: []byte("committed delete event before init"), 880 StartTs: 108, 881 CommitTs: 118, 882 }}, 883 }, 884 }, 885 }, 886 }} 887 initialized := mockInitializedEvent(3 /*regionID */, currentRequestID()) 888 eventsAfterInit := &cdcpb.ChangeDataEvent{Events: []*cdcpb.Event{ 889 { 890 RegionId: 3, 891 RequestId: currentRequestID(), 892 Event: &cdcpb.Event_Entries_{ 893 Entries: &cdcpb.Event_Entries{ 894 Entries: []*cdcpb.Event_Row{{ 895 Type: cdcpb.Event_PREWRITE, 896 OpType: cdcpb.Event_Row_PUT, 897 Key: []byte("a-rollback-event"), 898 StartTs: 128, 899 }}, 900 }, 901 }, 902 }, 903 { 904 RegionId: 3, 905 RequestId: currentRequestID(), 906 Event: &cdcpb.Event_Entries_{ 907 Entries: &cdcpb.Event_Entries{ 908 Entries: []*cdcpb.Event_Row{{ 909 Type: cdcpb.Event_ROLLBACK, 910 OpType: cdcpb.Event_Row_PUT, 911 Key: []byte("a-rollback-event"), 912 StartTs: 128, 913 CommitTs: 129, 914 }}, 915 }, 916 }, 917 }, 918 { 919 RegionId: 3, 920 RequestId: currentRequestID(), 921 Event: &cdcpb.Event_Entries_{ 922 Entries: &cdcpb.Event_Entries{ 923 Entries: []*cdcpb.Event_Row{{ 924 Type: cdcpb.Event_PREWRITE, 925 OpType: cdcpb.Event_Row_DELETE, 926 Key: []byte("a-delete-event"), 927 StartTs: 130, 928 }}, 929 }, 930 }, 931 }, 932 { 933 RegionId: 3, 934 RequestId: currentRequestID(), 935 Event: &cdcpb.Event_Entries_{ 936 Entries: &cdcpb.Event_Entries{ 937 Entries: []*cdcpb.Event_Row{{ 938 Type: cdcpb.Event_COMMIT, 939 OpType: cdcpb.Event_Row_DELETE, 940 Key: []byte("a-delete-event"), 941 StartTs: 130, 942 CommitTs: 140, 943 }}, 944 }, 945 }, 946 }, 947 { 948 RegionId: 3, 949 RequestId: currentRequestID(), 950 Event: &cdcpb.Event_Entries_{ 951 Entries: &cdcpb.Event_Entries{ 952 Entries: []*cdcpb.Event_Row{{ 953 Type: cdcpb.Event_PREWRITE, 954 OpType: cdcpb.Event_Row_PUT, 955 Key: []byte("a-normal-put"), 956 Value: []byte("normal put event"), 957 StartTs: 135, 958 }}, 959 }, 960 }, 961 }, 962 // simulate TiKV sends txn heartbeat, which is a prewrite event with empty value 963 { 964 RegionId: 3, 965 RequestId: currentRequestID(), 966 Event: &cdcpb.Event_Entries_{ 967 Entries: &cdcpb.Event_Entries{ 968 Entries: []*cdcpb.Event_Row{{ 969 Type: cdcpb.Event_PREWRITE, 970 OpType: cdcpb.Event_Row_PUT, 971 Key: []byte("a-normal-put"), 972 StartTs: 135, 973 }}, 974 }, 975 }, 976 }, 977 { 978 RegionId: 3, 979 RequestId: currentRequestID(), 980 Event: &cdcpb.Event_Entries_{ 981 Entries: &cdcpb.Event_Entries{ 982 Entries: []*cdcpb.Event_Row{{ 983 Type: cdcpb.Event_COMMIT, 984 OpType: cdcpb.Event_Row_PUT, 985 Key: []byte("a-normal-put"), 986 StartTs: 135, 987 CommitTs: 145, 988 }}, 989 }, 990 }, 991 }, 992 }} 993 eventResolved := &cdcpb.ChangeDataEvent{Events: []*cdcpb.Event{ 994 { 995 RegionId: 3, 996 RequestId: currentRequestID(), 997 Event: &cdcpb.Event_ResolvedTs{ResolvedTs: 135}, 998 }, 999 }} 1000 // batch resolved ts 1001 eventResolvedBatch := &cdcpb.ChangeDataEvent{ 1002 ResolvedTs: &cdcpb.ResolvedTs{ 1003 Regions: []uint64{3}, 1004 Ts: 145, 1005 }, 1006 } 1007 multiSize := 100 1008 regions := make([]uint64, multiSize) 1009 for i := range regions { 1010 regions[i] = 3 1011 } 1012 multipleResolved := &cdcpb.ChangeDataEvent{ 1013 ResolvedTs: &cdcpb.ResolvedTs{ 1014 Regions: regions, 1015 Ts: 160, 1016 }, 1017 } 1018 1019 expected := []model.RegionFeedEvent{ 1020 { 1021 Resolved: &model.ResolvedSpan{ 1022 Span: regionspan.ComparableSpan{Start: []byte("a"), End: []byte("b")}, 1023 ResolvedTs: 100, 1024 }, 1025 RegionID: 3, 1026 }, 1027 { 1028 Val: &model.RawKVEntry{ 1029 OpType: model.OpTypePut, 1030 Key: []byte("aaa"), 1031 Value: []byte("prewrite-commit-sequence-before-init"), 1032 StartTs: 110, 1033 CRTs: 120, 1034 RegionID: 3, 1035 }, 1036 RegionID: 3, 1037 }, 1038 { 1039 Val: &model.RawKVEntry{ 1040 OpType: model.OpTypePut, 1041 Key: []byte("aaaa"), 1042 Value: []byte("committed put event before init"), 1043 StartTs: 105, 1044 CRTs: 115, 1045 RegionID: 3, 1046 }, 1047 RegionID: 3, 1048 }, 1049 { 1050 Val: &model.RawKVEntry{ 1051 OpType: model.OpTypeDelete, 1052 Key: []byte("aaaa"), 1053 Value: []byte("committed delete event before init"), 1054 StartTs: 108, 1055 CRTs: 118, 1056 RegionID: 3, 1057 }, 1058 RegionID: 3, 1059 }, 1060 { 1061 Val: &model.RawKVEntry{ 1062 OpType: model.OpTypePut, 1063 Key: []byte("aaa"), 1064 Value: []byte("commit-prewrite-sequence-before-init"), 1065 StartTs: 112, 1066 CRTs: 122, 1067 RegionID: 3, 1068 }, 1069 RegionID: 3, 1070 }, 1071 { 1072 Val: &model.RawKVEntry{ 1073 OpType: model.OpTypeDelete, 1074 Key: []byte("a-delete-event"), 1075 StartTs: 130, 1076 CRTs: 140, 1077 RegionID: 3, 1078 }, 1079 RegionID: 3, 1080 }, 1081 { 1082 Val: &model.RawKVEntry{ 1083 OpType: model.OpTypePut, 1084 Key: []byte("a-normal-put"), 1085 Value: []byte("normal put event"), 1086 StartTs: 135, 1087 CRTs: 145, 1088 RegionID: 3, 1089 }, 1090 RegionID: 3, 1091 }, 1092 { 1093 Resolved: &model.ResolvedSpan{ 1094 Span: regionspan.ComparableSpan{Start: []byte("a"), End: []byte("b")}, 1095 ResolvedTs: 135, 1096 }, 1097 RegionID: 3, 1098 }, 1099 { 1100 Resolved: &model.ResolvedSpan{ 1101 Span: regionspan.ComparableSpan{Start: []byte("a"), End: []byte("b")}, 1102 ResolvedTs: 145, 1103 }, 1104 RegionID: 3, 1105 }, 1106 } 1107 multipleExpected := model.RegionFeedEvent{ 1108 Resolved: &model.ResolvedSpan{ 1109 Span: regionspan.ComparableSpan{Start: []byte("a"), End: []byte("b")}, 1110 ResolvedTs: 160, 1111 }, 1112 RegionID: 3, 1113 } 1114 1115 ch1 <- eventsBeforeInit 1116 ch1 <- initialized 1117 ch1 <- eventsAfterInit 1118 ch1 <- eventResolved 1119 ch1 <- eventResolvedBatch 1120 1121 for _, expectedEv := range expected { 1122 select { 1123 case event := <-eventCh: 1124 c.Assert(event, check.DeepEquals, expectedEv) 1125 case <-time.After(time.Second): 1126 c.Errorf("expected event %v not received", expectedEv) 1127 } 1128 } 1129 1130 ch1 <- multipleResolved 1131 for i := 0; i < multiSize; i++ { 1132 select { 1133 case event := <-eventCh: 1134 c.Assert(event, check.DeepEquals, multipleExpected) 1135 case <-time.After(time.Second): 1136 c.Errorf("expected event %v not received", multipleExpected) 1137 } 1138 } 1139 1140 cancel() 1141 } 1142 1143 func (s *etcdSuite) TestHandleFeedEvent(c *check.C) { 1144 defer testleak.AfterTest(c)() 1145 s.testHandleFeedEvent(c) 1146 } 1147 1148 func (s *etcdSuite) TestHandleFeedEventWithWorkerPool(c *check.C) { 1149 defer testleak.AfterTest(c)() 1150 hwm := regionWorkerHighWatermark 1151 lwm := regionWorkerLowWatermark 1152 regionWorkerHighWatermark = 8 1153 regionWorkerLowWatermark = 2 1154 defer func() { 1155 regionWorkerHighWatermark = hwm 1156 regionWorkerLowWatermark = lwm 1157 }() 1158 s.testHandleFeedEvent(c) 1159 } 1160 1161 // TestStreamSendWithError mainly tests the scenario that the `Send` call of a gPRC 1162 // stream of kv client meets error, and kv client can clean up the broken stream, 1163 // establish a new one and recover the normal event feed processing. 1164 func (s *etcdSuite) TestStreamSendWithError(c *check.C) { 1165 defer testleak.AfterTest(c)() 1166 defer s.TearDownTest(c) 1167 ctx, cancel := context.WithCancel(context.Background()) 1168 wg := &sync.WaitGroup{} 1169 defer wg.Wait() 1170 defer cancel() 1171 1172 server1Stopped := make(chan struct{}) 1173 ch1 := make(chan *cdcpb.ChangeDataEvent, 10) 1174 srv1 := newMockChangeDataService(c, ch1) 1175 server1, addr1 := newMockService(ctx, c, srv1, wg) 1176 srv1.recvLoop = func(server cdcpb.ChangeData_EventFeedServer) { 1177 defer func() { 1178 close(ch1) 1179 server1.Stop() 1180 server1Stopped <- struct{}{} 1181 }() 1182 // Only receives the first request to simulate a following kv client 1183 // stream.Send error. 1184 _, err := server.Recv() 1185 if err != nil { 1186 log.Error("mock server error", zap.Error(err)) 1187 } 1188 } 1189 1190 rpcClient, cluster, pdClient, err := mocktikv.NewTiKVAndPDClient("") 1191 c.Assert(err, check.IsNil) 1192 pdClient = &mockPDClient{Client: pdClient, versionGen: defaultVersionGen} 1193 tiStore, err := tikv.NewTestTiKVStore(rpcClient, pdClient, nil, nil, 0) 1194 c.Assert(err, check.IsNil) 1195 kvStorage := newStorageWithCurVersionCache(tiStore, addr1) 1196 defer kvStorage.Close() //nolint:errcheck 1197 1198 regionID3 := uint64(3) 1199 regionID4 := uint64(4) 1200 cluster.AddStore(1, addr1) 1201 cluster.Bootstrap(regionID3, []uint64{1}, []uint64{4}, 4) 1202 cluster.SplitRaw(regionID3, regionID4, []byte("b"), []uint64{5}, 5) 1203 1204 lockerResolver := txnutil.NewLockerResolver(kvStorage) 1205 isPullInit := &mockPullerInit{} 1206 grpcPool := NewGrpcPoolImpl(ctx, &security.Credential{}) 1207 defer grpcPool.Close() 1208 cdcClient := NewCDCClient(ctx, pdClient, kvStorage, grpcPool) 1209 eventCh := make(chan model.RegionFeedEvent, 10) 1210 wg.Add(1) 1211 go func() { 1212 defer wg.Done() 1213 err := cdcClient.EventFeed(ctx, regionspan.ComparableSpan{Start: []byte("a"), End: []byte("c")}, 100, false, lockerResolver, isPullInit, eventCh) 1214 c.Assert(errors.Cause(err), check.Equals, context.Canceled) 1215 cdcClient.Close() //nolint:errcheck 1216 }() 1217 1218 var requestIds sync.Map 1219 <-server1Stopped 1220 ch2 := make(chan *cdcpb.ChangeDataEvent, 10) 1221 srv2 := newMockChangeDataService(c, ch2) 1222 srv2.recvLoop = func(server cdcpb.ChangeData_EventFeedServer) { 1223 for { 1224 req, err := server.Recv() 1225 if err != nil { 1226 log.Error("mock server error", zap.Error(err)) 1227 return 1228 } 1229 requestIds.Store(req.RegionId, req.RequestId) 1230 } 1231 } 1232 // Reuse the same listen address as server 1 1233 server2, _ := newMockServiceSpecificAddr(ctx, c, srv2, addr1, wg) 1234 defer func() { 1235 close(ch2) 1236 server2.Stop() 1237 }() 1238 1239 // The expected request ids are agnostic because the kv client could retry 1240 // for more than one time, so we wait until the newly started server receives 1241 // requests for both two regions. 1242 err = retry.Do(context.Background(), func() error { 1243 _, ok1 := requestIds.Load(regionID3) 1244 _, ok2 := requestIds.Load(regionID4) 1245 if ok1 && ok2 { 1246 return nil 1247 } 1248 return errors.New("waiting for kv client requests received by server") 1249 }, retry.WithBackoffBaseDelay(200), retry.WithBackoffMaxDelay(60*1000), retry.WithMaxTries(10)) 1250 1251 c.Assert(err, check.IsNil) 1252 reqID1, _ := requestIds.Load(regionID3) 1253 reqID2, _ := requestIds.Load(regionID4) 1254 initialized1 := mockInitializedEvent(regionID3, reqID1.(uint64)) 1255 initialized2 := mockInitializedEvent(regionID4, reqID2.(uint64)) 1256 ch2 <- initialized1 1257 ch2 <- initialized2 1258 1259 // the event sequence is undeterministic 1260 initRegions := make(map[uint64]struct{}) 1261 for i := 0; i < 2; i++ { 1262 select { 1263 case event := <-eventCh: 1264 c.Assert(event.Resolved, check.NotNil) 1265 initRegions[event.RegionID] = struct{}{} 1266 case <-time.After(time.Second): 1267 c.Errorf("expected events are not receive, received: %v", initRegions) 1268 } 1269 } 1270 expectedInitRegions := map[uint64]struct{}{regionID3: {}, regionID4: {}} 1271 c.Assert(initRegions, check.DeepEquals, expectedInitRegions) 1272 1273 cancel() 1274 } 1275 1276 func (s *etcdSuite) testStreamRecvWithError(c *check.C, failpointStr string) { 1277 defer s.TearDownTest(c) 1278 ctx, cancel := context.WithCancel(context.Background()) 1279 wg := &sync.WaitGroup{} 1280 1281 ch1 := make(chan *cdcpb.ChangeDataEvent, 10) 1282 srv1 := newMockChangeDataService(c, ch1) 1283 server1, addr1 := newMockService(ctx, c, srv1, wg) 1284 1285 defer func() { 1286 close(ch1) 1287 server1.Stop() 1288 wg.Wait() 1289 }() 1290 1291 rpcClient, cluster, pdClient, err := mocktikv.NewTiKVAndPDClient("") 1292 c.Assert(err, check.IsNil) 1293 pdClient = &mockPDClient{Client: pdClient, versionGen: defaultVersionGen} 1294 tiStore, err := tikv.NewTestTiKVStore(rpcClient, pdClient, nil, nil, 0) 1295 c.Assert(err, check.IsNil) 1296 kvStorage := newStorageWithCurVersionCache(tiStore, addr1) 1297 defer kvStorage.Close() //nolint:errcheck 1298 1299 regionID := uint64(3) 1300 cluster.AddStore(1, addr1) 1301 cluster.Bootstrap(regionID, []uint64{1}, []uint64{4}, 4) 1302 1303 err = failpoint.Enable("github.com/pingcap/ticdc/cdc/kv/kvClientStreamRecvError", failpointStr) 1304 c.Assert(err, check.IsNil) 1305 defer func() { 1306 _ = failpoint.Disable("github.com/pingcap/ticdc/cdc/kv/kvClientStreamRecvError") 1307 }() 1308 baseAllocatedID := currentRequestID() 1309 lockresolver := txnutil.NewLockerResolver(kvStorage) 1310 isPullInit := &mockPullerInit{} 1311 grpcPool := NewGrpcPoolImpl(ctx, &security.Credential{}) 1312 defer grpcPool.Close() 1313 cdcClient := NewCDCClient(ctx, pdClient, kvStorage, grpcPool) 1314 eventCh := make(chan model.RegionFeedEvent, 40) 1315 wg.Add(1) 1316 go func() { 1317 defer wg.Done() 1318 err := cdcClient.EventFeed(ctx, regionspan.ComparableSpan{Start: []byte("a"), End: []byte("b")}, 100, false, lockresolver, isPullInit, eventCh) 1319 c.Assert(errors.Cause(err), check.Equals, context.Canceled) 1320 cdcClient.Close() //nolint:errcheck 1321 }() 1322 1323 // wait request id allocated with: new session, new request 1324 waitRequestID(c, baseAllocatedID+1) 1325 initialized1 := mockInitializedEvent(regionID, currentRequestID()) 1326 ch1 <- initialized1 1327 err = retry.Do(context.Background(), func() error { 1328 if len(ch1) == 0 { 1329 return nil 1330 } 1331 return errors.New("message is not sent") 1332 }, retry.WithBackoffBaseDelay(200), retry.WithBackoffMaxDelay(60*1000), retry.WithMaxTries(10)) 1333 1334 c.Assert(err, check.IsNil) 1335 1336 // another stream will be established, so we notify and wait the first 1337 // EventFeed loop exits. 1338 callback := srv1.notifyExit(0) 1339 select { 1340 case <-callback: 1341 case <-time.After(time.Second * 3): 1342 c.Error("event feed loop can't exit") 1343 } 1344 1345 // wait request id allocated with: new session, new request*2 1346 waitRequestID(c, baseAllocatedID+2) 1347 initialized2 := mockInitializedEvent(regionID, currentRequestID()) 1348 ch1 <- initialized2 1349 1350 resolved := &cdcpb.ChangeDataEvent{Events: []*cdcpb.Event{ 1351 { 1352 RegionId: regionID, 1353 RequestId: currentRequestID(), 1354 Event: &cdcpb.Event_ResolvedTs{ResolvedTs: 120}, 1355 }, 1356 }} 1357 ch1 <- resolved 1358 ch1 <- resolved 1359 1360 expected := []model.RegionFeedEvent{ 1361 { 1362 Resolved: &model.ResolvedSpan{ 1363 Span: regionspan.ComparableSpan{Start: []byte("a"), End: []byte("b")}, 1364 ResolvedTs: 120, 1365 }, 1366 RegionID: regionID, 1367 }, 1368 { 1369 Resolved: &model.ResolvedSpan{ 1370 Span: regionspan.ComparableSpan{Start: []byte("a"), End: []byte("b")}, 1371 ResolvedTs: 120, 1372 }, 1373 RegionID: regionID, 1374 }, 1375 } 1376 1377 events := make([]model.RegionFeedEvent, 0, 2) 1378 eventLoop: 1379 for { 1380 select { 1381 case ev := <-eventCh: 1382 if ev.Resolved.ResolvedTs != uint64(100) { 1383 events = append(events, ev) 1384 } 1385 case <-time.After(time.Second): 1386 break eventLoop 1387 } 1388 } 1389 c.Assert(events, check.DeepEquals, expected) 1390 cancel() 1391 } 1392 1393 // TestStreamRecvWithErrorAndResolvedGoBack mainly tests the scenario that the `Recv` call of a gPRC 1394 // stream in kv client meets error, and kv client reconnection with tikv with the current tso 1395 func (s *etcdSuite) TestStreamRecvWithErrorAndResolvedGoBack(c *check.C) { 1396 defer testleak.AfterTest(c)() 1397 defer s.TearDownTest(c) 1398 if !util.FailpointBuild { 1399 c.Skip("skip when this is not a failpoint build") 1400 } 1401 ctx, cancel := context.WithCancel(context.Background()) 1402 wg := &sync.WaitGroup{} 1403 1404 var requestID uint64 1405 ch1 := make(chan *cdcpb.ChangeDataEvent, 10) 1406 srv1 := newMockChangeDataService(c, ch1) 1407 srv1.recvLoop = func(server cdcpb.ChangeData_EventFeedServer) { 1408 for { 1409 req, err := server.Recv() 1410 if err != nil { 1411 log.Error("mock server error", zap.Error(err)) 1412 return 1413 } 1414 atomic.StoreUint64(&requestID, req.RequestId) 1415 } 1416 } 1417 server1, addr1 := newMockService(ctx, c, srv1, wg) 1418 1419 defer func() { 1420 close(ch1) 1421 server1.Stop() 1422 wg.Wait() 1423 }() 1424 1425 rpcClient, cluster, pdClient, err := mocktikv.NewTiKVAndPDClient("") 1426 c.Assert(err, check.IsNil) 1427 pdClient = &mockPDClient{Client: pdClient, versionGen: defaultVersionGen} 1428 tiStore, err := tikv.NewTestTiKVStore(rpcClient, pdClient, nil, nil, 0) 1429 c.Assert(err, check.IsNil) 1430 kvStorage := newStorageWithCurVersionCache(tiStore, addr1) 1431 defer kvStorage.Close() //nolint:errcheck 1432 1433 regionID := uint64(3) 1434 cluster.AddStore(1, addr1) 1435 cluster.Bootstrap(regionID, []uint64{1}, []uint64{4}, 4) 1436 1437 baseAllocatedID := currentRequestID() 1438 lockresolver := txnutil.NewLockerResolver(kvStorage) 1439 isPullInit := &mockPullerInit{} 1440 grpcPool := NewGrpcPoolImpl(ctx, &security.Credential{}) 1441 defer grpcPool.Close() 1442 cdcClient := NewCDCClient(ctx, pdClient, kvStorage, grpcPool) 1443 eventCh := make(chan model.RegionFeedEvent, 10) 1444 wg.Add(1) 1445 go func() { 1446 defer wg.Done() 1447 defer close(eventCh) 1448 err := cdcClient.EventFeed(ctx, regionspan.ComparableSpan{Start: []byte("a"), End: []byte("b")}, 100, false, lockresolver, isPullInit, eventCh) 1449 c.Assert(errors.Cause(err), check.Equals, context.Canceled) 1450 cdcClient.Close() //nolint:errcheck 1451 }() 1452 1453 // wait request id allocated with: new session, new request 1454 waitRequestID(c, baseAllocatedID+1) 1455 err = retry.Do(context.Background(), func() error { 1456 if atomic.LoadUint64(&requestID) == currentRequestID() { 1457 return nil 1458 } 1459 return errors.Errorf("request is not received, requestID: %d, expected: %d", 1460 atomic.LoadUint64(&requestID), currentRequestID()) 1461 }, retry.WithBackoffBaseDelay(50), retry.WithMaxTries(10)) 1462 1463 c.Assert(err, check.IsNil) 1464 initialized1 := mockInitializedEvent(regionID, currentRequestID()) 1465 ch1 <- initialized1 1466 err = retry.Do(context.Background(), func() error { 1467 if len(ch1) == 0 { 1468 return nil 1469 } 1470 return errors.New("message is not sent") 1471 }, retry.WithBackoffBaseDelay(200), retry.WithBackoffMaxDelay(60*1000), retry.WithMaxTries(10)) 1472 1473 c.Assert(err, check.IsNil) 1474 1475 resolved := &cdcpb.ChangeDataEvent{Events: []*cdcpb.Event{ 1476 { 1477 RegionId: regionID, 1478 RequestId: currentRequestID(), 1479 Event: &cdcpb.Event_ResolvedTs{ResolvedTs: 120}, 1480 }, 1481 }} 1482 ch1 <- resolved 1483 err = retry.Do(context.Background(), func() error { 1484 if len(ch1) == 0 { 1485 return nil 1486 } 1487 return errors.New("message is not sent") 1488 }, retry.WithBackoffBaseDelay(200), retry.WithBackoffMaxDelay(60*1000), retry.WithMaxTries(10)) 1489 1490 c.Assert(err, check.IsNil) 1491 err = failpoint.Enable("github.com/pingcap/ticdc/cdc/kv/kvClientStreamRecvError", "1*return(\"\")") 1492 c.Assert(err, check.IsNil) 1493 defer func() { 1494 _ = failpoint.Disable("github.com/pingcap/ticdc/cdc/kv/kvClientStreamRecvError") 1495 }() 1496 ch1 <- resolved 1497 1498 // another stream will be established, so we notify and wait the first 1499 // EventFeed loop exits. 1500 callback := srv1.notifyExit(0) 1501 select { 1502 case <-callback: 1503 case <-time.After(time.Second * 3): 1504 c.Error("event feed loop can't exit") 1505 } 1506 1507 // wait request id allocated with: new session, new request*2 1508 waitRequestID(c, baseAllocatedID+2) 1509 err = retry.Do(context.Background(), func() error { 1510 if atomic.LoadUint64(&requestID) == currentRequestID() { 1511 return nil 1512 } 1513 return errors.Errorf("request is not received, requestID: %d, expected: %d", 1514 atomic.LoadUint64(&requestID), currentRequestID()) 1515 }, retry.WithBackoffBaseDelay(50), retry.WithMaxTries(10)) 1516 1517 c.Assert(err, check.IsNil) 1518 initialized2 := mockInitializedEvent(regionID, currentRequestID()) 1519 ch1 <- initialized2 1520 err = retry.Do(context.Background(), func() error { 1521 if len(ch1) == 0 { 1522 return nil 1523 } 1524 return errors.New("message is not sent") 1525 }, retry.WithBackoffBaseDelay(200), retry.WithBackoffMaxDelay(60*1000), retry.WithMaxTries(10)) 1526 1527 c.Assert(err, check.IsNil) 1528 1529 resolved = &cdcpb.ChangeDataEvent{Events: []*cdcpb.Event{ 1530 { 1531 RegionId: regionID, 1532 RequestId: currentRequestID(), 1533 Event: &cdcpb.Event_ResolvedTs{ResolvedTs: 130}, 1534 }, 1535 }} 1536 ch1 <- resolved 1537 1538 received := make([]model.RegionFeedEvent, 0, 4) 1539 defer cancel() 1540 ReceiveLoop: 1541 for { 1542 select { 1543 case event, ok := <-eventCh: 1544 if !ok { 1545 break ReceiveLoop 1546 } 1547 received = append(received, event) 1548 if event.Resolved.ResolvedTs == 130 { 1549 break ReceiveLoop 1550 } 1551 case <-time.After(time.Second): 1552 c.Errorf("event received timeout") 1553 } 1554 } 1555 var lastResolvedTs uint64 1556 for _, e := range received { 1557 if lastResolvedTs > e.Resolved.ResolvedTs { 1558 c.Errorf("the resolvedTs is back off %#v", resolved) 1559 } 1560 } 1561 } 1562 1563 // TestStreamRecvWithErrorNormal mainly tests the scenario that the `Recv` call 1564 // of a gPRC stream in kv client meets a **logical related** error, and kv client 1565 // logs the error and re-establish new request. 1566 func (s *etcdSuite) TestStreamRecvWithErrorNormal(c *check.C) { 1567 defer testleak.AfterTest(c)() 1568 1569 // test client v2 1570 // enableKVClientV2 = true 1571 // s.testStreamRecvWithError(c, "1*return(\"injected stream recv error\")") 1572 1573 // test client v1 1574 s.testStreamRecvWithError(c, "1*return(\"injected stream recv error\")") 1575 } 1576 1577 // TestStreamRecvWithErrorIOEOF mainly tests the scenario that the `Recv` call 1578 // of a gPRC stream in kv client meets error io.EOF, and kv client logs the error 1579 // and re-establish new request 1580 func (s *etcdSuite) TestStreamRecvWithErrorIOEOF(c *check.C) { 1581 defer testleak.AfterTest(c)() 1582 1583 // test client v2 1584 // enableKVClientV2 = true 1585 // s.testStreamRecvWithError(c, "1*return(\"EOF\")") 1586 1587 // test client v1 1588 s.testStreamRecvWithError(c, "1*return(\"EOF\")") 1589 } 1590 1591 // TestIncompatibleTiKV tests TiCDC new request to TiKV meets `ErrVersionIncompatible` 1592 // error (in fact this error is raised before EventFeed API is really called), 1593 // TiCDC will wait 20s and then retry. This is a common scenario when rolling 1594 // upgrade a cluster and the new version is not compatible with the old version 1595 // (upgrade TiCDC before TiKV, since upgrade TiKV often takes much longer). 1596 func (s *etcdSuite) TestIncompatibleTiKV(c *check.C) { 1597 defer testleak.AfterTest(c)() 1598 defer s.TearDownTest(c) 1599 ctx, cancel := context.WithCancel(context.Background()) 1600 wg := &sync.WaitGroup{} 1601 1602 // the minimum valid TiKV version is "4.0.0-rc.1" 1603 incompatibilityVers := []string{"v2.1.10", "v3.0.10", "v3.1.0", "v4.0.0-rc"} 1604 var genLock sync.Mutex 1605 nextVer := -1 1606 call := int32(0) 1607 // 20 here not too much, since check version itself has 3 time retry, and 1608 // region cache could also call get store API, which will trigger version 1609 // generator too. 1610 versionGenCallBoundary := int32(20) 1611 gen := func() string { 1612 genLock.Lock() 1613 defer genLock.Unlock() 1614 atomic.AddInt32(&call, 1) 1615 if atomic.LoadInt32(&call) < versionGenCallBoundary { 1616 nextVer = (nextVer + 1) % len(incompatibilityVers) 1617 return incompatibilityVers[nextVer] 1618 } 1619 return defaultVersionGen() 1620 } 1621 1622 var requestIds sync.Map 1623 ch1 := make(chan *cdcpb.ChangeDataEvent, 10) 1624 srv1 := newMockChangeDataService(c, ch1) 1625 srv1.recvLoop = func(server cdcpb.ChangeData_EventFeedServer) { 1626 for { 1627 req, err := server.Recv() 1628 if err != nil { 1629 log.Error("mock server error", zap.Error(err)) 1630 return 1631 } 1632 requestIds.Store(req.RegionId, req.RequestId) 1633 } 1634 } 1635 server1, addr1 := newMockService(ctx, c, srv1, wg) 1636 1637 defer func() { 1638 close(ch1) 1639 server1.Stop() 1640 wg.Wait() 1641 }() 1642 1643 rpcClient, cluster, pdClient, err := mocktikv.NewTiKVAndPDClient("") 1644 c.Assert(err, check.IsNil) 1645 pdClient = &mockPDClient{Client: pdClient, versionGen: gen} 1646 tiStore, err := tikv.NewTestTiKVStore(rpcClient, pdClient, nil, nil, 0) 1647 c.Assert(err, check.IsNil) 1648 kvStorage := newStorageWithCurVersionCache(tiStore, addr1) 1649 defer kvStorage.Close() //nolint:errcheck 1650 1651 regionID := uint64(3) 1652 cluster.AddStore(1, addr1) 1653 cluster.Bootstrap(regionID, []uint64{1}, []uint64{4}, 4) 1654 1655 err = failpoint.Enable("github.com/pingcap/ticdc/cdc/kv/kvClientDelayWhenIncompatible", "return(true)") 1656 c.Assert(err, check.IsNil) 1657 defer func() { 1658 _ = failpoint.Disable("github.com/pingcap/ticdc/cdc/kv/kvClientDelayWhenIncompatible") 1659 }() 1660 lockresolver := txnutil.NewLockerResolver(kvStorage) 1661 isPullInit := &mockPullerInit{} 1662 grpcPool := NewGrpcPoolImpl(ctx, &security.Credential{}) 1663 defer grpcPool.Close() 1664 cdcClient := NewCDCClient(ctx, pdClient, kvStorage, grpcPool) 1665 eventCh := make(chan model.RegionFeedEvent, 10) 1666 wg.Add(1) 1667 go func() { 1668 defer wg.Done() 1669 err := cdcClient.EventFeed(ctx, regionspan.ComparableSpan{Start: []byte("a"), End: []byte("b")}, 100, false, lockresolver, isPullInit, eventCh) 1670 c.Assert(errors.Cause(err), check.Equals, context.Canceled) 1671 cdcClient.Close() //nolint:errcheck 1672 }() 1673 1674 err = retry.Do(context.Background(), func() error { 1675 if atomic.LoadInt32(&call) >= versionGenCallBoundary { 1676 return nil 1677 } 1678 return errors.Errorf("version generator is not updated in time, call time %d", atomic.LoadInt32(&call)) 1679 }, retry.WithBackoffBaseDelay(500), retry.WithBackoffMaxDelay(60*1000), retry.WithMaxTries(20)) 1680 1681 c.Assert(err, check.IsNil) 1682 err = retry.Do(context.Background(), func() error { 1683 _, ok := requestIds.Load(regionID) 1684 if ok { 1685 return nil 1686 } 1687 return errors.New("waiting for kv client requests received by server") 1688 }, retry.WithBackoffBaseDelay(200), retry.WithBackoffMaxDelay(60*1000), retry.WithMaxTries(10)) 1689 1690 c.Assert(err, check.IsNil) 1691 reqID, _ := requestIds.Load(regionID) 1692 initialized := mockInitializedEvent(regionID, reqID.(uint64)) 1693 ch1 <- initialized 1694 select { 1695 case event := <-eventCh: 1696 c.Assert(event.Resolved, check.NotNil) 1697 c.Assert(event.RegionID, check.Equals, regionID) 1698 case <-time.After(time.Second): 1699 c.Errorf("expected events are not receive") 1700 } 1701 1702 cancel() 1703 } 1704 1705 // TestPendingRegionError tests kv client should return an error when receiving 1706 // a new subscription (the first event of specific region) but the corresponding 1707 // region is not found in pending regions. 1708 func (s *etcdSuite) TestNoPendingRegionError(c *check.C) { 1709 defer testleak.AfterTest(c)() 1710 defer s.TearDownTest(c) 1711 ctx, cancel := context.WithCancel(context.Background()) 1712 wg := &sync.WaitGroup{} 1713 1714 ch1 := make(chan *cdcpb.ChangeDataEvent, 10) 1715 srv1 := newMockChangeDataService(c, ch1) 1716 server1, addr1 := newMockService(ctx, c, srv1, wg) 1717 defer func() { 1718 close(ch1) 1719 server1.Stop() 1720 wg.Wait() 1721 }() 1722 1723 rpcClient, cluster, pdClient, err := mocktikv.NewTiKVAndPDClient("") 1724 c.Assert(err, check.IsNil) 1725 pdClient = &mockPDClient{Client: pdClient, versionGen: defaultVersionGen} 1726 tiStore, err := tikv.NewTestTiKVStore(rpcClient, pdClient, nil, nil, 0) 1727 c.Assert(err, check.IsNil) 1728 kvStorage := newStorageWithCurVersionCache(tiStore, addr1) 1729 defer kvStorage.Close() //nolint:errcheck 1730 1731 cluster.AddStore(1, addr1) 1732 cluster.Bootstrap(3, []uint64{1}, []uint64{4}, 4) 1733 1734 baseAllocatedID := currentRequestID() 1735 lockresolver := txnutil.NewLockerResolver(kvStorage) 1736 isPullInit := &mockPullerInit{} 1737 grpcPool := NewGrpcPoolImpl(ctx, &security.Credential{}) 1738 defer grpcPool.Close() 1739 cdcClient := NewCDCClient(ctx, pdClient, kvStorage, grpcPool) 1740 eventCh := make(chan model.RegionFeedEvent, 10) 1741 var wg2 sync.WaitGroup 1742 if enableKVClientV2 { 1743 wg.Add(1) 1744 go func() { 1745 defer wg.Done() 1746 err := cdcClient.EventFeed(ctx, regionspan.ComparableSpan{Start: []byte("a"), End: []byte("b")}, 100, false, lockresolver, isPullInit, eventCh) 1747 c.Assert(errors.Cause(err), check.Equals, context.Canceled) 1748 cdcClient.Close() //nolint:errcheck 1749 }() 1750 } else { 1751 wg2.Add(1) 1752 go func() { 1753 defer wg2.Done() 1754 err := cdcClient.EventFeed(ctx, regionspan.ComparableSpan{Start: []byte("a"), End: []byte("b")}, 100, false, lockresolver, isPullInit, eventCh) 1755 if enableKVClientV2 { 1756 c.Assert(err, check.IsNil) 1757 } else { 1758 c.Assert(cerror.ErrNoPendingRegion.Equal(err), check.IsTrue) 1759 } 1760 cdcClient.Close() //nolint:errcheck 1761 }() 1762 } 1763 1764 // wait request id allocated with: new session, new request 1765 waitRequestID(c, baseAllocatedID+1) 1766 noPendingRegionEvent := &cdcpb.ChangeDataEvent{Events: []*cdcpb.Event{ 1767 { 1768 RegionId: 3, 1769 RequestId: currentRequestID() + 1, // an invalid request id 1770 Event: &cdcpb.Event_ResolvedTs{ResolvedTs: 100}, 1771 }, 1772 }} 1773 ch1 <- noPendingRegionEvent 1774 if enableKVClientV2 { 1775 initialized := mockInitializedEvent(3, currentRequestID()) 1776 ch1 <- initialized 1777 ev := <-eventCh 1778 c.Assert(ev.Resolved, check.NotNil) 1779 c.Assert(ev.Resolved.ResolvedTs, check.Equals, uint64(100)) 1780 1781 resolved := &cdcpb.ChangeDataEvent{Events: []*cdcpb.Event{ 1782 { 1783 RegionId: 3, 1784 RequestId: currentRequestID(), 1785 Event: &cdcpb.Event_ResolvedTs{ResolvedTs: 200}, 1786 }, 1787 }} 1788 ch1 <- resolved 1789 ev = <-eventCh 1790 c.Assert(ev.Resolved, check.NotNil) 1791 c.Assert(ev.Resolved.ResolvedTs, check.Equals, uint64(200)) 1792 } 1793 wg2.Wait() 1794 cancel() 1795 } 1796 1797 // TestDropStaleRequest tests kv client should drop an event if its request id is outdated. 1798 func (s *etcdSuite) TestDropStaleRequest(c *check.C) { 1799 defer testleak.AfterTest(c)() 1800 defer s.TearDownTest(c) 1801 ctx, cancel := context.WithCancel(context.Background()) 1802 wg := &sync.WaitGroup{} 1803 1804 ch1 := make(chan *cdcpb.ChangeDataEvent, 10) 1805 srv1 := newMockChangeDataService(c, ch1) 1806 server1, addr1 := newMockService(ctx, c, srv1, wg) 1807 1808 defer func() { 1809 close(ch1) 1810 server1.Stop() 1811 wg.Wait() 1812 }() 1813 1814 rpcClient, cluster, pdClient, err := mocktikv.NewTiKVAndPDClient("") 1815 c.Assert(err, check.IsNil) 1816 pdClient = &mockPDClient{Client: pdClient, versionGen: defaultVersionGen} 1817 tiStore, err := tikv.NewTestTiKVStore(rpcClient, pdClient, nil, nil, 0) 1818 c.Assert(err, check.IsNil) 1819 kvStorage := newStorageWithCurVersionCache(tiStore, addr1) 1820 defer kvStorage.Close() //nolint:errcheck 1821 1822 regionID := uint64(3) 1823 cluster.AddStore(1, addr1) 1824 cluster.Bootstrap(regionID, []uint64{1}, []uint64{4}, 4) 1825 1826 baseAllocatedID := currentRequestID() 1827 lockresolver := txnutil.NewLockerResolver(kvStorage) 1828 isPullInit := &mockPullerInit{} 1829 grpcPool := NewGrpcPoolImpl(ctx, &security.Credential{}) 1830 defer grpcPool.Close() 1831 cdcClient := NewCDCClient(ctx, pdClient, kvStorage, grpcPool) 1832 eventCh := make(chan model.RegionFeedEvent, 10) 1833 wg.Add(1) 1834 go func() { 1835 defer wg.Done() 1836 err := cdcClient.EventFeed(ctx, regionspan.ComparableSpan{Start: []byte("a"), End: []byte("b")}, 100, false, lockresolver, isPullInit, eventCh) 1837 c.Assert(errors.Cause(err), check.Equals, context.Canceled) 1838 cdcClient.Close() //nolint:errcheck 1839 }() 1840 1841 // wait request id allocated with: new session, new request 1842 waitRequestID(c, baseAllocatedID+1) 1843 1844 initialized := mockInitializedEvent(regionID, currentRequestID()) 1845 eventsAfterInit := &cdcpb.ChangeDataEvent{Events: []*cdcpb.Event{ 1846 { 1847 RegionId: regionID, 1848 RequestId: currentRequestID(), 1849 Event: &cdcpb.Event_ResolvedTs{ResolvedTs: 120}, 1850 }, 1851 // This event will be dropped 1852 { 1853 RegionId: regionID, 1854 RequestId: currentRequestID() - 1, 1855 Event: &cdcpb.Event_ResolvedTs{ResolvedTs: 125}, 1856 }, 1857 { 1858 RegionId: regionID, 1859 RequestId: currentRequestID(), 1860 Event: &cdcpb.Event_ResolvedTs{ResolvedTs: 130}, 1861 }, 1862 }} 1863 expected := []model.RegionFeedEvent{ 1864 { 1865 Resolved: &model.ResolvedSpan{ 1866 Span: regionspan.ComparableSpan{Start: []byte("a"), End: []byte("b")}, 1867 ResolvedTs: 100, 1868 }, 1869 RegionID: regionID, 1870 }, 1871 { 1872 Resolved: &model.ResolvedSpan{ 1873 Span: regionspan.ComparableSpan{Start: []byte("a"), End: []byte("b")}, 1874 ResolvedTs: 120, 1875 }, 1876 RegionID: regionID, 1877 }, 1878 { 1879 Resolved: &model.ResolvedSpan{ 1880 Span: regionspan.ComparableSpan{Start: []byte("a"), End: []byte("b")}, 1881 ResolvedTs: 130, 1882 }, 1883 RegionID: regionID, 1884 }, 1885 } 1886 1887 ch1 <- initialized 1888 ch1 <- eventsAfterInit 1889 1890 for _, expectedEv := range expected { 1891 select { 1892 case event := <-eventCh: 1893 c.Assert(event, check.DeepEquals, expectedEv) 1894 case <-time.After(time.Second): 1895 c.Errorf("expected event %v not received", expectedEv) 1896 } 1897 } 1898 cancel() 1899 } 1900 1901 // TestResolveLock tests the resolve lock logic in kv client 1902 func (s *etcdSuite) TestResolveLock(c *check.C) { 1903 defer testleak.AfterTest(c)() 1904 defer s.TearDownTest(c) 1905 ctx, cancel := context.WithCancel(context.Background()) 1906 wg := &sync.WaitGroup{} 1907 1908 ch1 := make(chan *cdcpb.ChangeDataEvent, 10) 1909 srv1 := newMockChangeDataService(c, ch1) 1910 server1, addr1 := newMockService(ctx, c, srv1, wg) 1911 1912 defer func() { 1913 close(ch1) 1914 server1.Stop() 1915 wg.Wait() 1916 }() 1917 1918 rpcClient, cluster, pdClient, err := mocktikv.NewTiKVAndPDClient("") 1919 c.Assert(err, check.IsNil) 1920 pdClient = &mockPDClient{Client: pdClient, versionGen: defaultVersionGen} 1921 tiStore, err := tikv.NewTestTiKVStore(rpcClient, pdClient, nil, nil, 0) 1922 c.Assert(err, check.IsNil) 1923 kvStorage := newStorageWithCurVersionCache(tiStore, addr1) 1924 defer kvStorage.Close() //nolint:errcheck 1925 1926 regionID := uint64(3) 1927 cluster.AddStore(1, addr1) 1928 cluster.Bootstrap(regionID, []uint64{1}, []uint64{4}, 4) 1929 1930 err = failpoint.Enable("github.com/pingcap/ticdc/cdc/kv/kvClientResolveLockInterval", "return(3)") 1931 c.Assert(err, check.IsNil) 1932 defer func() { 1933 _ = failpoint.Disable("github.com/pingcap/ticdc/cdc/kv/kvClientResolveLockInterval") 1934 }() 1935 baseAllocatedID := currentRequestID() 1936 lockresolver := txnutil.NewLockerResolver(kvStorage) 1937 isPullInit := &mockPullerInit{} 1938 grpcPool := NewGrpcPoolImpl(ctx, &security.Credential{}) 1939 defer grpcPool.Close() 1940 cdcClient := NewCDCClient(ctx, pdClient, kvStorage, grpcPool) 1941 eventCh := make(chan model.RegionFeedEvent, 10) 1942 wg.Add(1) 1943 go func() { 1944 defer wg.Done() 1945 err := cdcClient.EventFeed(ctx, regionspan.ComparableSpan{Start: []byte("a"), End: []byte("b")}, 100, false, lockresolver, isPullInit, eventCh) 1946 c.Assert(errors.Cause(err), check.Equals, context.Canceled) 1947 cdcClient.Close() //nolint:errcheck 1948 }() 1949 1950 // wait request id allocated with: new session, new request 1951 waitRequestID(c, baseAllocatedID+1) 1952 initialized := mockInitializedEvent(regionID, currentRequestID()) 1953 ch1 <- initialized 1954 physical, logical, err := pdClient.GetTS(ctx) 1955 c.Assert(err, check.IsNil) 1956 tso := oracle.ComposeTS(physical, logical) 1957 resolved := &cdcpb.ChangeDataEvent{Events: []*cdcpb.Event{ 1958 { 1959 RegionId: regionID, 1960 RequestId: currentRequestID(), 1961 Event: &cdcpb.Event_ResolvedTs{ResolvedTs: tso}, 1962 }, 1963 }} 1964 expected := []model.RegionFeedEvent{ 1965 { 1966 Resolved: &model.ResolvedSpan{ 1967 Span: regionspan.ComparableSpan{Start: []byte("a"), End: []byte("b")}, 1968 ResolvedTs: 100, 1969 }, 1970 RegionID: regionID, 1971 }, 1972 { 1973 Resolved: &model.ResolvedSpan{ 1974 Span: regionspan.ComparableSpan{Start: []byte("a"), End: []byte("b")}, 1975 ResolvedTs: tso, 1976 }, 1977 RegionID: regionID, 1978 }, 1979 } 1980 ch1 <- resolved 1981 for _, expectedEv := range expected { 1982 select { 1983 case event := <-eventCh: 1984 c.Assert(event, check.DeepEquals, expectedEv) 1985 case <-time.After(time.Second): 1986 c.Errorf("expected event %v not received", expectedEv) 1987 } 1988 } 1989 1990 // sleep 10s to simulate no resolved event longer than ResolveLockInterval 1991 // resolve lock check ticker is 5s. 1992 time.Sleep(10 * time.Second) 1993 1994 cancel() 1995 } 1996 1997 func (s *etcdSuite) testEventCommitTsFallback(c *check.C, events []*cdcpb.ChangeDataEvent) { 1998 defer s.TearDownTest(c) 1999 ctx, cancel := context.WithCancel(context.Background()) 2000 wg := &sync.WaitGroup{} 2001 2002 ch1 := make(chan *cdcpb.ChangeDataEvent, 10) 2003 srv1 := newMockChangeDataService(c, ch1) 2004 server1, addr1 := newMockService(ctx, c, srv1, wg) 2005 2006 defer func() { 2007 close(ch1) 2008 server1.Stop() 2009 wg.Wait() 2010 }() 2011 2012 rpcClient, cluster, pdClient, err := mocktikv.NewTiKVAndPDClient("") 2013 c.Assert(err, check.IsNil) 2014 pdClient = &mockPDClient{Client: pdClient, versionGen: defaultVersionGen} 2015 tiStore, err := tikv.NewTestTiKVStore(rpcClient, pdClient, nil, nil, 0) 2016 c.Assert(err, check.IsNil) 2017 kvStorage := newStorageWithCurVersionCache(tiStore, addr1) 2018 defer kvStorage.Close() //nolint:errcheck 2019 2020 regionID := uint64(3) 2021 cluster.AddStore(1, addr1) 2022 cluster.Bootstrap(regionID, []uint64{1}, []uint64{4}, 4) 2023 2024 logPanic = log.Error 2025 defer func() { 2026 logPanic = log.Panic 2027 }() 2028 2029 err = failpoint.Enable("github.com/pingcap/ticdc/cdc/kv/kvClientErrUnreachable", "return(true)") 2030 c.Assert(err, check.IsNil) 2031 defer func() { 2032 _ = failpoint.Disable("github.com/pingcap/ticdc/cdc/kv/kvClientErrUnreachable") 2033 }() 2034 baseAllocatedID := currentRequestID() 2035 lockresolver := txnutil.NewLockerResolver(kvStorage) 2036 isPullInit := &mockPullerInit{} 2037 grpcPool := NewGrpcPoolImpl(ctx, &security.Credential{}) 2038 defer grpcPool.Close() 2039 cdcClient := NewCDCClient(ctx, pdClient, kvStorage, grpcPool) 2040 eventCh := make(chan model.RegionFeedEvent, 10) 2041 var clientWg sync.WaitGroup 2042 clientWg.Add(1) 2043 go func() { 2044 defer clientWg.Done() 2045 err := cdcClient.EventFeed(ctx, regionspan.ComparableSpan{Start: []byte("a"), End: []byte("b")}, 100, false, lockresolver, isPullInit, eventCh) 2046 c.Assert(err, check.Equals, errUnreachable) 2047 cdcClient.Close() //nolint:errcheck 2048 }() 2049 2050 // wait request id allocated with: new session, new request 2051 waitRequestID(c, baseAllocatedID+1) 2052 for _, event := range events { 2053 for _, ev := range event.Events { 2054 ev.RequestId = currentRequestID() 2055 } 2056 ch1 <- event 2057 } 2058 clientWg.Wait() 2059 2060 cancel() 2061 } 2062 2063 // TestCommittedFallback tests kv client should panic when receiving a fallback committed event 2064 func (s *etcdSuite) TestCommittedFallback(c *check.C) { 2065 defer testleak.AfterTest(c)() 2066 events := []*cdcpb.ChangeDataEvent{ 2067 {Events: []*cdcpb.Event{ 2068 { 2069 RegionId: 3, 2070 RequestId: currentRequestID(), 2071 Event: &cdcpb.Event_Entries_{ 2072 Entries: &cdcpb.Event_Entries{ 2073 Entries: []*cdcpb.Event_Row{{ 2074 Type: cdcpb.Event_COMMITTED, 2075 OpType: cdcpb.Event_Row_PUT, 2076 Key: []byte("a"), 2077 Value: []byte("committed with commit ts before resolved ts"), 2078 StartTs: 92, 2079 CommitTs: 98, 2080 }}, 2081 }, 2082 }, 2083 }, 2084 }}, 2085 } 2086 s.testEventCommitTsFallback(c, events) 2087 } 2088 2089 // TestCommitFallback tests kv client should panic when receiving a fallback commit event 2090 func (s *etcdSuite) TestCommitFallback(c *check.C) { 2091 defer testleak.AfterTest(c)() 2092 events := []*cdcpb.ChangeDataEvent{ 2093 mockInitializedEvent(3, currentRequestID()), 2094 {Events: []*cdcpb.Event{ 2095 { 2096 RegionId: 3, 2097 RequestId: currentRequestID(), 2098 Event: &cdcpb.Event_Entries_{ 2099 Entries: &cdcpb.Event_Entries{ 2100 Entries: []*cdcpb.Event_Row{{ 2101 Type: cdcpb.Event_COMMIT, 2102 OpType: cdcpb.Event_Row_PUT, 2103 Key: []byte("a-commit-event-ts-fallback"), 2104 StartTs: 92, 2105 CommitTs: 98, 2106 }}, 2107 }, 2108 }, 2109 }, 2110 }}, 2111 } 2112 s.testEventCommitTsFallback(c, events) 2113 } 2114 2115 // TestDeuplicateRequest tests kv client should panic when meeting a duplicate error 2116 func (s *etcdSuite) TestDuplicateRequest(c *check.C) { 2117 defer testleak.AfterTest(c)() 2118 events := []*cdcpb.ChangeDataEvent{ 2119 {Events: []*cdcpb.Event{ 2120 { 2121 RegionId: 3, 2122 RequestId: currentRequestID(), 2123 Event: &cdcpb.Event_Error{ 2124 Error: &cdcpb.Error{ 2125 DuplicateRequest: &cdcpb.DuplicateRequest{RegionId: 3}, 2126 }, 2127 }, 2128 }, 2129 }}, 2130 } 2131 s.testEventCommitTsFallback(c, events) 2132 } 2133 2134 // testEventAfterFeedStop tests kv client can drop events sent after region feed is stopped 2135 // TODO: testEventAfterFeedStop is not stable, re-enable it after it is stable 2136 // nolint:unused 2137 func (s *etcdSuite) testEventAfterFeedStop(c *check.C) { 2138 defer testleak.AfterTest(c)() 2139 defer s.TearDownTest(c) 2140 ctx, cancel := context.WithCancel(context.Background()) 2141 wg := &sync.WaitGroup{} 2142 2143 server1Stopped := make(chan struct{}) 2144 ch1 := make(chan *cdcpb.ChangeDataEvent, 10) 2145 srv1 := newMockChangeDataService(c, ch1) 2146 server1, addr1 := newMockService(ctx, c, srv1, wg) 2147 srv1.recvLoop = func(server cdcpb.ChangeData_EventFeedServer) { 2148 defer func() { 2149 close(ch1) 2150 server1.Stop() 2151 server1Stopped <- struct{}{} 2152 }() 2153 for { 2154 _, err := server.Recv() 2155 if err != nil { 2156 log.Error("mock server error", zap.Error(err)) 2157 break 2158 } 2159 } 2160 } 2161 2162 rpcClient, cluster, pdClient, err := mocktikv.NewTiKVAndPDClient("") 2163 c.Assert(err, check.IsNil) 2164 pdClient = &mockPDClient{Client: pdClient, versionGen: defaultVersionGen} 2165 tiStore, err := tikv.NewTestTiKVStore(rpcClient, pdClient, nil, nil, 0) 2166 c.Assert(err, check.IsNil) 2167 kvStorage := newStorageWithCurVersionCache(tiStore, addr1) 2168 defer kvStorage.Close() //nolint:errcheck 2169 2170 regionID := uint64(3) 2171 cluster.AddStore(1, addr1) 2172 cluster.Bootstrap(regionID, []uint64{1}, []uint64{4}, 4) 2173 2174 // add 2s delay to simulate event feed processor has been marked stopped, but 2175 // before event feed processor is reconstruct, some duplicated events are 2176 // sent to event feed processor. 2177 err = failpoint.Enable("github.com/pingcap/ticdc/cdc/kv/kvClientSingleFeedProcessDelay", "1*sleep(2000)") 2178 c.Assert(err, check.IsNil) 2179 defer func() { 2180 _ = failpoint.Disable("github.com/pingcap/ticdc/cdc/kv/kvClientSingleFeedProcessDelay") 2181 }() 2182 baseAllocatedID := currentRequestID() 2183 lockresolver := txnutil.NewLockerResolver(kvStorage) 2184 isPullInit := &mockPullerInit{} 2185 grpcPool := NewGrpcPoolImpl(ctx, &security.Credential{}) 2186 defer grpcPool.Close() 2187 cdcClient := NewCDCClient(ctx, pdClient, kvStorage, grpcPool) 2188 eventCh := make(chan model.RegionFeedEvent, 10) 2189 wg.Add(1) 2190 go func() { 2191 defer wg.Done() 2192 err := cdcClient.EventFeed(ctx, regionspan.ComparableSpan{Start: []byte("a"), End: []byte("b")}, 100, false, lockresolver, isPullInit, eventCh) 2193 c.Assert(errors.Cause(err), check.Equals, context.Canceled) 2194 cdcClient.Close() //nolint:errcheck 2195 }() 2196 2197 // wait request id allocated with: new session, new request 2198 waitRequestID(c, baseAllocatedID+1) 2199 // an error event will mark the corresponding region feed as stopped 2200 epochNotMatch := &cdcpb.ChangeDataEvent{Events: []*cdcpb.Event{ 2201 { 2202 RegionId: 3, 2203 RequestId: currentRequestID(), 2204 Event: &cdcpb.Event_Error{ 2205 Error: &cdcpb.Error{ 2206 EpochNotMatch: &errorpb.EpochNotMatch{}, 2207 }, 2208 }, 2209 }, 2210 }} 2211 ch1 <- epochNotMatch 2212 2213 // sleep to ensure event feed processor has been marked as stopped 2214 time.Sleep(1 * time.Second) 2215 committed := &cdcpb.ChangeDataEvent{Events: []*cdcpb.Event{ 2216 { 2217 RegionId: 3, 2218 RequestId: currentRequestID(), 2219 Event: &cdcpb.Event_Entries_{ 2220 Entries: &cdcpb.Event_Entries{ 2221 Entries: []*cdcpb.Event_Row{{ 2222 Type: cdcpb.Event_COMMITTED, 2223 OpType: cdcpb.Event_Row_PUT, 2224 Key: []byte("a"), 2225 Value: []byte("committed put event before init"), 2226 StartTs: 105, 2227 CommitTs: 115, 2228 }}, 2229 }, 2230 }, 2231 }, 2232 }} 2233 initialized := mockInitializedEvent(regionID, currentRequestID()) 2234 resolved := &cdcpb.ChangeDataEvent{ 2235 ResolvedTs: &cdcpb.ResolvedTs{ 2236 Regions: []uint64{3}, 2237 Ts: 120, 2238 }, 2239 } 2240 // clone to avoid data race, these are exactly the same events 2241 committedClone := proto.Clone(committed).(*cdcpb.ChangeDataEvent) 2242 initializedClone := proto.Clone(initialized).(*cdcpb.ChangeDataEvent) 2243 resolvedClone := proto.Clone(resolved).(*cdcpb.ChangeDataEvent) 2244 ch1 <- committed 2245 ch1 <- initialized 2246 ch1 <- resolved 2247 2248 <-server1Stopped 2249 2250 var requestID uint64 2251 ch2 := make(chan *cdcpb.ChangeDataEvent, 10) 2252 srv2 := newMockChangeDataService(c, ch2) 2253 // Reuse the same listen addresss as server 1 to simulate TiKV handles the 2254 // gRPC stream terminate and reconnect. 2255 server2, _ := newMockServiceSpecificAddr(ctx, c, srv2, addr1, wg) 2256 srv2.recvLoop = func(server cdcpb.ChangeData_EventFeedServer) { 2257 for { 2258 req, err := server.Recv() 2259 if err != nil { 2260 log.Error("mock server error", zap.Error(err)) 2261 return 2262 } 2263 atomic.StoreUint64(&requestID, req.RequestId) 2264 } 2265 } 2266 defer func() { 2267 close(ch2) 2268 server2.Stop() 2269 wg.Wait() 2270 }() 2271 2272 err = retry.Run(time.Millisecond*500, 10, func() error { 2273 if atomic.LoadUint64(&requestID) > 0 { 2274 return nil 2275 } 2276 return errors.New("waiting for kv client requests received by server") 2277 }) 2278 log.Info("retry check request id", zap.Error(err)) 2279 c.Assert(err, check.IsNil) 2280 2281 // wait request id allocated with: new session, 2 * new request 2282 committedClone.Events[0].RequestId = currentRequestID() 2283 initializedClone.Events[0].RequestId = currentRequestID() 2284 ch2 <- committedClone 2285 ch2 <- initializedClone 2286 ch2 <- resolvedClone 2287 2288 expected := []model.RegionFeedEvent{ 2289 { 2290 Resolved: &model.ResolvedSpan{ 2291 Span: regionspan.ComparableSpan{Start: []byte("a"), End: []byte("b")}, 2292 ResolvedTs: 100, 2293 }, 2294 RegionID: regionID, 2295 }, 2296 { 2297 Resolved: &model.ResolvedSpan{ 2298 Span: regionspan.ComparableSpan{Start: []byte("a"), End: []byte("b")}, 2299 ResolvedTs: 100, 2300 }, 2301 RegionID: regionID, 2302 }, 2303 { 2304 Val: &model.RawKVEntry{ 2305 OpType: model.OpTypePut, 2306 Key: []byte("a"), 2307 Value: []byte("committed put event before init"), 2308 StartTs: 105, 2309 CRTs: 115, 2310 RegionID: 3, 2311 }, 2312 RegionID: 3, 2313 }, 2314 { 2315 Resolved: &model.ResolvedSpan{ 2316 Span: regionspan.ComparableSpan{Start: []byte("a"), End: []byte("b")}, 2317 ResolvedTs: 120, 2318 }, 2319 RegionID: regionID, 2320 }, 2321 } 2322 for _, expectedEv := range expected { 2323 select { 2324 case event := <-eventCh: 2325 c.Assert(event, check.DeepEquals, expectedEv) 2326 case <-time.After(time.Second): 2327 c.Errorf("expected event %v not received", expectedEv) 2328 } 2329 } 2330 cancel() 2331 } 2332 2333 func (s *etcdSuite) TestOutOfRegionRangeEvent(c *check.C) { 2334 defer testleak.AfterTest(c)() 2335 defer s.TearDownTest(c) 2336 ctx, cancel := context.WithCancel(context.Background()) 2337 wg := &sync.WaitGroup{} 2338 2339 ch1 := make(chan *cdcpb.ChangeDataEvent, 10) 2340 srv1 := newMockChangeDataService(c, ch1) 2341 server1, addr1 := newMockService(ctx, c, srv1, wg) 2342 2343 defer func() { 2344 close(ch1) 2345 server1.Stop() 2346 wg.Wait() 2347 }() 2348 2349 rpcClient, cluster, pdClient, err := mocktikv.NewTiKVAndPDClient("") 2350 c.Assert(err, check.IsNil) 2351 pdClient = &mockPDClient{Client: pdClient, versionGen: defaultVersionGen} 2352 tiStore, err := tikv.NewTestTiKVStore(rpcClient, pdClient, nil, nil, 0) 2353 c.Assert(err, check.IsNil) 2354 kvStorage := newStorageWithCurVersionCache(tiStore, addr1) 2355 defer kvStorage.Close() //nolint:errcheck 2356 2357 cluster.AddStore(1, addr1) 2358 cluster.Bootstrap(3, []uint64{1}, []uint64{4}, 4) 2359 2360 baseAllocatedID := currentRequestID() 2361 lockresolver := txnutil.NewLockerResolver(kvStorage) 2362 isPullInit := &mockPullerInit{} 2363 grpcPool := NewGrpcPoolImpl(ctx, &security.Credential{}) 2364 defer grpcPool.Close() 2365 cdcClient := NewCDCClient(ctx, pdClient, kvStorage, grpcPool) 2366 eventCh := make(chan model.RegionFeedEvent, 10) 2367 wg.Add(1) 2368 go func() { 2369 defer wg.Done() 2370 err := cdcClient.EventFeed(ctx, regionspan.ComparableSpan{Start: []byte("a"), End: []byte("b")}, 100, false, lockresolver, isPullInit, eventCh) 2371 c.Assert(errors.Cause(err), check.Equals, context.Canceled) 2372 cdcClient.Close() //nolint:errcheck 2373 }() 2374 2375 // wait request id allocated with: new session, new request 2376 waitRequestID(c, baseAllocatedID+1) 2377 2378 eventsBeforeInit := &cdcpb.ChangeDataEvent{Events: []*cdcpb.Event{ 2379 // will be filtered out 2380 { 2381 RegionId: 3, 2382 RequestId: currentRequestID(), 2383 Event: &cdcpb.Event_Entries_{ 2384 Entries: &cdcpb.Event_Entries{ 2385 Entries: []*cdcpb.Event_Row{{ 2386 Type: cdcpb.Event_COMMITTED, 2387 OpType: cdcpb.Event_Row_PUT, 2388 Key: []byte("ccc"), 2389 Value: []byte("key out of region range"), 2390 StartTs: 105, 2391 CommitTs: 115, 2392 }}, 2393 }, 2394 }, 2395 }, 2396 { 2397 RegionId: 3, 2398 RequestId: currentRequestID(), 2399 Event: &cdcpb.Event_Entries_{ 2400 Entries: &cdcpb.Event_Entries{ 2401 Entries: []*cdcpb.Event_Row{{ 2402 Type: cdcpb.Event_COMMITTED, 2403 OpType: cdcpb.Event_Row_PUT, 2404 Key: []byte("aaaa"), 2405 Value: []byte("committed put event before init"), 2406 StartTs: 105, 2407 CommitTs: 115, 2408 }}, 2409 }, 2410 }, 2411 }, 2412 }} 2413 initialized := mockInitializedEvent(3 /*regionID */, currentRequestID()) 2414 eventsAfterInit := &cdcpb.ChangeDataEvent{Events: []*cdcpb.Event{ 2415 // will be filtered out 2416 { 2417 RegionId: 3, 2418 RequestId: currentRequestID(), 2419 Event: &cdcpb.Event_Entries_{ 2420 Entries: &cdcpb.Event_Entries{ 2421 Entries: []*cdcpb.Event_Row{{ 2422 Type: cdcpb.Event_PREWRITE, 2423 OpType: cdcpb.Event_Row_PUT, 2424 Key: []byte("cccd"), 2425 Value: []byte("key out of region range"), 2426 StartTs: 135, 2427 }}, 2428 }, 2429 }, 2430 }, 2431 // will be filtered out 2432 { 2433 RegionId: 3, 2434 RequestId: currentRequestID(), 2435 Event: &cdcpb.Event_Entries_{ 2436 Entries: &cdcpb.Event_Entries{ 2437 Entries: []*cdcpb.Event_Row{{ 2438 Type: cdcpb.Event_COMMIT, 2439 OpType: cdcpb.Event_Row_PUT, 2440 Key: []byte("cccd"), 2441 StartTs: 135, 2442 CommitTs: 145, 2443 }}, 2444 }, 2445 }, 2446 }, 2447 { 2448 RegionId: 3, 2449 RequestId: currentRequestID(), 2450 Event: &cdcpb.Event_Entries_{ 2451 Entries: &cdcpb.Event_Entries{ 2452 Entries: []*cdcpb.Event_Row{{ 2453 Type: cdcpb.Event_PREWRITE, 2454 OpType: cdcpb.Event_Row_PUT, 2455 Key: []byte("a-normal-put"), 2456 Value: []byte("normal put event"), 2457 StartTs: 135, 2458 }}, 2459 }, 2460 }, 2461 }, 2462 { 2463 RegionId: 3, 2464 RequestId: currentRequestID(), 2465 Event: &cdcpb.Event_Entries_{ 2466 Entries: &cdcpb.Event_Entries{ 2467 Entries: []*cdcpb.Event_Row{{ 2468 Type: cdcpb.Event_COMMIT, 2469 OpType: cdcpb.Event_Row_PUT, 2470 Key: []byte("a-normal-put"), 2471 StartTs: 135, 2472 CommitTs: 145, 2473 }}, 2474 }, 2475 }, 2476 }, 2477 }} 2478 // batch resolved ts 2479 eventResolvedBatch := &cdcpb.ChangeDataEvent{ 2480 ResolvedTs: &cdcpb.ResolvedTs{ 2481 Regions: []uint64{3}, 2482 Ts: 145, 2483 }, 2484 } 2485 2486 expected := []model.RegionFeedEvent{ 2487 { 2488 Resolved: &model.ResolvedSpan{ 2489 Span: regionspan.ComparableSpan{Start: []byte("a"), End: []byte("b")}, 2490 ResolvedTs: 100, 2491 }, 2492 RegionID: 3, 2493 }, 2494 { 2495 Val: &model.RawKVEntry{ 2496 OpType: model.OpTypePut, 2497 Key: []byte("aaaa"), 2498 Value: []byte("committed put event before init"), 2499 StartTs: 105, 2500 CRTs: 115, 2501 RegionID: 3, 2502 }, 2503 RegionID: 3, 2504 }, 2505 { 2506 Val: &model.RawKVEntry{ 2507 OpType: model.OpTypePut, 2508 Key: []byte("a-normal-put"), 2509 Value: []byte("normal put event"), 2510 StartTs: 135, 2511 CRTs: 145, 2512 RegionID: 3, 2513 }, 2514 RegionID: 3, 2515 }, 2516 { 2517 Resolved: &model.ResolvedSpan{ 2518 Span: regionspan.ComparableSpan{Start: []byte("a"), End: []byte("b")}, 2519 ResolvedTs: 145, 2520 }, 2521 RegionID: 3, 2522 }, 2523 } 2524 2525 ch1 <- eventsBeforeInit 2526 ch1 <- initialized 2527 ch1 <- eventsAfterInit 2528 ch1 <- eventResolvedBatch 2529 2530 for _, expectedEv := range expected { 2531 select { 2532 case event := <-eventCh: 2533 c.Assert(event, check.DeepEquals, expectedEv) 2534 case <-time.After(time.Second): 2535 c.Errorf("expected event %v not received", expectedEv) 2536 } 2537 } 2538 2539 cancel() 2540 } 2541 2542 func (s *clientSuite) TestSingleRegionInfoClone(c *check.C) { 2543 defer testleak.AfterTest(c)() 2544 sri := newSingleRegionInfo( 2545 tikv.RegionVerID{}, 2546 regionspan.ComparableSpan{Start: []byte("a"), End: []byte("c")}, 2547 1000, &tikv.RPCContext{}) 2548 sri2 := sri.partialClone() 2549 sri2.ts = 2000 2550 sri2.span.End[0] = 'b' 2551 c.Assert(sri.ts, check.Equals, uint64(1000)) 2552 c.Assert(sri.span.String(), check.Equals, "[61, 63)") 2553 c.Assert(sri2.ts, check.Equals, uint64(2000)) 2554 c.Assert(sri2.span.String(), check.Equals, "[61, 62)") 2555 c.Assert(sri2.rpcCtx, check.DeepEquals, &tikv.RPCContext{}) 2556 } 2557 2558 // TestResolveLockNoCandidate tests the resolved ts manager can work normally 2559 // when no region exceeds reslove lock interval, that is what candidate means. 2560 func (s *etcdSuite) TestResolveLockNoCandidate(c *check.C) { 2561 defer testleak.AfterTest(c)() 2562 defer s.TearDownTest(c) 2563 ctx, cancel := context.WithCancel(context.Background()) 2564 wg := &sync.WaitGroup{} 2565 2566 ch1 := make(chan *cdcpb.ChangeDataEvent, 10) 2567 srv1 := newMockChangeDataService(c, ch1) 2568 server1, addr1 := newMockService(ctx, c, srv1, wg) 2569 2570 defer func() { 2571 close(ch1) 2572 server1.Stop() 2573 wg.Wait() 2574 }() 2575 2576 rpcClient, cluster, pdClient, err := mocktikv.NewTiKVAndPDClient("") 2577 c.Assert(err, check.IsNil) 2578 pdClient = &mockPDClient{Client: pdClient, versionGen: defaultVersionGen} 2579 tiStore, err := tikv.NewTestTiKVStore(rpcClient, pdClient, nil, nil, 0) 2580 c.Assert(err, check.IsNil) 2581 kvStorage := newStorageWithCurVersionCache(tiStore, addr1) 2582 defer kvStorage.Close() //nolint:errcheck 2583 2584 regionID := uint64(3) 2585 storeID := uint64(1) 2586 peerID := uint64(4) 2587 cluster.AddStore(storeID, addr1) 2588 cluster.Bootstrap(regionID, []uint64{storeID}, []uint64{peerID}, peerID) 2589 2590 baseAllocatedID := currentRequestID() 2591 lockresolver := txnutil.NewLockerResolver(kvStorage) 2592 isPullInit := &mockPullerInit{} 2593 grpcPool := NewGrpcPoolImpl(ctx, &security.Credential{}) 2594 defer grpcPool.Close() 2595 cdcClient := NewCDCClient(ctx, pdClient, kvStorage, grpcPool) 2596 eventCh := make(chan model.RegionFeedEvent, 10) 2597 wg.Add(1) 2598 go func() { 2599 defer wg.Done() 2600 err := cdcClient.EventFeed(ctx, regionspan.ComparableSpan{Start: []byte("a"), End: []byte("b")}, 100, false, lockresolver, isPullInit, eventCh) 2601 c.Assert(errors.Cause(err), check.Equals, context.Canceled) 2602 cdcClient.Close() //nolint:errcheck 2603 }() 2604 2605 // wait request id allocated with: new session, new request 2606 waitRequestID(c, baseAllocatedID+1) 2607 initialized := mockInitializedEvent(regionID, currentRequestID()) 2608 ch1 <- initialized 2609 2610 var wg2 sync.WaitGroup 2611 wg2.Add(1) 2612 go func() { 2613 defer wg2.Done() 2614 for i := 0; i < 6; i++ { 2615 physical, logical, err := pdClient.GetTS(ctx) 2616 c.Assert(err, check.IsNil) 2617 tso := oracle.ComposeTS(physical, logical) 2618 resolved := &cdcpb.ChangeDataEvent{Events: []*cdcpb.Event{ 2619 { 2620 RegionId: regionID, 2621 RequestId: currentRequestID(), 2622 Event: &cdcpb.Event_ResolvedTs{ResolvedTs: tso}, 2623 }, 2624 }} 2625 ch1 <- resolved 2626 select { 2627 case event := <-eventCh: 2628 c.Assert(event.Resolved, check.NotNil) 2629 case <-time.After(time.Second): 2630 c.Error("resovled event not received") 2631 } 2632 // will sleep 6s totally, to ensure resolve lock fired once 2633 time.Sleep(time.Second) 2634 } 2635 }() 2636 2637 wg2.Wait() 2638 cancel() 2639 } 2640 2641 // TestFailRegionReentrant tests one region could be failover multiple times, 2642 // kv client must avoid duplicated `onRegionFail` call for the same region. 2643 // In this test 2644 // 1. An `unknownErr` is sent to kv client first to trigger `handleSingleRegionError` in region worker. 2645 // 2. We delay the kv client to re-create a new region request by 500ms via failpoint. 2646 // 3. Before new region request is fired, simulate kv client `stream.Recv` returns an error, the stream 2647 // handler will signal region worker to exit, which will evict all active region states then. 2648 func (s *etcdSuite) TestFailRegionReentrant(c *check.C) { 2649 defer testleak.AfterTest(c)() 2650 defer s.TearDownTest(c) 2651 ctx, cancel := context.WithCancel(context.Background()) 2652 wg := &sync.WaitGroup{} 2653 2654 ch1 := make(chan *cdcpb.ChangeDataEvent, 10) 2655 srv1 := newMockChangeDataService(c, ch1) 2656 server1, addr1 := newMockService(ctx, c, srv1, wg) 2657 2658 defer func() { 2659 close(ch1) 2660 server1.Stop() 2661 wg.Wait() 2662 }() 2663 2664 rpcClient, cluster, pdClient, err := mocktikv.NewTiKVAndPDClient("") 2665 c.Assert(err, check.IsNil) 2666 pdClient = &mockPDClient{Client: pdClient, versionGen: defaultVersionGen} 2667 tiStore, err := tikv.NewTestTiKVStore(rpcClient, pdClient, nil, nil, 0) 2668 c.Assert(err, check.IsNil) 2669 kvStorage := newStorageWithCurVersionCache(tiStore, addr1) 2670 defer kvStorage.Close() //nolint:errcheck 2671 2672 regionID := uint64(3) 2673 cluster.AddStore(1, addr1) 2674 cluster.Bootstrap(regionID, []uint64{1}, []uint64{4}, 4) 2675 2676 err = failpoint.Enable("github.com/pingcap/ticdc/cdc/kv/kvClientRegionReentrantError", "1*return(\"ok\")->1*return(\"error\")") 2677 c.Assert(err, check.IsNil) 2678 err = failpoint.Enable("github.com/pingcap/ticdc/cdc/kv/kvClientRegionReentrantErrorDelay", "sleep(500)") 2679 c.Assert(err, check.IsNil) 2680 defer func() { 2681 _ = failpoint.Disable("github.com/pingcap/ticdc/cdc/kv/kvClientRegionReentrantError") 2682 _ = failpoint.Disable("github.com/pingcap/ticdc/cdc/kv/kvClientRegionReentrantErrorDelay") 2683 }() 2684 baseAllocatedID := currentRequestID() 2685 lockresolver := txnutil.NewLockerResolver(kvStorage.(tikv.Storage)) 2686 isPullInit := &mockPullerInit{} 2687 grpcPool := NewGrpcPoolImpl(ctx, &security.Credential{}) 2688 defer grpcPool.Close() 2689 cdcClient := NewCDCClient(ctx, pdClient, kvStorage.(tikv.Storage), grpcPool) 2690 eventCh := make(chan model.RegionFeedEvent, 10) 2691 wg.Add(1) 2692 go func() { 2693 defer wg.Done() 2694 err := cdcClient.EventFeed(ctx, regionspan.ComparableSpan{Start: []byte("a"), End: []byte("b")}, 100, false, lockresolver, isPullInit, eventCh) 2695 c.Assert(errors.Cause(err), check.Equals, context.Canceled) 2696 cdcClient.Close() //nolint:errcheck 2697 }() 2698 2699 // wait request id allocated with: new session, new request 2700 waitRequestID(c, baseAllocatedID+1) 2701 unknownErr := &cdcpb.ChangeDataEvent{Events: []*cdcpb.Event{ 2702 { 2703 RegionId: 3, 2704 RequestId: currentRequestID(), 2705 Event: &cdcpb.Event_Error{ 2706 Error: &cdcpb.Error{}, 2707 }, 2708 }, 2709 }} 2710 ch1 <- unknownErr 2711 // use a fake event to trigger one more stream.Recv 2712 initialized := mockInitializedEvent(regionID, currentRequestID()) 2713 ch1 <- initialized 2714 // since re-establish new region request is delayed by `kvClientRegionReentrantErrorDelay` 2715 // there will be reentrant region failover, the kv client should not panic. 2716 time.Sleep(time.Second) 2717 cancel() 2718 } 2719 2720 // TestClientV1UnlockRangeReentrant tests clientV1 can handle region reconnection 2721 // with unstable TiKV store correctly. The test workflow is as follows: 2722 // 1. kv client establishes two regions request, naming region-1, region-2, they 2723 // belong to the same TiKV store. 2724 // 2. The region-1 is firstly established, yet region-2 has some delay after its 2725 // region state is inserted into `pendingRegions` 2726 // 3. At this time the TiKV store crashes and `stream.Recv` returns error. In the 2727 // defer function of `receiveFromStream`, all pending regions will be cleaned 2728 // up, which means the region lock will be unlocked once for these regions. 2729 // 4. In step-2, the region-2 continues to run, it can't get store stream which 2730 // has been deleted in step-3, so it will create new stream but fails because 2731 // of unstable TiKV store, at this point, the kv client should handle with the 2732 // pending region correctly. 2733 func (s *etcdSuite) TestClientV1UnlockRangeReentrant(c *check.C) { 2734 defer testleak.AfterTest(c)() 2735 defer s.TearDownTest(c) 2736 2737 ctx, cancel := context.WithCancel(context.Background()) 2738 wg := &sync.WaitGroup{} 2739 2740 ch1 := make(chan *cdcpb.ChangeDataEvent, 10) 2741 srv1 := newMockChangeDataService(c, ch1) 2742 server1, addr1 := newMockService(ctx, c, srv1, wg) 2743 2744 rpcClient, cluster, pdClient, err := mocktikv.NewTiKVAndPDClient("") 2745 c.Assert(err, check.IsNil) 2746 pdClient = &mockPDClient{Client: pdClient, versionGen: defaultVersionGen} 2747 tiStore, err := tikv.NewTestTiKVStore(rpcClient, pdClient, nil, nil, 0) 2748 c.Assert(err, check.IsNil) 2749 kvStorage := newStorageWithCurVersionCache(tiStore, addr1) 2750 defer kvStorage.Close() //nolint:errcheck 2751 2752 regionID3 := uint64(3) 2753 regionID4 := uint64(4) 2754 cluster.AddStore(1, addr1) 2755 cluster.Bootstrap(regionID3, []uint64{1}, []uint64{4}, 4) 2756 cluster.SplitRaw(regionID3, regionID4, []byte("b"), []uint64{5}, 5) 2757 2758 err = failpoint.Enable("github.com/pingcap/ticdc/cdc/kv/kvClientStreamRecvError", "1*return(\"injected stream recv error\")") 2759 c.Assert(err, check.IsNil) 2760 err = failpoint.Enable("github.com/pingcap/ticdc/cdc/kv/kvClientPendingRegionDelay", "1*sleep(0)->1*sleep(2000)") 2761 c.Assert(err, check.IsNil) 2762 defer func() { 2763 _ = failpoint.Disable("github.com/pingcap/ticdc/cdc/kv/kvClientStreamRecvError") 2764 _ = failpoint.Disable("github.com/pingcap/ticdc/cdc/kv/kvClientPendingRegionDelay") 2765 }() 2766 lockresolver := txnutil.NewLockerResolver(kvStorage) 2767 isPullInit := &mockPullerInit{} 2768 grpcPool := NewGrpcPoolImpl(ctx, &security.Credential{}) 2769 defer grpcPool.Close() 2770 cdcClient := NewCDCClient(ctx, pdClient, kvStorage, grpcPool) 2771 eventCh := make(chan model.RegionFeedEvent, 10) 2772 wg.Add(1) 2773 go func() { 2774 defer wg.Done() 2775 err := cdcClient.EventFeed(ctx, regionspan.ComparableSpan{Start: []byte("a"), End: []byte("c")}, 100, false, lockresolver, isPullInit, eventCh) 2776 c.Assert(errors.Cause(err), check.Equals, context.Canceled) 2777 cdcClient.Close() //nolint:errcheck 2778 }() 2779 2780 // wait the second region is scheduled 2781 time.Sleep(time.Millisecond * 500) 2782 close(ch1) 2783 server1.Stop() 2784 // wait the kvClientPendingRegionDelay ends, and the second region is processed 2785 time.Sleep(time.Second * 2) 2786 cancel() 2787 wg.Wait() 2788 } 2789 2790 // TestClientErrNoPendingRegion has the similar procedure with TestClientV1UnlockRangeReentrant 2791 // The difference is the delay injected point for region 2 2792 func (s *etcdSuite) TestClientV1ErrNoPendingRegion(c *check.C) { 2793 defer testleak.AfterTest(c)() 2794 defer s.TearDownTest(c) 2795 2796 ctx, cancel := context.WithCancel(context.Background()) 2797 wg := &sync.WaitGroup{} 2798 2799 ch1 := make(chan *cdcpb.ChangeDataEvent, 10) 2800 srv1 := newMockChangeDataService(c, ch1) 2801 server1, addr1 := newMockService(ctx, c, srv1, wg) 2802 2803 rpcClient, cluster, pdClient, err := mocktikv.NewTiKVAndPDClient("") 2804 c.Assert(err, check.IsNil) 2805 pdClient = &mockPDClient{Client: pdClient, versionGen: defaultVersionGen} 2806 tiStore, err := tikv.NewTestTiKVStore(rpcClient, pdClient, nil, nil, 0) 2807 c.Assert(err, check.IsNil) 2808 kvStorage := newStorageWithCurVersionCache(tiStore, addr1) 2809 defer kvStorage.Close() //nolint:errcheck 2810 2811 regionID3 := uint64(3) 2812 regionID4 := uint64(4) 2813 cluster.AddStore(1, addr1) 2814 cluster.Bootstrap(regionID3, []uint64{1}, []uint64{4}, 4) 2815 cluster.SplitRaw(regionID3, regionID4, []byte("b"), []uint64{5}, 5) 2816 2817 err = failpoint.Enable("github.com/pingcap/ticdc/cdc/kv/kvClientStreamRecvError", "1*return(\"injected stream recv error\")") 2818 c.Assert(err, check.IsNil) 2819 err = failpoint.Enable("github.com/pingcap/ticdc/cdc/kv/kvClientPendingRegionDelay", "1*sleep(0)->2*sleep(1000)") 2820 c.Assert(err, check.IsNil) 2821 err = failpoint.Enable("github.com/pingcap/ticdc/cdc/kv/kvClientStreamCloseDelay", "sleep(2000)") 2822 c.Assert(err, check.IsNil) 2823 defer func() { 2824 _ = failpoint.Disable("github.com/pingcap/ticdc/cdc/kv/kvClientStreamRecvError") 2825 _ = failpoint.Disable("github.com/pingcap/ticdc/cdc/kv/kvClientPendingRegionDelay") 2826 _ = failpoint.Disable("github.com/pingcap/ticdc/cdc/kv/kvClientStreamCloseDelay") 2827 }() 2828 lockresolver := txnutil.NewLockerResolver(kvStorage) 2829 isPullInit := &mockPullerInit{} 2830 grpcPool := NewGrpcPoolImpl(ctx, &security.Credential{}) 2831 defer grpcPool.Close() 2832 cdcClient := NewCDCClient(ctx, pdClient, kvStorage, grpcPool) 2833 eventCh := make(chan model.RegionFeedEvent, 10) 2834 wg.Add(1) 2835 go func() { 2836 defer wg.Done() 2837 err := cdcClient.EventFeed(ctx, regionspan.ComparableSpan{Start: []byte("a"), End: []byte("c")}, 100, false, lockresolver, isPullInit, eventCh) 2838 c.Assert(errors.Cause(err), check.Equals, context.Canceled) 2839 cdcClient.Close() //nolint:errcheck 2840 }() 2841 2842 baseAllocatedID := currentRequestID() 2843 // wait the second region is scheduled 2844 time.Sleep(time.Millisecond * 500) 2845 waitRequestID(c, baseAllocatedID+1) 2846 initialized := mockInitializedEvent(regionID3, currentRequestID()) 2847 ch1 <- initialized 2848 waitRequestID(c, baseAllocatedID+2) 2849 initialized = mockInitializedEvent(regionID4, currentRequestID()) 2850 ch1 <- initialized 2851 // wait the kvClientPendingRegionDelay ends, and the second region is processed 2852 time.Sleep(time.Second * 2) 2853 cancel() 2854 close(ch1) 2855 server1.Stop() 2856 wg.Wait() 2857 } 2858 2859 // TestKVClientForceReconnect force reconnect gRPC stream can work 2860 func (s *etcdSuite) TestKVClientForceReconnect(c *check.C) { 2861 defer testleak.AfterTest(c)() 2862 defer s.TearDownTest(c) 2863 2864 // test kv client v1 2865 s.testKVClientForceReconnect(c) 2866 2867 // enableKVClientV2 = true 2868 // s.testKVClientForceReconnect(c) 2869 } 2870 2871 func (s *etcdSuite) testKVClientForceReconnect(c *check.C) { 2872 ctx, cancel := context.WithCancel(context.Background()) 2873 wg := &sync.WaitGroup{} 2874 2875 server1Stopped := make(chan struct{}) 2876 ch1 := make(chan *cdcpb.ChangeDataEvent, 10) 2877 srv1 := newMockChangeDataService(c, ch1) 2878 server1, addr1 := newMockService(ctx, c, srv1, wg) 2879 srv1.recvLoop = func(server cdcpb.ChangeData_EventFeedServer) { 2880 defer func() { 2881 close(ch1) 2882 server1.Stop() 2883 server1Stopped <- struct{}{} 2884 }() 2885 for { 2886 _, err := server.Recv() 2887 if err != nil { 2888 log.Error("mock server error", zap.Error(err)) 2889 break 2890 } 2891 } 2892 } 2893 2894 rpcClient, cluster, pdClient, err := mocktikv.NewTiKVAndPDClient("") 2895 c.Assert(err, check.IsNil) 2896 pdClient = &mockPDClient{Client: pdClient, versionGen: defaultVersionGen} 2897 tiStore, err := tikv.NewTestTiKVStore(rpcClient, pdClient, nil, nil, 0) 2898 c.Assert(err, check.IsNil) 2899 kvStorage := newStorageWithCurVersionCache(tiStore, addr1) 2900 defer kvStorage.Close() //nolint:errcheck 2901 2902 regionID3 := uint64(3) 2903 cluster.AddStore(1, addr1) 2904 cluster.Bootstrap(regionID3, []uint64{1}, []uint64{4}, 4) 2905 2906 err = failpoint.Enable("github.com/pingcap/ticdc/cdc/kv/kvClientResolveLockInterval", "return(1)") 2907 c.Assert(err, check.IsNil) 2908 originalReconnectInterval := reconnectInterval 2909 reconnectInterval = 3 * time.Second 2910 defer func() { 2911 _ = failpoint.Disable("github.com/pingcap/ticdc/cdc/kv/kvClientResolveLockInterval") 2912 reconnectInterval = originalReconnectInterval 2913 }() 2914 2915 lockresolver := txnutil.NewLockerResolver(kvStorage) 2916 isPullInit := &mockPullerInit{} 2917 grpcPool := NewGrpcPoolImpl(ctx, &security.Credential{}) 2918 defer grpcPool.Close() 2919 cdcClient := NewCDCClient(ctx, pdClient, kvStorage, grpcPool) 2920 eventCh := make(chan model.RegionFeedEvent, 10) 2921 wg.Add(1) 2922 go func() { 2923 defer wg.Done() 2924 err := cdcClient.EventFeed(ctx, regionspan.ComparableSpan{Start: []byte("a"), End: []byte("c")}, 100, false, lockresolver, isPullInit, eventCh) 2925 c.Assert(errors.Cause(err), check.Equals, context.Canceled) 2926 cdcClient.Close() //nolint:errcheck 2927 }() 2928 2929 baseAllocatedID := currentRequestID() 2930 waitRequestID(c, baseAllocatedID+1) 2931 initialized := mockInitializedEvent(regionID3, currentRequestID()) 2932 ch1 <- initialized 2933 2934 <-server1Stopped 2935 2936 var requestIds sync.Map 2937 ch2 := make(chan *cdcpb.ChangeDataEvent, 10) 2938 srv2 := newMockChangeDataService(c, ch2) 2939 srv2.recvLoop = func(server cdcpb.ChangeData_EventFeedServer) { 2940 for { 2941 req, err := server.Recv() 2942 if err != nil { 2943 log.Error("mock server error", zap.Error(err)) 2944 return 2945 } 2946 requestIds.Store(req.RegionId, req.RequestId) 2947 } 2948 } 2949 // Reuse the same listen addresss as server 1 to simulate TiKV handles the 2950 // gRPC stream terminate and reconnect. 2951 server2, _ := newMockServiceSpecificAddr(ctx, c, srv2, addr1, wg) 2952 defer func() { 2953 close(ch2) 2954 server2.Stop() 2955 wg.Wait() 2956 }() 2957 2958 // The second TiKV could start up slowly, which causes the kv client retries 2959 // to TiKV for more than one time, so we can't determine the correct requestID 2960 // here, we must use the real request ID received by TiKV server 2961 err = retry.Do(context.Background(), func() error { 2962 _, ok := requestIds.Load(regionID3) 2963 if ok { 2964 return nil 2965 } 2966 return errors.New("waiting for kv client requests received by server") 2967 }, retry.WithBackoffBaseDelay(300), retry.WithBackoffMaxDelay(60*1000), retry.WithMaxTries(10)) 2968 2969 c.Assert(err, check.IsNil) 2970 requestID, _ := requestIds.Load(regionID3) 2971 2972 initialized = mockInitializedEvent(regionID3, requestID.(uint64)) 2973 ch2 <- initialized 2974 2975 resolved := &cdcpb.ChangeDataEvent{Events: []*cdcpb.Event{ 2976 { 2977 RegionId: regionID3, 2978 RequestId: requestID.(uint64), 2979 Event: &cdcpb.Event_ResolvedTs{ResolvedTs: 135}, 2980 }, 2981 }} 2982 ch2 <- resolved 2983 2984 expected := model.RegionFeedEvent{ 2985 Resolved: &model.ResolvedSpan{ 2986 Span: regionspan.ComparableSpan{Start: []byte("a"), End: []byte("c")}, 2987 ResolvedTs: 135, 2988 }, 2989 RegionID: regionID3, 2990 } 2991 2992 eventLoop: 2993 for { 2994 select { 2995 case ev := <-eventCh: 2996 if ev.Resolved != nil && ev.Resolved.ResolvedTs == uint64(100) { 2997 continue 2998 } 2999 c.Assert(ev, check.DeepEquals, expected) 3000 break eventLoop 3001 case <-time.After(time.Second): 3002 c.Errorf("expected event %v not received", expected) 3003 } 3004 } 3005 3006 cancel() 3007 } 3008 3009 // TestConcurrentProcessRangeRequest when region range request channel is full, 3010 // the kv client can process it correctly without deadlock. This is more likely 3011 // to happen when region split and merge frequently and large stale requests exist. 3012 func (s *etcdSuite) TestConcurrentProcessRangeRequest(c *check.C) { 3013 defer testleak.AfterTest(c)() 3014 defer s.TearDownTest(c) 3015 ctx, cancel := context.WithCancel(context.Background()) 3016 wg := &sync.WaitGroup{} 3017 3018 requestIDs := new(sync.Map) 3019 ch1 := make(chan *cdcpb.ChangeDataEvent, 10) 3020 srv1 := newMockChangeDataService(c, ch1) 3021 srv1.recvLoop = func(server cdcpb.ChangeData_EventFeedServer) { 3022 for { 3023 req, err := server.Recv() 3024 if err != nil { 3025 return 3026 } 3027 requestIDs.Store(req.RegionId, req.RequestId) 3028 } 3029 } 3030 server1, addr1 := newMockService(ctx, c, srv1, wg) 3031 3032 defer func() { 3033 close(ch1) 3034 server1.Stop() 3035 wg.Wait() 3036 }() 3037 3038 rpcClient, cluster, pdClient, err := mocktikv.NewTiKVAndPDClient("") 3039 c.Assert(err, check.IsNil) 3040 pdClient = &mockPDClient{Client: pdClient, versionGen: defaultVersionGen} 3041 tiStore, err := tikv.NewTestTiKVStore(rpcClient, pdClient, nil, nil, 0) 3042 c.Assert(err, check.IsNil) 3043 kvStorage := newStorageWithCurVersionCache(tiStore, addr1) 3044 defer kvStorage.Close() //nolint:errcheck 3045 3046 storeID := uint64(1) 3047 regionID := uint64(1000) 3048 peerID := regionID + 1 3049 cluster.AddStore(storeID, addr1) 3050 cluster.Bootstrap(regionID, []uint64{storeID}, []uint64{peerID}, peerID) 3051 3052 err = failpoint.Enable("github.com/pingcap/ticdc/cdc/kv/kvClientMockRangeLock", "1*return(20)") 3053 c.Assert(err, check.IsNil) 3054 defer func() { 3055 _ = failpoint.Disable("github.com/pingcap/ticdc/cdc/kv/kvClientMockRangeLock") 3056 }() 3057 lockresolver := txnutil.NewLockerResolver(kvStorage) 3058 isPullInit := &mockPullerInit{} 3059 grpcPool := NewGrpcPoolImpl(ctx, &security.Credential{}) 3060 defer grpcPool.Close() 3061 cdcClient := NewCDCClient(ctx, pdClient, kvStorage, grpcPool) 3062 eventCh := make(chan model.RegionFeedEvent, 100) 3063 wg.Add(1) 3064 go func() { 3065 defer wg.Done() 3066 err := cdcClient.EventFeed(ctx, regionspan.ComparableSpan{Start: []byte("a"), End: []byte("z")}, 100, false, lockresolver, isPullInit, eventCh) 3067 c.Assert(errors.Cause(err), check.Equals, context.Canceled) 3068 cdcClient.Close() //nolint:errcheck 3069 }() 3070 3071 // the kv client is blocked by failpoint injection, and after region has split 3072 // into more sub regions, the kv client will continue to handle and will find 3073 // stale region requests (which is also caused by failpoint injection). 3074 regionNum := 20 3075 for i := 1; i < regionNum; i++ { 3076 regionID := uint64(i + 1000) 3077 peerID := regionID + 1 3078 // split regions to [min, b1001), [b1001, b1002), ... [bN, max) 3079 cluster.SplitRaw(regionID-1, regionID, []byte(fmt.Sprintf("b%d", regionID)), []uint64{peerID}, peerID) 3080 } 3081 3082 // wait for all regions requested from cdc kv client 3083 err = retry.Do(context.Background(), func() error { 3084 count := 0 3085 requestIDs.Range(func(_, _ interface{}) bool { 3086 count++ 3087 return true 3088 }) 3089 if count == regionNum { 3090 return nil 3091 } 3092 return errors.Errorf("region number %d is not as expected %d", count, regionNum) 3093 }, retry.WithBackoffBaseDelay(200), retry.WithBackoffMaxDelay(60*1000), retry.WithMaxTries(20)) 3094 3095 c.Assert(err, check.IsNil) 3096 3097 // send initialized event and a resolved ts event to each region 3098 requestIDs.Range(func(key, value interface{}) bool { 3099 regionID := key.(uint64) 3100 requestID := value.(uint64) 3101 initialized := mockInitializedEvent(regionID, requestID) 3102 ch1 <- initialized 3103 resolved := &cdcpb.ChangeDataEvent{Events: []*cdcpb.Event{ 3104 { 3105 RegionId: regionID, 3106 RequestId: requestID, 3107 Event: &cdcpb.Event_ResolvedTs{ResolvedTs: 120}, 3108 }, 3109 }} 3110 ch1 <- resolved 3111 return true 3112 }) 3113 3114 resolvedCount := 0 3115 checkEvent: 3116 for { 3117 select { 3118 case <-eventCh: 3119 resolvedCount++ 3120 log.Info("receive resolved count", zap.Int("count", resolvedCount)) 3121 if resolvedCount == regionNum*2 { 3122 break checkEvent 3123 } 3124 case <-time.After(time.Second): 3125 c.Errorf("no more events received") 3126 } 3127 } 3128 3129 cancel() 3130 } 3131 3132 // TestEvTimeUpdate creates a new event feed, send N committed events every 100ms, 3133 // use failpoint to set reconnect interval to 1s, the last event time of region 3134 // should be updated correctly and no reconnect is triggered 3135 func (s *etcdSuite) TestEvTimeUpdate(c *check.C) { 3136 defer testleak.AfterTest(c)() 3137 3138 defer s.TearDownTest(c) 3139 ctx, cancel := context.WithCancel(context.Background()) 3140 wg := &sync.WaitGroup{} 3141 3142 ch1 := make(chan *cdcpb.ChangeDataEvent, 10) 3143 srv1 := newMockChangeDataService(c, ch1) 3144 server1, addr1 := newMockService(ctx, c, srv1, wg) 3145 3146 defer func() { 3147 close(ch1) 3148 server1.Stop() 3149 wg.Wait() 3150 }() 3151 3152 rpcClient, cluster, pdClient, err := mocktikv.NewTiKVAndPDClient("") 3153 c.Assert(err, check.IsNil) 3154 pdClient = &mockPDClient{Client: pdClient, versionGen: defaultVersionGen} 3155 tiStore, err := tikv.NewTestTiKVStore(rpcClient, pdClient, nil, nil, 0) 3156 c.Assert(err, check.IsNil) 3157 kvStorage := newStorageWithCurVersionCache(tiStore, addr1) 3158 defer kvStorage.Close() //nolint:errcheck 3159 3160 cluster.AddStore(1, addr1) 3161 cluster.Bootstrap(3, []uint64{1}, []uint64{4}, 4) 3162 3163 originalReconnectInterval := reconnectInterval 3164 reconnectInterval = 1500 * time.Millisecond 3165 err = failpoint.Enable("github.com/pingcap/ticdc/cdc/kv/kvClientCheckUnInitRegionInterval", "return(2)") 3166 c.Assert(err, check.IsNil) 3167 defer func() { 3168 _ = failpoint.Disable("github.com/pingcap/ticdc/cdc/kv/kvClientCheckUnInitRegionInterval") 3169 reconnectInterval = originalReconnectInterval 3170 }() 3171 3172 baseAllocatedID := currentRequestID() 3173 lockresolver := txnutil.NewLockerResolver(kvStorage) 3174 isPullInit := &mockPullerInit{} 3175 grpcPool := NewGrpcPoolImpl(ctx, &security.Credential{}) 3176 defer grpcPool.Close() 3177 cdcClient := NewCDCClient(ctx, pdClient, kvStorage, grpcPool) 3178 eventCh := make(chan model.RegionFeedEvent, 10) 3179 wg.Add(1) 3180 go func() { 3181 defer wg.Done() 3182 err := cdcClient.EventFeed(ctx, regionspan.ComparableSpan{Start: []byte("a"), End: []byte("b")}, 100, false, lockresolver, isPullInit, eventCh) 3183 c.Assert(errors.Cause(err), check.Equals, context.Canceled) 3184 cdcClient.Close() //nolint:errcheck 3185 }() 3186 3187 // wait request id allocated with: new session, new request 3188 waitRequestID(c, baseAllocatedID+1) 3189 3190 eventCount := 20 3191 for i := 0; i < eventCount; i++ { 3192 events := &cdcpb.ChangeDataEvent{Events: []*cdcpb.Event{ 3193 { 3194 RegionId: 3, 3195 RequestId: currentRequestID(), 3196 Event: &cdcpb.Event_Entries_{ 3197 Entries: &cdcpb.Event_Entries{ 3198 Entries: []*cdcpb.Event_Row{{ 3199 Type: cdcpb.Event_COMMITTED, 3200 OpType: cdcpb.Event_Row_PUT, 3201 Key: []byte("aaaa"), 3202 Value: []byte("committed put event before init"), 3203 StartTs: 105, 3204 CommitTs: 115, 3205 }}, 3206 }, 3207 }, 3208 }, 3209 }} 3210 ch1 <- events 3211 time.Sleep(time.Millisecond * 100) 3212 } 3213 3214 expected := []model.RegionFeedEvent{ 3215 { 3216 Resolved: &model.ResolvedSpan{ 3217 Span: regionspan.ComparableSpan{Start: []byte("a"), End: []byte("b")}, 3218 ResolvedTs: 100, 3219 }, 3220 RegionID: 3, 3221 }, 3222 { 3223 Val: &model.RawKVEntry{ 3224 OpType: model.OpTypePut, 3225 Key: []byte("aaaa"), 3226 Value: []byte("committed put event before init"), 3227 StartTs: 105, 3228 CRTs: 115, 3229 RegionID: 3, 3230 }, 3231 RegionID: 3, 3232 }, 3233 } 3234 3235 for i := 0; i < eventCount+1; i++ { 3236 select { 3237 case event := <-eventCh: 3238 if i == 0 { 3239 c.Assert(event, check.DeepEquals, expected[0]) 3240 } else { 3241 c.Assert(event, check.DeepEquals, expected[1]) 3242 } 3243 case <-time.After(time.Second): 3244 c.Errorf("expected event not received, %d received", i) 3245 } 3246 } 3247 3248 cancel() 3249 } 3250 3251 // TestRegionWorkerExitWhenIsIdle tests region worker can exit, and cancel gRPC 3252 // stream automatically when it is idle. 3253 func (s *etcdSuite) TestRegionWorkerExitWhenIsIdle(c *check.C) { 3254 defer testleak.AfterTest(c)() 3255 defer s.TearDownTest(c) 3256 3257 if !enableKVClientV2 { 3258 return 3259 } 3260 3261 ctx, cancel := context.WithCancel(context.Background()) 3262 wg := &sync.WaitGroup{} 3263 3264 server1Stopped := make(chan struct{}) 3265 ch1 := make(chan *cdcpb.ChangeDataEvent, 10) 3266 srv1 := newMockChangeDataService(c, ch1) 3267 server1, addr1 := newMockService(ctx, c, srv1, wg) 3268 srv1.recvLoop = func(server cdcpb.ChangeData_EventFeedServer) { 3269 defer func() { 3270 close(ch1) 3271 server1.Stop() 3272 server1Stopped <- struct{}{} 3273 }() 3274 for { 3275 _, err := server.Recv() 3276 if err != nil { 3277 log.Error("mock server error", zap.Error(err)) 3278 break 3279 } 3280 } 3281 } 3282 3283 rpcClient, cluster, pdClient, err := mocktikv.NewTiKVAndPDClient("") 3284 c.Assert(err, check.IsNil) 3285 pdClient = &mockPDClient{Client: pdClient, versionGen: defaultVersionGen} 3286 tiStore, err := tikv.NewTestTiKVStore(rpcClient, pdClient, nil, nil, 0) 3287 c.Assert(err, check.IsNil) 3288 kvStorage := newStorageWithCurVersionCache(tiStore, addr1) 3289 defer kvStorage.Close() //nolint:errcheck 3290 3291 regionID := uint64(3) 3292 cluster.AddStore(1, addr1) 3293 cluster.Bootstrap(regionID, []uint64{1}, []uint64{4}, 4) 3294 3295 baseAllocatedID := currentRequestID() 3296 lockresolver := txnutil.NewLockerResolver(kvStorage) 3297 isPullInit := &mockPullerInit{} 3298 grpcPool := NewGrpcPoolImpl(ctx, &security.Credential{}) 3299 defer grpcPool.Close() 3300 cdcClient := NewCDCClient(ctx, pdClient, kvStorage, grpcPool) 3301 eventCh := make(chan model.RegionFeedEvent, 10) 3302 wg.Add(1) 3303 go func() { 3304 defer wg.Done() 3305 err := cdcClient.EventFeed(ctx, regionspan.ComparableSpan{Start: []byte("a"), End: []byte("b")}, 100, false, lockresolver, isPullInit, eventCh) 3306 c.Assert(errors.Cause(err), check.Equals, context.Canceled) 3307 cdcClient.Close() //nolint:errcheck 3308 }() 3309 3310 // wait request id allocated with: new session, new request 3311 waitRequestID(c, baseAllocatedID+1) 3312 // an error event will mark the corresponding region feed as stopped 3313 epochNotMatch := &cdcpb.ChangeDataEvent{Events: []*cdcpb.Event{ 3314 { 3315 RegionId: 3, 3316 RequestId: currentRequestID(), 3317 Event: &cdcpb.Event_Error{ 3318 Error: &cdcpb.Error{ 3319 EpochNotMatch: &errorpb.EpochNotMatch{}, 3320 }, 3321 }, 3322 }, 3323 }} 3324 ch1 <- epochNotMatch 3325 3326 select { 3327 case <-server1Stopped: 3328 case <-time.After(time.Second): 3329 c.Error("stream is not terminated by cdc kv client") 3330 } 3331 cancel() 3332 }