github.com/pingcap/ticdc@v0.0.0-20220526033649-485a10ef2652/cdc/kv/token_region.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 "sync" 19 "time" 20 21 "github.com/pingcap/errors" 22 "github.com/pingcap/ticdc/pkg/util" 23 "github.com/prometheus/client_golang/prometheus" 24 ) 25 26 const ( 27 // buffer size for ranged region consumer 28 regionRouterChanSize = 16 29 // sizedRegionRouter checks region buffer every 100ms 30 sizedRegionCheckInterval = 100 * time.Millisecond 31 ) 32 33 // LimitRegionRouter defines an interface that can buffer singleRegionInfo 34 // and provide token based consumption 35 type LimitRegionRouter interface { 36 // Chan returns a singleRegionInfo channel that can be consumed from 37 Chan() <-chan singleRegionInfo 38 // AddRegion adds an singleRegionInfo to buffer, this function is thread-safe 39 AddRegion(task singleRegionInfo) 40 // Acquire acquires one token 41 Acquire(id string) 42 // Release gives back one token, this function is thread-safe 43 Release(id string) 44 // Run runs in background and does some logic work 45 Run(ctx context.Context) error 46 } 47 48 type srrMetrics struct { 49 capture string 50 changefeed string 51 tokens map[string]prometheus.Gauge 52 } 53 54 func newSrrMetrics(ctx context.Context) *srrMetrics { 55 captureAddr := util.CaptureAddrFromCtx(ctx) 56 changefeed := util.ChangefeedIDFromCtx(ctx) 57 return &srrMetrics{ 58 capture: captureAddr, 59 changefeed: changefeed, 60 tokens: make(map[string]prometheus.Gauge), 61 } 62 } 63 64 type sizedRegionRouter struct { 65 buffer map[string][]singleRegionInfo 66 output chan singleRegionInfo 67 lock sync.Mutex 68 metrics *srrMetrics 69 tokens map[string]int 70 sizeLimit int 71 } 72 73 // NewSizedRegionRouter creates a new sizedRegionRouter 74 func NewSizedRegionRouter(ctx context.Context, sizeLimit int) *sizedRegionRouter { 75 return &sizedRegionRouter{ 76 buffer: make(map[string][]singleRegionInfo), 77 output: make(chan singleRegionInfo, regionRouterChanSize), 78 sizeLimit: sizeLimit, 79 tokens: make(map[string]int), 80 metrics: newSrrMetrics(ctx), 81 } 82 } 83 84 func (r *sizedRegionRouter) Chan() <-chan singleRegionInfo { 85 return r.output 86 } 87 88 func (r *sizedRegionRouter) AddRegion(sri singleRegionInfo) { 89 r.lock.Lock() 90 var id string 91 // if rpcCtx is not provided, use the default "" bucket 92 if sri.rpcCtx != nil { 93 id = sri.rpcCtx.Addr 94 } 95 if r.sizeLimit > r.tokens[id] && len(r.output) < regionRouterChanSize { 96 r.output <- sri 97 } else { 98 r.buffer[id] = append(r.buffer[id], sri) 99 } 100 r.lock.Unlock() 101 } 102 103 func (r *sizedRegionRouter) Acquire(id string) { 104 r.lock.Lock() 105 defer r.lock.Unlock() 106 r.tokens[id]++ 107 if _, ok := r.metrics.tokens[id]; !ok { 108 r.metrics.tokens[id] = clientRegionTokenSize.WithLabelValues(id, r.metrics.changefeed, r.metrics.capture) 109 } 110 r.metrics.tokens[id].Inc() 111 } 112 113 func (r *sizedRegionRouter) Release(id string) { 114 r.lock.Lock() 115 defer r.lock.Unlock() 116 r.tokens[id]-- 117 if _, ok := r.metrics.tokens[id]; !ok { 118 r.metrics.tokens[id] = clientRegionTokenSize.WithLabelValues(id, r.metrics.changefeed, r.metrics.capture) 119 } 120 r.metrics.tokens[id].Dec() 121 } 122 123 func (r *sizedRegionRouter) Run(ctx context.Context) error { 124 ticker := time.NewTicker(sizedRegionCheckInterval) 125 defer ticker.Stop() 126 for { 127 select { 128 case <-ctx.Done(): 129 return errors.Trace(ctx.Err()) 130 case <-ticker.C: 131 r.lock.Lock() 132 for id, buf := range r.buffer { 133 available := r.sizeLimit - r.tokens[id] 134 // the tokens used could be more then size limit, since we have 135 // a sized channel as level1 cache 136 if available <= 0 { 137 continue 138 } 139 if available > len(buf) { 140 available = len(buf) 141 } 142 // to avoid deadlock because when consuming from the output channel. 143 // onRegionFail could decrease tokens, which requires lock protection. 144 if available > regionRouterChanSize-len(r.output) { 145 available = regionRouterChanSize - len(r.output) 146 } 147 if available == 0 { 148 continue 149 } 150 for i := 0; i < available; i++ { 151 select { 152 case <-ctx.Done(): 153 r.lock.Unlock() 154 return errors.Trace(ctx.Err()) 155 case r.output <- buf[i]: 156 } 157 } 158 r.buffer[id] = r.buffer[id][available:] 159 } 160 r.lock.Unlock() 161 } 162 } 163 }