github.com/pingcap/ticdc@v0.0.0-20220526033649-485a10ef2652/cdc/kv/token_region_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 kv 15 16 import ( 17 "context" 18 "fmt" 19 "sync/atomic" 20 "time" 21 22 "github.com/pingcap/check" 23 "github.com/pingcap/errors" 24 "github.com/pingcap/ticdc/pkg/util/testleak" 25 "github.com/pingcap/tidb/store/tikv" 26 "golang.org/x/sync/errgroup" 27 ) 28 29 type tokenRegionSuite struct { 30 } 31 32 var _ = check.Suite(&tokenRegionSuite{}) 33 34 func (s *tokenRegionSuite) TestRouter(c *check.C) { 35 defer testleak.AfterTest(c)() 36 store := "store-1" 37 limit := 10 38 r := NewSizedRegionRouter(context.Background(), limit) 39 for i := 0; i < limit; i++ { 40 r.AddRegion(singleRegionInfo{ts: uint64(i), rpcCtx: &tikv.RPCContext{Addr: store}}) 41 } 42 regions := make([]singleRegionInfo, 0, limit) 43 // limit is less than regionScanLimitPerTable 44 for i := 0; i < limit; i++ { 45 select { 46 case sri := <-r.Chan(): 47 c.Assert(sri.ts, check.Equals, uint64(i)) 48 r.Acquire(store) 49 regions = append(regions, sri) 50 default: 51 c.Error("expect region info from router") 52 } 53 } 54 c.Assert(r.tokens[store], check.Equals, limit) 55 for range regions { 56 r.Release(store) 57 } 58 c.Assert(r.tokens[store], check.Equals, 0) 59 } 60 61 func (s *tokenRegionSuite) TestRouterWithFastConsumer(c *check.C) { 62 defer testleak.AfterTest(c)() 63 s.testRouterWithConsumer(c, func() {}) 64 } 65 66 func (s *tokenRegionSuite) TestRouterWithSlowConsumer(c *check.C) { 67 defer testleak.AfterTest(c)() 68 s.testRouterWithConsumer(c, func() { time.Sleep(time.Millisecond * 15) }) 69 } 70 71 func (s *tokenRegionSuite) testRouterWithConsumer(c *check.C, funcDoSth func()) { 72 ctx, cancel := context.WithCancel(context.Background()) 73 defer cancel() 74 75 store := "store-1" 76 limit := 20 77 r := NewSizedRegionRouter(context.Background(), limit) 78 for i := 0; i < limit*2; i++ { 79 r.AddRegion(singleRegionInfo{ts: uint64(i), rpcCtx: &tikv.RPCContext{Addr: store}}) 80 } 81 received := uint64(0) 82 for i := 0; i < regionRouterChanSize; i++ { 83 <-r.Chan() 84 atomic.AddUint64(&received, 1) 85 r.Acquire(store) 86 } 87 88 wg, ctx := errgroup.WithContext(ctx) 89 wg.Go(func() error { 90 return r.Run(ctx) 91 }) 92 93 wg.Go(func() error { 94 for i := 0; i < regionRouterChanSize; i++ { 95 r.Release(store) 96 } 97 return nil 98 }) 99 100 wg.Go(func() error { 101 for { 102 select { 103 case <-ctx.Done(): 104 return ctx.Err() 105 case <-r.Chan(): 106 r.Acquire(store) 107 atomic.AddUint64(&received, 1) 108 r.Release(store) 109 funcDoSth() 110 if atomic.LoadUint64(&received) == uint64(limit*4) { 111 cancel() 112 } 113 } 114 } 115 }) 116 117 for i := 0; i < limit*2; i++ { 118 r.AddRegion(singleRegionInfo{ts: uint64(i), rpcCtx: &tikv.RPCContext{Addr: store}}) 119 } 120 121 err := wg.Wait() 122 c.Assert(errors.Cause(err), check.Equals, context.Canceled) 123 c.Assert(r.tokens[store], check.Equals, 0) 124 } 125 126 func (s *tokenRegionSuite) TestRouterWithMultiStores(c *check.C) { 127 defer testleak.AfterTest(c)() 128 129 ctx, cancel := context.WithCancel(context.Background()) 130 defer cancel() 131 132 storeN := 10 133 stores := make([]string, 0, storeN) 134 for i := 0; i < storeN; i++ { 135 stores = append(stores, fmt.Sprintf("store-%d", i)) 136 } 137 limit := 20 138 r := NewSizedRegionRouter(context.Background(), limit) 139 140 for _, store := range stores { 141 for j := 0; j < limit*2; j++ { 142 r.AddRegion(singleRegionInfo{ts: uint64(j), rpcCtx: &tikv.RPCContext{Addr: store}}) 143 } 144 } 145 received := uint64(0) 146 wg, ctx := errgroup.WithContext(ctx) 147 wg.Go(func() error { 148 return r.Run(ctx) 149 }) 150 151 for _, store := range stores { 152 store := store 153 wg.Go(func() error { 154 for { 155 select { 156 case <-ctx.Done(): 157 return ctx.Err() 158 case <-r.Chan(): 159 r.Acquire(store) 160 atomic.AddUint64(&received, 1) 161 r.Release(store) 162 if atomic.LoadUint64(&received) == uint64(limit*4*storeN) { 163 cancel() 164 } 165 } 166 } 167 }) 168 } 169 170 for _, store := range stores { 171 for i := 0; i < limit*2; i++ { 172 r.AddRegion(singleRegionInfo{ts: uint64(i), rpcCtx: &tikv.RPCContext{Addr: store}}) 173 } 174 } 175 176 err := wg.Wait() 177 c.Assert(errors.Cause(err), check.Equals, context.Canceled) 178 for _, store := range stores { 179 c.Assert(r.tokens[store], check.Equals, 0) 180 } 181 }