github.com/pingcap/ticdc@v0.0.0-20220526033649-485a10ef2652/cdc/puller/sorter/merger_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 sorter 15 16 import ( 17 "context" 18 "sync/atomic" 19 "time" 20 21 "github.com/pingcap/check" 22 "github.com/pingcap/failpoint" 23 "github.com/pingcap/log" 24 "github.com/pingcap/ticdc/cdc/model" 25 "github.com/pingcap/ticdc/pkg/util/testleak" 26 "go.uber.org/zap" 27 "go.uber.org/zap/zapcore" 28 "golang.org/x/sync/errgroup" 29 ) 30 31 type mockFlushTaskBuilder struct { 32 task *flushTask 33 writer backEndWriter 34 totalCount int 35 } 36 37 var backEndCounterForTest int64 38 39 func newMockFlushTaskBuilder() *mockFlushTaskBuilder { 40 backEnd := newMemoryBackEnd() 41 atomic.AddInt64(&backEndCounterForTest, 1) 42 43 task := &flushTask{ 44 backend: backEnd, 45 tsLowerBound: 0, 46 maxResolvedTs: 0, 47 finished: make(chan error, 2), 48 } 49 50 task.dealloc = func() error { 51 if task.backend != nil { 52 atomic.AddInt64(&backEndCounterForTest, -1) 53 task.backend = nil 54 return backEnd.free() 55 } 56 return nil 57 } 58 59 writer, _ := backEnd.writer() 60 61 return &mockFlushTaskBuilder{ 62 task: task, 63 writer: writer, 64 } 65 } 66 67 func (b *mockFlushTaskBuilder) generateRowChanges(tsRangeBegin, tsRangeEnd uint64, count int) *mockFlushTaskBuilder { 68 if b.task.tsLowerBound == 0 { 69 b.task.tsLowerBound = tsRangeBegin 70 } 71 density := float64(tsRangeEnd-tsRangeBegin) / float64(count) 72 for fTs := float64(tsRangeBegin); fTs < float64(tsRangeEnd); fTs += density { 73 ts := uint64(fTs) 74 kvEntry := generateMockRawKV(ts) 75 _ = b.writer.writeNext(model.NewPolymorphicEvent(kvEntry)) 76 b.totalCount++ 77 } 78 return b 79 } 80 81 func (b *mockFlushTaskBuilder) addResolved(ts uint64) *mockFlushTaskBuilder { 82 _ = b.writer.writeNext(model.NewResolvedPolymorphicEvent(0, ts)) 83 b.task.maxResolvedTs = ts 84 return b 85 } 86 87 func (b *mockFlushTaskBuilder) build() *flushTask { 88 _ = b.writer.flushAndClose() 89 return b.task 90 } 91 92 // TestMergerSingleHeap simulates a situation where there is only one data stream 93 // It tests the most basic scenario. 94 func (s *sorterSuite) TestMergerSingleHeap(c *check.C) { 95 defer testleak.AfterTest(c)() 96 err := failpoint.Enable("github.com/pingcap/ticdc/cdc/puller/sorter/sorterDebug", "return(true)") 97 if err != nil { 98 log.Panic("Could not enable failpoint", zap.Error(err)) 99 } 100 101 ctx, cancel := context.WithTimeout(context.TODO(), time.Second*10) 102 defer cancel() 103 wg, ctx := errgroup.WithContext(ctx) 104 inChan := make(chan *flushTask, 1024) 105 outChan := make(chan *model.PolymorphicEvent, 1024) 106 107 wg.Go(func() error { 108 return runMerger(ctx, 1, inChan, outChan, func() {}) 109 }) 110 111 totalCount := 0 112 builder := newMockFlushTaskBuilder() 113 task1 := builder.generateRowChanges(1000, 100000, 2048).addResolved(100001).build() 114 totalCount += builder.totalCount 115 builder = newMockFlushTaskBuilder() 116 task2 := builder.generateRowChanges(100002, 200000, 2048).addResolved(200001).build() 117 totalCount += builder.totalCount 118 builder = newMockFlushTaskBuilder() 119 task3 := builder.generateRowChanges(200002, 300000, 2048).addResolved(300001).build() 120 totalCount += builder.totalCount 121 122 wg.Go(func() error { 123 inChan <- task1 124 close(task1.finished) 125 inChan <- task2 126 close(task2.finished) 127 inChan <- task3 128 close(task3.finished) 129 130 return nil 131 }) 132 133 wg.Go(func() error { 134 count := 0 135 lastTs := uint64(0) 136 lastResolved := uint64(0) 137 for { 138 select { 139 case <-ctx.Done(): 140 return ctx.Err() 141 case event := <-outChan: 142 switch event.RawKV.OpType { 143 case model.OpTypePut: 144 count++ 145 c.Assert(event.CRTs, check.GreaterEqual, lastTs) 146 c.Assert(event.CRTs, check.GreaterEqual, lastResolved) 147 lastTs = event.CRTs 148 case model.OpTypeResolved: 149 c.Assert(event.CRTs, check.GreaterEqual, lastResolved) 150 lastResolved = event.CRTs 151 } 152 if lastResolved >= 300001 { 153 c.Assert(count, check.Equals, totalCount) 154 cancel() 155 return nil 156 } 157 } 158 } 159 }) 160 c.Assert(wg.Wait(), check.ErrorMatches, ".*context canceled.*") 161 c.Assert(atomic.LoadInt64(&backEndCounterForTest), check.Equals, int64(0)) 162 } 163 164 // TestMergerSingleHeapRetire simulates a situation where the resolved event is not the last event in a flushTask 165 func (s *sorterSuite) TestMergerSingleHeapRetire(c *check.C) { 166 defer testleak.AfterTest(c)() 167 err := failpoint.Enable("github.com/pingcap/ticdc/cdc/puller/sorter/sorterDebug", "return(true)") 168 if err != nil { 169 log.Panic("Could not enable failpoint", zap.Error(err)) 170 } 171 172 ctx, cancel := context.WithTimeout(context.TODO(), time.Second*10) 173 defer cancel() 174 wg, ctx := errgroup.WithContext(ctx) 175 inChan := make(chan *flushTask, 1024) 176 outChan := make(chan *model.PolymorphicEvent, 1024) 177 178 wg.Go(func() error { 179 return runMerger(ctx, 1, inChan, outChan, func() {}) 180 }) 181 182 totalCount := 0 183 builder := newMockFlushTaskBuilder() 184 task1 := builder.generateRowChanges(1000, 100000, 2048).addResolved(100001).build() 185 totalCount += builder.totalCount 186 builder = newMockFlushTaskBuilder() 187 task2 := builder.generateRowChanges(100002, 200000, 2048).build() 188 totalCount += builder.totalCount 189 builder = newMockFlushTaskBuilder() 190 task3 := builder.generateRowChanges(200002, 300000, 2048).addResolved(300001).build() 191 totalCount += builder.totalCount 192 193 wg.Go(func() error { 194 inChan <- task1 195 close(task1.finished) 196 inChan <- task2 197 close(task2.finished) 198 inChan <- task3 199 close(task3.finished) 200 201 return nil 202 }) 203 204 wg.Go(func() error { 205 count := 0 206 lastTs := uint64(0) 207 lastResolved := uint64(0) 208 for { 209 select { 210 case <-ctx.Done(): 211 return ctx.Err() 212 case event := <-outChan: 213 switch event.RawKV.OpType { 214 case model.OpTypePut: 215 count++ 216 c.Assert(event.CRTs, check.GreaterEqual, lastResolved) 217 c.Assert(event.CRTs, check.GreaterEqual, lastTs) 218 lastTs = event.CRTs 219 case model.OpTypeResolved: 220 c.Assert(event.CRTs, check.GreaterEqual, lastResolved) 221 lastResolved = event.CRTs 222 } 223 if lastResolved >= 300001 { 224 c.Assert(count, check.Equals, totalCount) 225 cancel() 226 return nil 227 } 228 } 229 } 230 }) 231 232 c.Assert(wg.Wait(), check.ErrorMatches, ".*context canceled.*") 233 c.Assert(atomic.LoadInt64(&backEndCounterForTest), check.Equals, int64(0)) 234 } 235 236 // TestMergerSortDelay simulates a situation where merging takes a long time. 237 // Expects intermediate resolved events to be generated, so that the sink would not get stuck in a real life situation. 238 func (s *sorterSuite) TestMergerSortDelay(c *check.C) { 239 defer testleak.AfterTest(c)() 240 err := failpoint.Enable("github.com/pingcap/ticdc/cdc/puller/sorter/sorterDebug", "return(true)") 241 c.Assert(err, check.IsNil) 242 243 // enable the failpoint to simulate delays 244 err = failpoint.Enable("github.com/pingcap/ticdc/cdc/puller/sorter/sorterMergeDelay", "sleep(5)") 245 c.Assert(err, check.IsNil) 246 defer func() { 247 _ = failpoint.Disable("github.com/pingcap/ticdc/cdc/puller/sorter/sorterMergeDelay") 248 }() 249 250 log.SetLevel(zapcore.DebugLevel) 251 defer log.SetLevel(zapcore.InfoLevel) 252 253 ctx, cancel := context.WithTimeout(context.TODO(), time.Second*10) 254 defer cancel() 255 wg, ctx := errgroup.WithContext(ctx) 256 inChan := make(chan *flushTask, 1024) 257 outChan := make(chan *model.PolymorphicEvent, 1024) 258 259 wg.Go(func() error { 260 return runMerger(ctx, 1, inChan, outChan, func() {}) 261 }) 262 263 totalCount := 0 264 builder := newMockFlushTaskBuilder() 265 task1 := builder.generateRowChanges(1000, 1000000, 1024).addResolved(1000001).build() 266 totalCount += builder.totalCount 267 268 wg.Go(func() error { 269 inChan <- task1 270 close(task1.finished) 271 return nil 272 }) 273 274 wg.Go(func() error { 275 var ( 276 count int 277 lastTs uint64 278 lastResolved uint64 279 lastResolvedTime time.Time 280 ) 281 for { 282 select { 283 case <-ctx.Done(): 284 return ctx.Err() 285 case event := <-outChan: 286 switch event.RawKV.OpType { 287 case model.OpTypePut: 288 count++ 289 c.Assert(event.CRTs, check.GreaterEqual, lastResolved) 290 c.Assert(event.CRTs, check.GreaterEqual, lastTs) 291 lastTs = event.CRTs 292 case model.OpTypeResolved: 293 c.Assert(event.CRTs, check.GreaterEqual, lastResolved) 294 if !lastResolvedTime.IsZero() { 295 c.Assert(time.Since(lastResolvedTime), check.LessEqual, 2*time.Second) 296 } 297 log.Debug("resolved event received", zap.Uint64("ts", event.CRTs)) 298 lastResolvedTime = time.Now() 299 lastResolved = event.CRTs 300 } 301 if lastResolved >= 1000001 { 302 c.Assert(count, check.Equals, totalCount) 303 cancel() 304 return nil 305 } 306 } 307 } 308 }) 309 310 c.Assert(wg.Wait(), check.ErrorMatches, ".*context canceled.*") 311 close(inChan) 312 mergerCleanUp(inChan) 313 c.Assert(atomic.LoadInt64(&backEndCounterForTest), check.Equals, int64(0)) 314 } 315 316 // TestMergerCancel simulates a situation where the merger is cancelled with pending data. 317 // Expects proper clean-up of the data. 318 func (s *sorterSuite) TestMergerCancel(c *check.C) { 319 defer testleak.AfterTest(c)() 320 err := failpoint.Enable("github.com/pingcap/ticdc/cdc/puller/sorter/sorterDebug", "return(true)") 321 c.Assert(err, check.IsNil) 322 323 // enable the failpoint to simulate delays 324 err = failpoint.Enable("github.com/pingcap/ticdc/cdc/puller/sorter/sorterMergeDelay", "sleep(10)") 325 c.Assert(err, check.IsNil) 326 defer func() { 327 _ = failpoint.Disable("github.com/pingcap/ticdc/cdc/puller/sorter/sorterMergeDelay") 328 }() 329 330 log.SetLevel(zapcore.DebugLevel) 331 defer log.SetLevel(zapcore.InfoLevel) 332 333 ctx, cancel := context.WithTimeout(context.TODO(), time.Second*10) 334 defer cancel() 335 wg, ctx := errgroup.WithContext(ctx) 336 inChan := make(chan *flushTask, 1024) 337 outChan := make(chan *model.PolymorphicEvent, 1024) 338 339 wg.Go(func() error { 340 return runMerger(ctx, 1, inChan, outChan, func() {}) 341 }) 342 343 builder := newMockFlushTaskBuilder() 344 task1 := builder.generateRowChanges(1000, 100000, 2048).addResolved(100001).build() 345 builder = newMockFlushTaskBuilder() 346 task2 := builder.generateRowChanges(100002, 200000, 2048).addResolved(200001).build() 347 builder = newMockFlushTaskBuilder() 348 task3 := builder.generateRowChanges(200002, 300000, 2048).addResolved(300001).build() 349 350 wg.Go(func() error { 351 inChan <- task1 352 close(task1.finished) 353 inChan <- task2 354 close(task2.finished) 355 inChan <- task3 356 close(task3.finished) 357 return nil 358 }) 359 360 wg.Go(func() error { 361 for { 362 select { 363 case <-ctx.Done(): 364 return ctx.Err() 365 case <-outChan: 366 // We just drain the data here. We don't care about it. 367 } 368 } 369 }) 370 371 time.Sleep(5 * time.Second) 372 cancel() 373 c.Assert(wg.Wait(), check.ErrorMatches, ".*context canceled.*") 374 close(inChan) 375 mergerCleanUp(inChan) 376 c.Assert(atomic.LoadInt64(&backEndCounterForTest), check.Equals, int64(0)) 377 } 378 379 // TestMergerCancel simulates a situation where the merger is cancelled with pending data. 380 // Expects proper clean-up of the data. 381 func (s *sorterSuite) TestMergerCancelWithUnfinishedFlushTasks(c *check.C) { 382 defer testleak.AfterTest(c)() 383 err := failpoint.Enable("github.com/pingcap/ticdc/cdc/puller/sorter/sorterDebug", "return(true)") 384 c.Assert(err, check.IsNil) 385 386 log.SetLevel(zapcore.DebugLevel) 387 defer log.SetLevel(zapcore.InfoLevel) 388 389 ctx, cancel := context.WithTimeout(context.TODO(), time.Second*10) 390 wg, ctx := errgroup.WithContext(ctx) 391 inChan := make(chan *flushTask, 1024) 392 outChan := make(chan *model.PolymorphicEvent, 1024) 393 394 wg.Go(func() error { 395 return runMerger(ctx, 1, inChan, outChan, func() {}) 396 }) 397 398 builder := newMockFlushTaskBuilder() 399 task1 := builder.generateRowChanges(1000, 100000, 2048).addResolved(100001).build() 400 builder = newMockFlushTaskBuilder() 401 task2 := builder.generateRowChanges(100002, 200000, 2048).addResolved(200001).build() 402 builder = newMockFlushTaskBuilder() 403 task3 := builder.generateRowChanges(200002, 300000, 2048).addResolved(300001).build() 404 405 wg.Go(func() error { 406 inChan <- task1 407 inChan <- task2 408 inChan <- task3 409 close(task2.finished) 410 close(task1.finished) 411 time.Sleep(1 * time.Second) 412 cancel() 413 return nil 414 }) 415 416 wg.Go(func() error { 417 for { 418 select { 419 case <-ctx.Done(): 420 return ctx.Err() 421 case <-outChan: 422 // We just drain the data here. We don't care about it. 423 } 424 } 425 }) 426 427 c.Assert(wg.Wait(), check.ErrorMatches, ".*context canceled.*") 428 close(inChan) 429 mergerCleanUp(inChan) 430 // Leaking one task is expected 431 c.Assert(atomic.LoadInt64(&backEndCounterForTest), check.Equals, int64(1)) 432 atomic.StoreInt64(&backEndCounterForTest, 0) 433 } 434 435 // TestMergerCancel simulates a situation where the input channel is abruptly closed. 436 // There is expected to be NO fatal error. 437 func (s *sorterSuite) TestMergerCloseChannel(c *check.C) { 438 defer testleak.AfterTest(c)() 439 err := failpoint.Enable("github.com/pingcap/ticdc/cdc/puller/sorter/sorterDebug", "return(true)") 440 c.Assert(err, check.IsNil) 441 442 log.SetLevel(zapcore.DebugLevel) 443 defer log.SetLevel(zapcore.InfoLevel) 444 445 ctx, cancel := context.WithTimeout(context.TODO(), time.Second*15) 446 defer cancel() 447 wg, ctx := errgroup.WithContext(ctx) 448 inChan := make(chan *flushTask, 1024) 449 outChan := make(chan *model.PolymorphicEvent, 1024) 450 451 builder := newMockFlushTaskBuilder() 452 task1 := builder.generateRowChanges(1000, 100000, 2048).addResolved(100001).build() 453 454 inChan <- task1 455 close(task1.finished) 456 457 wg.Go(func() error { 458 return runMerger(ctx, 1, inChan, outChan, func() {}) 459 }) 460 461 wg.Go(func() error { 462 for { 463 select { 464 case <-ctx.Done(): 465 return ctx.Err() 466 case <-outChan: 467 // We just drain the data here. We don't care about it. 468 } 469 } 470 }) 471 472 time.Sleep(5 * time.Second) 473 close(inChan) 474 time.Sleep(5 * time.Second) 475 cancel() 476 c.Assert(wg.Wait(), check.ErrorMatches, ".*context canceled.*") 477 mergerCleanUp(inChan) 478 c.Assert(atomic.LoadInt64(&backEndCounterForTest), check.Equals, int64(0)) 479 } 480 481 // TestMergerOutputBlocked simulates a situation where the output channel is blocked for 482 // a significant period of time. 483 func (s *sorterSuite) TestMergerOutputBlocked(c *check.C) { 484 defer testleak.AfterTest(c)() 485 err := failpoint.Enable("github.com/pingcap/ticdc/cdc/puller/sorter/sorterDebug", "return(true)") 486 c.Assert(err, check.IsNil) 487 defer failpoint.Disable("github.com/pingcap/ticdc/cdc/puller/sorter/sorterDebug") //nolint:errcheck 488 489 ctx, cancel := context.WithTimeout(context.TODO(), time.Second*25) 490 defer cancel() 491 wg, ctx := errgroup.WithContext(ctx) 492 // use unbuffered channel to make sure that the input has been processed 493 inChan := make(chan *flushTask) 494 // make a small channel to test blocking 495 outChan := make(chan *model.PolymorphicEvent, 1) 496 497 wg.Go(func() error { 498 return runMerger(ctx, 1, inChan, outChan, func() {}) 499 }) 500 501 totalCount := 0 502 builder := newMockFlushTaskBuilder() 503 task1 := builder.generateRowChanges(1000, 100000, 2048).addResolved(100001).build() 504 totalCount += builder.totalCount 505 builder = newMockFlushTaskBuilder() 506 task2 := builder.generateRowChanges(100002, 200000, 2048).addResolved(200001).build() 507 totalCount += builder.totalCount 508 builder = newMockFlushTaskBuilder() 509 task3 := builder.generateRowChanges(200002, 300000, 2048).addResolved(300001).build() 510 totalCount += builder.totalCount 511 512 wg.Go(func() error { 513 inChan <- task1 514 close(task1.finished) 515 inChan <- task2 516 close(task2.finished) 517 inChan <- task3 518 close(task3.finished) 519 520 return nil 521 }) 522 523 wg.Go(func() error { 524 time.Sleep(10 * time.Second) 525 count := 0 526 lastTs := uint64(0) 527 lastResolved := uint64(0) 528 for { 529 select { 530 case <-ctx.Done(): 531 return ctx.Err() 532 case event := <-outChan: 533 switch event.RawKV.OpType { 534 case model.OpTypePut: 535 count++ 536 c.Assert(event.CRTs, check.GreaterEqual, lastTs) 537 c.Assert(event.CRTs, check.GreaterEqual, lastResolved) 538 lastTs = event.CRTs 539 case model.OpTypeResolved: 540 c.Assert(event.CRTs, check.GreaterEqual, lastResolved) 541 lastResolved = event.CRTs 542 } 543 if lastResolved >= 300001 { 544 c.Assert(count, check.Equals, totalCount) 545 cancel() 546 return nil 547 } 548 } 549 } 550 }) 551 c.Assert(wg.Wait(), check.ErrorMatches, ".*context canceled.*") 552 c.Assert(atomic.LoadInt64(&backEndCounterForTest), check.Equals, int64(0)) 553 }