github.com/pingcap/ticdc@v0.0.0-20220526033649-485a10ef2652/cdc/sink/common/flow_control_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 common 15 16 import ( 17 "context" 18 "math/rand" 19 "sync" 20 "sync/atomic" 21 "testing" 22 "time" 23 24 "github.com/pingcap/check" 25 "github.com/pingcap/ticdc/pkg/util/testleak" 26 "golang.org/x/sync/errgroup" 27 ) 28 29 type flowControlSuite struct{} 30 31 var _ = check.Suite(&flowControlSuite{}) 32 33 func dummyCallBack() error { 34 return nil 35 } 36 37 type mockCallBacker struct { 38 timesCalled int 39 injectedErr error 40 } 41 42 func (c *mockCallBacker) cb() error { 43 c.timesCalled += 1 44 return c.injectedErr 45 } 46 47 func (s *flowControlSuite) TestMemoryQuotaBasic(c *check.C) { 48 defer testleak.AfterTest(c)() 49 50 controller := NewTableMemoryQuota(1024) 51 sizeCh := make(chan uint64, 1024) 52 var ( 53 wg sync.WaitGroup 54 consumed uint64 55 ) 56 57 wg.Add(1) 58 go func() { 59 defer wg.Done() 60 61 for i := 0; i < 100000; i++ { 62 size := (rand.Int() % 128) + 128 63 err := controller.ConsumeWithBlocking(uint64(size), dummyCallBack) 64 c.Assert(err, check.IsNil) 65 66 c.Assert(atomic.AddUint64(&consumed, uint64(size)), check.Less, uint64(1024)) 67 sizeCh <- uint64(size) 68 } 69 70 close(sizeCh) 71 }() 72 73 wg.Add(1) 74 go func() { 75 defer wg.Done() 76 77 for size := range sizeCh { 78 c.Assert(atomic.LoadUint64(&consumed), check.GreaterEqual, size) 79 atomic.AddUint64(&consumed, -size) 80 controller.Release(size) 81 } 82 }() 83 84 wg.Wait() 85 c.Assert(atomic.LoadUint64(&consumed), check.Equals, uint64(0)) 86 c.Assert(controller.GetConsumption(), check.Equals, uint64(0)) 87 } 88 89 func (s *flowControlSuite) TestMemoryQuotaForceConsume(c *check.C) { 90 defer testleak.AfterTest(c)() 91 92 controller := NewTableMemoryQuota(1024) 93 sizeCh := make(chan uint64, 1024) 94 var ( 95 wg sync.WaitGroup 96 consumed uint64 97 ) 98 99 wg.Add(1) 100 go func() { 101 defer wg.Done() 102 103 for i := 0; i < 100000; i++ { 104 size := (rand.Int() % 128) + 128 105 106 if rand.Int()%3 == 0 { 107 err := controller.ConsumeWithBlocking(uint64(size), dummyCallBack) 108 c.Assert(err, check.IsNil) 109 c.Assert(atomic.AddUint64(&consumed, uint64(size)), check.Less, uint64(1024)) 110 } else { 111 err := controller.ForceConsume(uint64(size)) 112 c.Assert(err, check.IsNil) 113 atomic.AddUint64(&consumed, uint64(size)) 114 } 115 sizeCh <- uint64(size) 116 } 117 118 close(sizeCh) 119 }() 120 121 wg.Add(1) 122 go func() { 123 defer wg.Done() 124 125 for size := range sizeCh { 126 c.Assert(atomic.LoadUint64(&consumed), check.GreaterEqual, size) 127 atomic.AddUint64(&consumed, -size) 128 controller.Release(size) 129 } 130 }() 131 132 wg.Wait() 133 c.Assert(atomic.LoadUint64(&consumed), check.Equals, uint64(0)) 134 } 135 136 // TestMemoryQuotaAbort verifies that Abort works 137 func (s *flowControlSuite) TestMemoryQuotaAbort(c *check.C) { 138 defer testleak.AfterTest(c)() 139 140 controller := NewTableMemoryQuota(1024) 141 var wg sync.WaitGroup 142 wg.Add(1) 143 go func() { 144 defer wg.Done() 145 err := controller.ConsumeWithBlocking(700, dummyCallBack) 146 c.Assert(err, check.IsNil) 147 148 err = controller.ConsumeWithBlocking(700, dummyCallBack) 149 c.Assert(err, check.ErrorMatches, ".*ErrFlowControllerAborted.*") 150 151 err = controller.ForceConsume(700) 152 c.Assert(err, check.ErrorMatches, ".*ErrFlowControllerAborted.*") 153 }() 154 155 time.Sleep(2 * time.Second) 156 controller.Abort() 157 158 wg.Wait() 159 } 160 161 // TestMemoryQuotaReleaseZero verifies that releasing 0 bytes is successful 162 func (s *flowControlSuite) TestMemoryQuotaReleaseZero(c *check.C) { 163 defer testleak.AfterTest(c)() 164 165 controller := NewTableMemoryQuota(1024) 166 controller.Release(0) 167 } 168 169 type mockedEvent struct { 170 resolvedTs uint64 171 size uint64 172 } 173 174 func (s *flowControlSuite) TestFlowControlBasic(c *check.C) { 175 defer testleak.AfterTest(c)() 176 var consumedBytes uint64 177 ctx, cancel := context.WithTimeout(context.TODO(), time.Second*5) 178 defer cancel() 179 errg, ctx := errgroup.WithContext(ctx) 180 mockedRowsCh := make(chan *commitTsSizeEntry, 1024) 181 flowController := NewTableFlowController(2048) 182 183 errg.Go(func() error { 184 lastCommitTs := uint64(1) 185 for i := 0; i < 10000; i++ { 186 if rand.Int()%15 == 0 { 187 lastCommitTs += 10 188 } 189 size := uint64(128 + rand.Int()%64) 190 select { 191 case <-ctx.Done(): 192 return ctx.Err() 193 case mockedRowsCh <- &commitTsSizeEntry{ 194 CommitTs: lastCommitTs, 195 Size: size, 196 }: 197 } 198 } 199 200 close(mockedRowsCh) 201 return nil 202 }) 203 204 eventCh := make(chan *mockedEvent, 1024) 205 errg.Go(func() error { 206 defer close(eventCh) 207 resolvedTs := uint64(0) 208 for { 209 var mockedRow *commitTsSizeEntry 210 select { 211 case <-ctx.Done(): 212 return ctx.Err() 213 case mockedRow = <-mockedRowsCh: 214 } 215 216 if mockedRow == nil { 217 break 218 } 219 220 atomic.AddUint64(&consumedBytes, mockedRow.Size) 221 updatedResolvedTs := false 222 if resolvedTs != mockedRow.CommitTs { 223 c.Assert(resolvedTs, check.Less, mockedRow.CommitTs) 224 select { 225 case <-ctx.Done(): 226 return ctx.Err() 227 case eventCh <- &mockedEvent{ 228 resolvedTs: resolvedTs, 229 }: 230 } 231 resolvedTs = mockedRow.CommitTs 232 updatedResolvedTs = true 233 } 234 err := flowController.Consume(mockedRow.CommitTs, mockedRow.Size, dummyCallBack) 235 c.Check(err, check.IsNil) 236 select { 237 case <-ctx.Done(): 238 return ctx.Err() 239 case eventCh <- &mockedEvent{ 240 size: mockedRow.Size, 241 }: 242 } 243 if updatedResolvedTs { 244 // new Txn 245 c.Assert(atomic.LoadUint64(&consumedBytes), check.Less, uint64(2048)) 246 c.Assert(flowController.GetConsumption(), check.Less, uint64(2048)) 247 } 248 } 249 select { 250 case <-ctx.Done(): 251 return ctx.Err() 252 case eventCh <- &mockedEvent{ 253 resolvedTs: resolvedTs, 254 }: 255 } 256 257 return nil 258 }) 259 260 errg.Go(func() error { 261 for { 262 var event *mockedEvent 263 select { 264 case <-ctx.Done(): 265 return ctx.Err() 266 case event = <-eventCh: 267 } 268 269 if event == nil { 270 break 271 } 272 273 if event.size != 0 { 274 atomic.AddUint64(&consumedBytes, -event.size) 275 } else { 276 flowController.Release(event.resolvedTs) 277 } 278 } 279 280 return nil 281 }) 282 283 c.Assert(errg.Wait(), check.IsNil) 284 c.Assert(atomic.LoadUint64(&consumedBytes), check.Equals, uint64(0)) 285 } 286 287 func (s *flowControlSuite) TestFlowControlAbort(c *check.C) { 288 defer testleak.AfterTest(c)() 289 290 callBacker := &mockCallBacker{} 291 controller := NewTableFlowController(1024) 292 var wg sync.WaitGroup 293 wg.Add(1) 294 go func() { 295 defer wg.Done() 296 297 err := controller.Consume(1, 1000, callBacker.cb) 298 c.Assert(err, check.IsNil) 299 c.Assert(callBacker.timesCalled, check.Equals, 0) 300 err = controller.Consume(2, 1000, callBacker.cb) 301 c.Assert(err, check.ErrorMatches, ".*ErrFlowControllerAborted.*") 302 c.Assert(callBacker.timesCalled, check.Equals, 1) 303 err = controller.Consume(2, 10, callBacker.cb) 304 c.Assert(err, check.ErrorMatches, ".*ErrFlowControllerAborted.*") 305 c.Assert(callBacker.timesCalled, check.Equals, 1) 306 }() 307 308 time.Sleep(3 * time.Second) 309 controller.Abort() 310 311 wg.Wait() 312 } 313 314 func (s *flowControlSuite) TestFlowControlCallBack(c *check.C) { 315 defer testleak.AfterTest(c)() 316 var consumedBytes uint64 317 ctx, cancel := context.WithTimeout(context.TODO(), time.Second*5) 318 defer cancel() 319 errg, ctx := errgroup.WithContext(ctx) 320 mockedRowsCh := make(chan *commitTsSizeEntry, 1024) 321 flowController := NewTableFlowController(512) 322 323 errg.Go(func() error { 324 lastCommitTs := uint64(1) 325 for i := 0; i < 10000; i++ { 326 if rand.Int()%15 == 0 { 327 lastCommitTs += 10 328 } 329 size := uint64(128 + rand.Int()%64) 330 select { 331 case <-ctx.Done(): 332 return ctx.Err() 333 case mockedRowsCh <- &commitTsSizeEntry{ 334 CommitTs: lastCommitTs, 335 Size: size, 336 }: 337 } 338 } 339 340 close(mockedRowsCh) 341 return nil 342 }) 343 344 eventCh := make(chan *mockedEvent, 1024) 345 errg.Go(func() error { 346 defer close(eventCh) 347 lastCRTs := uint64(0) 348 for { 349 var mockedRow *commitTsSizeEntry 350 select { 351 case <-ctx.Done(): 352 return ctx.Err() 353 case mockedRow = <-mockedRowsCh: 354 } 355 356 if mockedRow == nil { 357 break 358 } 359 360 atomic.AddUint64(&consumedBytes, mockedRow.Size) 361 err := flowController.Consume(mockedRow.CommitTs, mockedRow.Size, func() error { 362 select { 363 case <-ctx.Done(): 364 return ctx.Err() 365 case eventCh <- &mockedEvent{ 366 resolvedTs: lastCRTs, 367 }: 368 } 369 return nil 370 }) 371 c.Assert(err, check.IsNil) 372 lastCRTs = mockedRow.CommitTs 373 374 select { 375 case <-ctx.Done(): 376 return ctx.Err() 377 case eventCh <- &mockedEvent{ 378 size: mockedRow.Size, 379 }: 380 } 381 } 382 select { 383 case <-ctx.Done(): 384 return ctx.Err() 385 case eventCh <- &mockedEvent{ 386 resolvedTs: lastCRTs, 387 }: 388 } 389 390 return nil 391 }) 392 393 errg.Go(func() error { 394 for { 395 var event *mockedEvent 396 select { 397 case <-ctx.Done(): 398 return ctx.Err() 399 case event = <-eventCh: 400 } 401 402 if event == nil { 403 break 404 } 405 406 if event.size != 0 { 407 atomic.AddUint64(&consumedBytes, -event.size) 408 } else { 409 flowController.Release(event.resolvedTs) 410 } 411 } 412 413 return nil 414 }) 415 416 c.Assert(errg.Wait(), check.IsNil) 417 c.Assert(atomic.LoadUint64(&consumedBytes), check.Equals, uint64(0)) 418 } 419 420 func (s *flowControlSuite) TestFlowControlCallBackNotBlockingRelease(c *check.C) { 421 defer testleak.AfterTest(c)() 422 423 var wg sync.WaitGroup 424 controller := NewTableFlowController(512) 425 wg.Add(1) 426 427 ctx, cancel := context.WithCancel(context.TODO()) 428 defer cancel() 429 430 go func() { 431 defer wg.Done() 432 err := controller.Consume(1, 511, func() error { 433 c.Fatalf("unreachable") 434 return nil 435 }) 436 c.Assert(err, check.IsNil) 437 438 var isBlocked int32 439 wg.Add(1) 440 go func() { 441 defer wg.Done() 442 <-time.After(time.Second * 1) 443 // makes sure that this test case is valid 444 c.Assert(atomic.LoadInt32(&isBlocked), check.Equals, int32(1)) 445 controller.Release(1) 446 cancel() 447 }() 448 449 err = controller.Consume(2, 511, func() error { 450 atomic.StoreInt32(&isBlocked, 1) 451 <-ctx.Done() 452 atomic.StoreInt32(&isBlocked, 0) 453 return ctx.Err() 454 }) 455 456 c.Assert(err, check.ErrorMatches, ".*context canceled.*") 457 }() 458 459 wg.Wait() 460 } 461 462 func (s *flowControlSuite) TestFlowControlCallBackError(c *check.C) { 463 defer testleak.AfterTest(c)() 464 465 var wg sync.WaitGroup 466 controller := NewTableFlowController(512) 467 wg.Add(1) 468 469 ctx, cancel := context.WithCancel(context.TODO()) 470 defer cancel() 471 472 go func() { 473 defer wg.Done() 474 err := controller.Consume(1, 511, func() error { 475 c.Fatalf("unreachable") 476 return nil 477 }) 478 c.Assert(err, check.IsNil) 479 err = controller.Consume(2, 511, func() error { 480 <-ctx.Done() 481 return ctx.Err() 482 }) 483 c.Assert(err, check.ErrorMatches, ".*context canceled.*") 484 }() 485 486 time.Sleep(100 * time.Millisecond) 487 cancel() 488 489 wg.Wait() 490 } 491 492 func (s *flowControlSuite) TestFlowControlConsumeLargerThanQuota(c *check.C) { 493 defer testleak.AfterTest(c)() 494 495 controller := NewTableFlowController(1024) 496 err := controller.Consume(1, 2048, func() error { 497 c.Fatalf("unreachable") 498 return nil 499 }) 500 c.Assert(err, check.ErrorMatches, ".*ErrFlowControllerEventLargerThanQuota.*") 501 } 502 503 func BenchmarkTableFlowController(B *testing.B) { 504 ctx, cancel := context.WithTimeout(context.TODO(), time.Second*5) 505 defer cancel() 506 errg, ctx := errgroup.WithContext(ctx) 507 mockedRowsCh := make(chan *commitTsSizeEntry, 102400) 508 flowController := NewTableFlowController(20 * 1024 * 1024) // 20M 509 510 errg.Go(func() error { 511 lastCommitTs := uint64(1) 512 for i := 0; i < B.N; i++ { 513 if rand.Int()%15 == 0 { 514 lastCommitTs += 10 515 } 516 size := uint64(1024 + rand.Int()%1024) 517 select { 518 case <-ctx.Done(): 519 return ctx.Err() 520 case mockedRowsCh <- &commitTsSizeEntry{ 521 CommitTs: lastCommitTs, 522 Size: size, 523 }: 524 } 525 } 526 527 close(mockedRowsCh) 528 return nil 529 }) 530 531 eventCh := make(chan *mockedEvent, 102400) 532 errg.Go(func() error { 533 defer close(eventCh) 534 resolvedTs := uint64(0) 535 for { 536 var mockedRow *commitTsSizeEntry 537 select { 538 case <-ctx.Done(): 539 return ctx.Err() 540 case mockedRow = <-mockedRowsCh: 541 } 542 543 if mockedRow == nil { 544 break 545 } 546 547 if resolvedTs != mockedRow.CommitTs { 548 select { 549 case <-ctx.Done(): 550 return ctx.Err() 551 case eventCh <- &mockedEvent{ 552 resolvedTs: resolvedTs, 553 }: 554 } 555 resolvedTs = mockedRow.CommitTs 556 } 557 err := flowController.Consume(mockedRow.CommitTs, mockedRow.Size, dummyCallBack) 558 if err != nil { 559 B.Fatal(err) 560 } 561 select { 562 case <-ctx.Done(): 563 return ctx.Err() 564 case eventCh <- &mockedEvent{ 565 size: mockedRow.Size, 566 }: 567 } 568 } 569 select { 570 case <-ctx.Done(): 571 return ctx.Err() 572 case eventCh <- &mockedEvent{ 573 resolvedTs: resolvedTs, 574 }: 575 } 576 577 return nil 578 }) 579 580 errg.Go(func() error { 581 for { 582 var event *mockedEvent 583 select { 584 case <-ctx.Done(): 585 return ctx.Err() 586 case event = <-eventCh: 587 } 588 589 if event == nil { 590 break 591 } 592 593 if event.size == 0 { 594 flowController.Release(event.resolvedTs) 595 } 596 } 597 598 return nil 599 }) 600 }