github.com/pingcap/tiflow@v0.0.0-20240520035814-5bf52d54e205/dm/syncer/binlogstream/streamer_controller_test.go (about) 1 // Copyright 2021 PingCAP, Inc. 2 // 3 // Licensed under the Apache License, Version 2.0 (the "License"); 4 // you may not use this file except in compliance with the License. 5 // You may obtain a copy of the License at 6 // 7 // http://www.apache.org/licenses/LICENSE-2.0 8 // 9 // Unless required by applicable law or agreed to in writing, software 10 // distributed under the License is distributed on an "AS IS" BASIS, 11 // See the License for the specific language governing permissions and 12 // limitations under the License. 13 14 package binlogstream 15 16 import ( 17 "context" 18 "fmt" 19 "testing" 20 "time" 21 22 "github.com/go-mysql-org/go-mysql/mysql" 23 "github.com/go-mysql-org/go-mysql/replication" 24 "github.com/google/uuid" 25 "github.com/pingcap/errors" 26 "github.com/pingcap/tiflow/dm/pb" 27 "github.com/pingcap/tiflow/dm/pkg/binlog" 28 "github.com/pingcap/tiflow/dm/pkg/binlog/event" 29 "github.com/pingcap/tiflow/dm/pkg/binlog/reader" 30 tcontext "github.com/pingcap/tiflow/dm/pkg/context" 31 "github.com/pingcap/tiflow/dm/pkg/gtid" 32 "github.com/pingcap/tiflow/dm/pkg/log" 33 "github.com/pingcap/tiflow/dm/relay" 34 "github.com/stretchr/testify/require" 35 ) 36 37 func TestToBinlogType(t *testing.T) { 38 testCases := []struct { 39 relay relay.Process 40 tp BinlogType 41 }{ 42 { 43 &relay.Relay{}, 44 LocalBinlog, 45 }, { 46 nil, 47 RemoteBinlog, 48 }, 49 } 50 51 for _, testCase := range testCases { 52 tp := RelayToBinlogType(testCase.relay) 53 require.Equal(t, testCase.tp, tp) 54 } 55 } 56 57 func TestCanErrorRetry(t *testing.T) { 58 relay2 := &relay.Relay{} 59 controller := NewStreamerController( 60 replication.BinlogSyncerConfig{}, 61 true, 62 nil, 63 "", 64 nil, 65 relay2, 66 log.L(), 67 ) 68 69 mockErr := errors.New("test") 70 71 // local binlog puller can always retry 72 for i := 0; i < 5; i++ { 73 require.True(t, controller.CanRetry(mockErr)) 74 } 75 76 origCfg := minErrorRetryInterval 77 minErrorRetryInterval = 100 * time.Millisecond 78 defer func() { 79 minErrorRetryInterval = origCfg 80 }() 81 82 // test with remote binlog 83 controller = NewStreamerController( 84 replication.BinlogSyncerConfig{}, 85 true, 86 nil, 87 "", 88 nil, 89 nil, 90 log.L(), 91 ) 92 93 require.True(t, controller.CanRetry(mockErr)) 94 require.False(t, controller.CanRetry(mockErr)) 95 time.Sleep(100 * time.Millisecond) 96 require.True(t, controller.CanRetry(mockErr)) 97 } 98 99 type mockStream struct { 100 i int 101 events []*replication.BinlogEvent 102 } 103 104 func (m *mockStream) GetEvent(ctx context.Context) (*replication.BinlogEvent, error) { 105 if m.i < len(m.events) { 106 e := m.events[m.i] 107 m.i++ 108 return e, nil 109 } 110 <-ctx.Done() 111 return nil, ctx.Err() 112 } 113 114 type mockStreamProducer struct { 115 stream *mockStream 116 } 117 118 func (m *mockStreamProducer) GenerateStreamFrom(location binlog.Location) (reader.Streamer, error) { 119 return m.stream, nil 120 } 121 122 type expectedInfo struct { 123 pos uint32 124 suffix int 125 data []byte 126 op pb.ErrorOp 127 } 128 129 func TestGetEventWithInject(t *testing.T) { 130 upstream := &mockStream{ 131 events: []*replication.BinlogEvent{ 132 { 133 Header: &replication.EventHeader{LogPos: 1010, EventSize: 10}, 134 }, 135 { 136 Header: &replication.EventHeader{LogPos: 1020, EventSize: 10}, 137 Event: &replication.QueryEvent{Query: []byte("a DDL")}, 138 RawData: []byte("should inject before me at 1010"), 139 }, 140 }, 141 } 142 producer := &mockStreamProducer{upstream} 143 144 controller := NewStreamerController4Test(producer, upstream) 145 146 injectReq := &pb.HandleWorkerErrorRequest{ 147 Op: pb.ErrorOp_Inject, 148 BinlogPos: "(bin.000001, 1010)", 149 } 150 injectEvents := []*replication.BinlogEvent{ 151 { 152 Header: &replication.EventHeader{}, 153 Event: &replication.QueryEvent{Query: []byte("a DDL")}, 154 RawData: []byte("inject 1"), 155 }, 156 { 157 Header: &replication.EventHeader{}, 158 Event: &replication.QueryEvent{Query: []byte("a DDL")}, 159 RawData: []byte("inject 2"), 160 }, 161 } 162 err := controller.Set(injectReq, injectEvents) 163 require.NoError(t, err) 164 loc := binlog.Location{Position: mysql.Position{ 165 Name: "bin.000001", 166 Pos: 1000, 167 }} 168 controller.streamModifier.reset(loc) 169 controller.upstream.locationRecorder.reset(loc) 170 171 expecteds := []expectedInfo{ 172 {1010, 0, nil, pb.ErrorOp_InvalidErrorOp}, 173 {1010, 1, []byte("inject 1"), pb.ErrorOp_Inject}, 174 {1010, 2, []byte("inject 2"), pb.ErrorOp_Inject}, 175 {1020, 0, []byte("should inject before me at 1010"), pb.ErrorOp_InvalidErrorOp}, 176 } 177 178 checkGetEvent(t, controller, expecteds) 179 } 180 181 func TestGetEventWithReplace(t *testing.T) { 182 upstream := &mockStream{ 183 events: []*replication.BinlogEvent{ 184 { 185 Header: &replication.EventHeader{LogPos: 1010, EventSize: 10}, 186 }, 187 { 188 Header: &replication.EventHeader{LogPos: 1020, EventSize: 10}, 189 RawData: []byte("should replace me at 1010"), 190 }, 191 }, 192 } 193 producer := &mockStreamProducer{upstream} 194 195 controller := NewStreamerController4Test(producer, upstream) 196 197 replaceReq := &pb.HandleWorkerErrorRequest{ 198 Op: pb.ErrorOp_Replace, 199 BinlogPos: "(bin.000001, 1010)", 200 } 201 replaceEvents := []*replication.BinlogEvent{ 202 { 203 Header: &replication.EventHeader{}, 204 Event: &replication.QueryEvent{Query: []byte("a DDL")}, 205 RawData: []byte("replace 1"), 206 }, 207 { 208 Header: &replication.EventHeader{}, 209 Event: &replication.QueryEvent{Query: []byte("a DDL")}, 210 RawData: []byte("replace 2"), 211 }, 212 } 213 err := controller.Set(replaceReq, replaceEvents) 214 require.NoError(t, err) 215 loc := binlog.Location{Position: mysql.Position{ 216 Name: "bin.000001", 217 Pos: 1000, 218 }} 219 controller.streamModifier.reset(loc) 220 controller.upstream.locationRecorder.reset(loc) 221 222 expecteds := []expectedInfo{ 223 {1010, 0, nil, pb.ErrorOp_InvalidErrorOp}, 224 {1010, 1, []byte("replace 1"), pb.ErrorOp_Replace}, 225 {1020, 0, []byte("replace 2"), pb.ErrorOp_Replace}, 226 } 227 228 checkGetEvent(t, controller, expecteds) 229 } 230 231 func TestGetEventWithSkip(t *testing.T) { 232 upstream := &mockStream{ 233 events: []*replication.BinlogEvent{ 234 { 235 Header: &replication.EventHeader{LogPos: 1010, EventSize: 10}, 236 }, 237 { 238 Header: &replication.EventHeader{LogPos: 1020, EventSize: 10}, 239 RawData: []byte("should skip me at 1010"), 240 }, 241 }, 242 } 243 producer := &mockStreamProducer{upstream} 244 245 controller := NewStreamerController4Test(producer, upstream) 246 247 replaceReq := &pb.HandleWorkerErrorRequest{ 248 Op: pb.ErrorOp_Skip, 249 BinlogPos: "(bin.000001, 1010)", 250 } 251 err := controller.Set(replaceReq, nil) 252 require.NoError(t, err) 253 loc := binlog.Location{Position: mysql.Position{ 254 Name: "bin.000001", 255 Pos: 1000, 256 }} 257 controller.streamModifier.reset(loc) 258 controller.upstream.locationRecorder.reset(loc) 259 260 expecteds := []expectedInfo{ 261 {1010, 0, nil, pb.ErrorOp_InvalidErrorOp}, 262 {1020, 0, []byte("should skip me at 1010"), pb.ErrorOp_Skip}, 263 } 264 265 checkGetEvent(t, controller, expecteds) 266 } 267 268 func checkGetEvent(t *testing.T, controller *StreamerController, expecteds []expectedInfo) { 269 t.Helper() 270 271 var lastLoc binlog.Location 272 273 ctx := tcontext.Background() 274 for i, expected := range expecteds { 275 event, op, err := controller.GetEvent(ctx) 276 require.NoError(t, err) 277 require.Equal(t, expected.pos, event.Header.LogPos) 278 require.Equal(t, expected.op, op) 279 280 curEndLoc := controller.GetCurEndLocation() 281 require.Equal(t, expected.pos, curEndLoc.Position.Pos) 282 require.Equal(t, expected.suffix, curEndLoc.Suffix) 283 284 if i > 0 { 285 curStartLoc := controller.GetCurStartLocation() 286 require.Equal(t, lastLoc.Position.Pos, curStartLoc.Position.Pos) 287 require.Equal(t, lastLoc.Suffix, curStartLoc.Suffix) 288 } 289 290 lastLoc = curEndLoc 291 } 292 ctx, cancel := ctx.WithTimeout(10 * time.Millisecond) 293 defer cancel() 294 // nolint:dogsled 295 _, _, err := controller.GetEvent(ctx) 296 require.ErrorIs(t, err, context.DeadlineExceeded) 297 } 298 299 func (s *testLocationSuite) TestLocationsWithGTID() { 300 events := s.generateDDLEvents() 301 302 // we have 5 events, only last 2 of them 303 s.Require().Len(events, 5) 304 startLoc := binlog.Location{ 305 Position: mysql.Position{ 306 Name: s.binlogFile, 307 Pos: events[2].Header.LogPos, 308 }, 309 } 310 err := startLoc.SetGTID(s.prevGSet) 311 s.Require().NoError(err) 312 313 events = events[3:5] 314 { 315 s.Require().Equal(replication.GTID_EVENT, events[0].Header.EventType) 316 s.Require().Equal(uint32(259), events[0].Header.LogPos) 317 e := events[0].Event.(*replication.GTIDEvent) 318 gtid := fmt.Sprintf("%s:%d", uuid.Must(uuid.FromBytes(e.SID)), e.GNO) 319 s.Require().Equal("3ccc475b-2343-11e7-be21-6c0b84d59f30:15", gtid) 320 } 321 { 322 s.Require().Equal(replication.QUERY_EVENT, events[1].Header.EventType) 323 s.Require().Equal(uint32(322), events[1].Header.LogPos) 324 } 325 326 replaceReq := &pb.HandleWorkerErrorRequest{ 327 Op: pb.ErrorOp_Replace, 328 BinlogPos: "(mysql-bin.000001, 259)", 329 } 330 replaceEvents := []*replication.BinlogEvent{ 331 { 332 Header: &replication.EventHeader{}, 333 Event: &replication.QueryEvent{Query: []byte("a DDL")}, 334 RawData: []byte("first replace event"), 335 }, 336 { 337 Header: &replication.EventHeader{}, 338 Event: &replication.QueryEvent{Query: []byte("a DDL")}, 339 RawData: []byte("seconds replace event"), 340 }, 341 } 342 343 // expected 3 events, check their start/end locations 344 expectedLocations := make([]binlog.Location, 4) 345 expectedLocations[0] = startLoc 346 expectedLocations[1] = startLoc 347 expectedLocations[1].Position.Pos = events[0].Header.LogPos 348 expectedLocations[2] = expectedLocations[1] 349 expectedLocations[2].Suffix = 1 350 expectedLocations[3] = startLoc 351 expectedLocations[3].Position.Pos = events[1].Header.LogPos 352 err = expectedLocations[3].SetGTID(s.currGSet) 353 s.Require().NoError(err) 354 355 upstream := &mockStream{ 356 events: events, 357 } 358 producer := &mockStreamProducer{upstream} 359 360 controller := NewStreamerController4Test(producer, upstream) 361 362 err = controller.Set(replaceReq, replaceEvents) 363 s.Require().NoError(err) 364 365 controller.streamModifier.reset(startLoc) 366 controller.locations.reset(startLoc) 367 controller.upstream.locationRecorder.reset(startLoc) 368 ctx := tcontext.Background() 369 370 for i := 1; i < len(expectedLocations); i++ { 371 // nolint:dogsled 372 _, _, err = controller.GetEvent(ctx) 373 s.Require().NoError(err) 374 s.Require().Equal(expectedLocations[i-1].String(), controller.GetCurStartLocation().String()) 375 s.Require().Equal(expectedLocations[i].String(), controller.GetCurEndLocation().String()) 376 } 377 378 ctx, cancel := ctx.WithTimeout(10 * time.Millisecond) 379 defer cancel() 380 // nolint:dogsled 381 _, _, err = controller.GetEvent(ctx) 382 s.Require().ErrorIs(err, context.DeadlineExceeded) 383 } 384 385 func TestResetToMiddleOfReplace(t *testing.T) { 386 // this test case is to check we start at {empty binlog position, GTID: "3ccc475b-2343-11e7-be21-6c0b84d59f30:1-3", Suffix: 1} 387 // while that location we have replaced events 388 389 binlogName := "bin.000001" 390 fakeRotateHeader := &replication.EventHeader{EventType: replication.ROTATE_EVENT, Flags: replication.LOG_EVENT_ARTIFICIAL_F} 391 fakeRotate, err := event.GenRotateEvent(fakeRotateHeader, 0, []byte(binlogName), 0) 392 require.NoError(t, err) 393 394 uuid := "3ccc475b-2343-11e7-be21-6c0b84d59f30" 395 gtidHeader := &replication.EventHeader{ 396 EventType: replication.GTID_EVENT, LogPos: 1111, EventSize: 77, 397 } 398 gtidEvent, err := event.GenGTIDEvent(gtidHeader, 1034, 0, uuid, 4, 0, 0) 399 require.NoError(t, err) 400 require.Equal(t, uint32(1099), gtidEvent.Header.LogPos) 401 upstream := &mockStream{ 402 events: []*replication.BinlogEvent{ 403 fakeRotate, 404 { 405 Header: &replication.EventHeader{EventType: replication.FORMAT_DESCRIPTION_EVENT, LogPos: 125, EventSize: 121}, 406 }, 407 { 408 Header: &replication.EventHeader{EventType: replication.PREVIOUS_GTIDS_EVENT, LogPos: 156, EventSize: 31}, 409 }, 410 { 411 Header: &replication.EventHeader{EventType: replication.HEARTBEAT_EVENT, LogPos: 1034, EventSize: 39}, 412 }, 413 gtidEvent, 414 { 415 Header: &replication.EventHeader{EventType: replication.QUERY_EVENT, LogPos: 1199, EventSize: 100}, 416 Event: &replication.QueryEvent{Query: []byte("a DDL")}, 417 RawData: []byte("should replace me at 1099"), 418 }, 419 }, 420 } 421 producer := &mockStreamProducer{upstream} 422 423 controller := NewStreamerController4Test(producer, upstream) 424 425 replaceReq := &pb.HandleWorkerErrorRequest{ 426 Op: pb.ErrorOp_Replace, 427 BinlogPos: "(bin.000001, 1099)", 428 } 429 replaceEvents := []*replication.BinlogEvent{ 430 { 431 Header: &replication.EventHeader{EventType: replication.QUERY_EVENT}, 432 Event: &replication.QueryEvent{Query: []byte("a DDL")}, 433 RawData: []byte("replace 1"), 434 }, 435 { 436 Header: &replication.EventHeader{EventType: replication.QUERY_EVENT}, 437 Event: &replication.QueryEvent{Query: []byte("a DDL")}, 438 RawData: []byte("replace 2"), 439 }, 440 } 441 err = controller.Set(replaceReq, replaceEvents) 442 require.NoError(t, err) 443 444 startSyncLoc := binlog.MustZeroLocation(mysql.MySQLFlavor) 445 gset, err := gtid.ParserGTID(mysql.MySQLFlavor, "3ccc475b-2343-11e7-be21-6c0b84d59f30:1-3") 446 require.NoError(t, err) 447 err = startSyncLoc.SetGTID(gset) 448 require.NoError(t, err) 449 startSyncLoc.Suffix = 1 450 451 replaceLoc := startSyncLoc 452 replaceLoc.Position.Name = binlogName 453 replaceLoc.Position.Pos = 1099 454 455 controller.streamModifier.reset(replaceLoc) 456 controller.locations.reset(startSyncLoc) 457 controller.upstream, err = newLocationStream(producer, startSyncLoc) 458 require.NoError(t, err) 459 460 expecteds := []struct { 461 tp replication.EventType 462 startLocStr string 463 endLocStr string 464 }{ 465 { 466 tp: replication.ROTATE_EVENT, 467 startLocStr: "position: (, 4), gtid-set: 3ccc475b-2343-11e7-be21-6c0b84d59f30:1-3, suffix: 1", 468 endLocStr: "position: (bin.000001, 4), gtid-set: 3ccc475b-2343-11e7-be21-6c0b84d59f30:1-3, suffix: 1", 469 }, 470 { 471 tp: replication.FORMAT_DESCRIPTION_EVENT, 472 startLocStr: "position: (bin.000001, 4), gtid-set: 3ccc475b-2343-11e7-be21-6c0b84d59f30:1-3, suffix: 1", 473 endLocStr: "position: (bin.000001, 4), gtid-set: 3ccc475b-2343-11e7-be21-6c0b84d59f30:1-3, suffix: 1", 474 }, 475 { 476 tp: replication.PREVIOUS_GTIDS_EVENT, 477 startLocStr: "position: (bin.000001, 4), gtid-set: 3ccc475b-2343-11e7-be21-6c0b84d59f30:1-3, suffix: 1", 478 endLocStr: "position: (bin.000001, 4), gtid-set: 3ccc475b-2343-11e7-be21-6c0b84d59f30:1-3, suffix: 1", 479 }, 480 { 481 tp: replication.HEARTBEAT_EVENT, 482 startLocStr: "position: (bin.000001, 4), gtid-set: 3ccc475b-2343-11e7-be21-6c0b84d59f30:1-3, suffix: 1", 483 endLocStr: "position: (bin.000001, 4), gtid-set: 3ccc475b-2343-11e7-be21-6c0b84d59f30:1-3, suffix: 1", 484 }, 485 { 486 tp: replication.GTID_EVENT, 487 startLocStr: "position: (bin.000001, 4), gtid-set: 3ccc475b-2343-11e7-be21-6c0b84d59f30:1-3, suffix: 1", 488 endLocStr: "position: (bin.000001, 1099), gtid-set: 3ccc475b-2343-11e7-be21-6c0b84d59f30:1-3, suffix: 1", 489 }, 490 { 491 tp: replication.QUERY_EVENT, 492 startLocStr: "position: (bin.000001, 1099), gtid-set: 3ccc475b-2343-11e7-be21-6c0b84d59f30:1-3, suffix: 1", 493 endLocStr: "position: (bin.000001, 1199), gtid-set: 3ccc475b-2343-11e7-be21-6c0b84d59f30:1-4", 494 }, 495 } 496 497 ctx := tcontext.Background() 498 for i, expected := range expecteds { 499 event, _, err2 := controller.GetEvent(ctx) 500 require.NoError(t, err2) 501 require.Equal(t, expected.tp, event.Header.EventType) 502 require.Equal(t, expected.startLocStr, controller.GetCurStartLocation().String()) 503 require.Equal(t, expected.endLocStr, controller.GetCurEndLocation().String()) 504 505 if i == len(expecteds) { 506 require.Equal(t, []byte("replace 2"), event.RawData) 507 } 508 } 509 ctx, cancel := ctx.WithTimeout(10 * time.Millisecond) 510 defer cancel() 511 // nolint:dogsled 512 _, _, err = controller.GetEvent(ctx) 513 require.ErrorIs(t, err, context.DeadlineExceeded) 514 } 515 516 func TestResetToLastOfInject(t *testing.T) { 517 // this test case is to check we start at {empty binlog position, GTID: "3ccc475b-2343-11e7-be21-6c0b84d59f30:1-3", Suffix: 1} 518 // while that location we have injected events 519 520 binlogName := "bin.000001" 521 fakeRotateHeader := &replication.EventHeader{EventType: replication.ROTATE_EVENT, Flags: replication.LOG_EVENT_ARTIFICIAL_F} 522 fakeRotate, err := event.GenRotateEvent(fakeRotateHeader, 0, []byte(binlogName), 0) 523 require.NoError(t, err) 524 525 uuid := "3ccc475b-2343-11e7-be21-6c0b84d59f30" 526 gtidHeader := &replication.EventHeader{ 527 EventType: replication.GTID_EVENT, LogPos: 1111, EventSize: 77, 528 } 529 gtidEvent, err := event.GenGTIDEvent(gtidHeader, 1034, 0, uuid, 4, 0, 0) 530 require.NoError(t, err) 531 require.Equal(t, uint32(1099), gtidEvent.Header.LogPos) 532 upstream := &mockStream{ 533 events: []*replication.BinlogEvent{ 534 fakeRotate, 535 { 536 Header: &replication.EventHeader{EventType: replication.FORMAT_DESCRIPTION_EVENT, LogPos: 125, EventSize: 121}, 537 }, 538 { 539 Header: &replication.EventHeader{EventType: replication.PREVIOUS_GTIDS_EVENT, LogPos: 156, EventSize: 31}, 540 }, 541 { 542 Header: &replication.EventHeader{EventType: replication.HEARTBEAT_EVENT, LogPos: 1034, EventSize: 39}, 543 }, 544 gtidEvent, 545 { 546 Header: &replication.EventHeader{EventType: replication.QUERY_EVENT, LogPos: 1199, EventSize: 100}, 547 Event: &replication.QueryEvent{Query: []byte("a DDL")}, 548 RawData: []byte("should inject before me at 1099"), 549 }, 550 }, 551 } 552 producer := &mockStreamProducer{upstream} 553 554 controller := NewStreamerController4Test(producer, upstream) 555 556 replaceReq := &pb.HandleWorkerErrorRequest{ 557 Op: pb.ErrorOp_Inject, 558 BinlogPos: "(bin.000001, 1099)", 559 } 560 replaceEvents := []*replication.BinlogEvent{ 561 { 562 Header: &replication.EventHeader{EventType: replication.QUERY_EVENT}, 563 Event: &replication.QueryEvent{Query: []byte("a DDL")}, 564 RawData: []byte("replace 1"), 565 }, 566 } 567 err = controller.Set(replaceReq, replaceEvents) 568 require.NoError(t, err) 569 570 startSyncLoc := binlog.MustZeroLocation(mysql.MySQLFlavor) 571 gset, err := gtid.ParserGTID(mysql.MySQLFlavor, "3ccc475b-2343-11e7-be21-6c0b84d59f30:1-3") 572 require.NoError(t, err) 573 err = startSyncLoc.SetGTID(gset) 574 require.NoError(t, err) 575 startSyncLoc.Suffix = 1 576 577 replaceLoc := startSyncLoc 578 replaceLoc.Position.Name = binlogName 579 replaceLoc.Position.Pos = 1099 580 581 controller.streamModifier.reset(replaceLoc) 582 controller.locations.reset(startSyncLoc) 583 controller.upstream, err = newLocationStream(producer, startSyncLoc) 584 require.NoError(t, err) 585 586 expecteds := []struct { 587 tp replication.EventType 588 startLocStr string 589 endLocStr string 590 }{ 591 { 592 tp: replication.ROTATE_EVENT, 593 startLocStr: "position: (, 4), gtid-set: 3ccc475b-2343-11e7-be21-6c0b84d59f30:1-3, suffix: 1", 594 endLocStr: "position: (bin.000001, 4), gtid-set: 3ccc475b-2343-11e7-be21-6c0b84d59f30:1-3, suffix: 1", 595 }, 596 { 597 tp: replication.FORMAT_DESCRIPTION_EVENT, 598 startLocStr: "position: (bin.000001, 4), gtid-set: 3ccc475b-2343-11e7-be21-6c0b84d59f30:1-3, suffix: 1", 599 endLocStr: "position: (bin.000001, 4), gtid-set: 3ccc475b-2343-11e7-be21-6c0b84d59f30:1-3, suffix: 1", 600 }, 601 { 602 tp: replication.PREVIOUS_GTIDS_EVENT, 603 startLocStr: "position: (bin.000001, 4), gtid-set: 3ccc475b-2343-11e7-be21-6c0b84d59f30:1-3, suffix: 1", 604 endLocStr: "position: (bin.000001, 4), gtid-set: 3ccc475b-2343-11e7-be21-6c0b84d59f30:1-3, suffix: 1", 605 }, 606 { 607 tp: replication.HEARTBEAT_EVENT, 608 startLocStr: "position: (bin.000001, 4), gtid-set: 3ccc475b-2343-11e7-be21-6c0b84d59f30:1-3, suffix: 1", 609 endLocStr: "position: (bin.000001, 4), gtid-set: 3ccc475b-2343-11e7-be21-6c0b84d59f30:1-3, suffix: 1", 610 }, 611 { 612 tp: replication.GTID_EVENT, 613 startLocStr: "position: (bin.000001, 4), gtid-set: 3ccc475b-2343-11e7-be21-6c0b84d59f30:1-3, suffix: 1", 614 endLocStr: "position: (bin.000001, 1099), gtid-set: 3ccc475b-2343-11e7-be21-6c0b84d59f30:1-3, suffix: 1", 615 }, 616 { 617 tp: replication.QUERY_EVENT, 618 startLocStr: "position: (bin.000001, 1099), gtid-set: 3ccc475b-2343-11e7-be21-6c0b84d59f30:1-3, suffix: 1", 619 endLocStr: "position: (bin.000001, 1199), gtid-set: 3ccc475b-2343-11e7-be21-6c0b84d59f30:1-4", 620 }, 621 } 622 623 ctx := tcontext.Background() 624 for i, expected := range expecteds { 625 event, _, err2 := controller.GetEvent(ctx) 626 require.NoError(t, err2) 627 require.Equal(t, expected.tp, event.Header.EventType) 628 require.Equal(t, expected.startLocStr, controller.GetCurStartLocation().String()) 629 require.Equal(t, expected.endLocStr, controller.GetCurEndLocation().String()) 630 631 if i == len(expecteds) { 632 require.Equal(t, []byte("should inject before me at 1099"), event.RawData) 633 } 634 } 635 ctx, cancel := ctx.WithTimeout(10 * time.Millisecond) 636 defer cancel() 637 // nolint:dogsled 638 _, _, err = controller.GetEvent(ctx) 639 require.ErrorIs(t, err, context.DeadlineExceeded) 640 }