github.com/pingcap/ticdc@v0.0.0-20220526033649-485a10ef2652/cdc/kv/grpc_pool_impl.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/log" 22 cerror "github.com/pingcap/ticdc/pkg/errors" 23 "github.com/pingcap/ticdc/pkg/security" 24 "go.uber.org/zap" 25 "google.golang.org/grpc" 26 gbackoff "google.golang.org/grpc/backoff" 27 "google.golang.org/grpc/keepalive" 28 ) 29 30 const ( 31 // The default max number of TiKV concurrent streams in each connection is 1024 32 grpcConnCapacity = 1000 33 34 // resizeBucket means how many buckets will be extended when resizing an conn array 35 resizeBucketStep = 2 36 37 updateMetricInterval = 1 * time.Minute 38 recycleConnInterval = 10 * time.Minute 39 ) 40 41 // connArray is an array of sharedConn 42 type connArray struct { 43 // target is TiKV storage address 44 target string 45 46 mu sync.Mutex 47 conns []*sharedConn 48 49 // next is used for fetching sharedConn in a round robin way 50 next int 51 } 52 53 func newConnArray(target string) *connArray { 54 return &connArray{target: target} 55 } 56 57 // resize increases conn array size by `size` parameter 58 func (ca *connArray) resize(ctx context.Context, credential *security.Credential, size int) error { 59 conns := make([]*sharedConn, 0, size) 60 for i := 0; i < size; i++ { 61 conn, err := createClientConn(ctx, credential, ca.target) 62 if err != nil { 63 return err 64 } 65 conns = append(conns, &sharedConn{ClientConn: conn, active: 0}) 66 } 67 ca.conns = append(ca.conns, conns...) 68 return nil 69 } 70 71 func createClientConn(ctx context.Context, credential *security.Credential, target string) (*grpc.ClientConn, error) { 72 grpcTLSOption, err := credential.ToGRPCDialOption() 73 if err != nil { 74 return nil, err 75 } 76 ctx, cancel := context.WithTimeout(ctx, dialTimeout) 77 defer cancel() 78 79 conn, err := grpc.DialContext( 80 ctx, 81 target, 82 grpcTLSOption, 83 grpc.WithInitialWindowSize(grpcInitialWindowSize), 84 grpc.WithInitialConnWindowSize(grpcInitialConnWindowSize), 85 grpc.WithDefaultCallOptions(grpc.MaxCallRecvMsgSize(grpcMaxCallRecvMsgSize)), 86 grpc.WithUnaryInterceptor(grpcMetrics.UnaryClientInterceptor()), 87 grpc.WithStreamInterceptor(grpcMetrics.StreamClientInterceptor()), 88 grpc.WithConnectParams(grpc.ConnectParams{ 89 Backoff: gbackoff.Config{ 90 BaseDelay: time.Second, 91 Multiplier: 1.1, 92 Jitter: 0.1, 93 MaxDelay: 3 * time.Second, 94 }, 95 MinConnectTimeout: 3 * time.Second, 96 }), 97 grpc.WithKeepaliveParams(keepalive.ClientParameters{ 98 Time: 10 * time.Second, 99 Timeout: 3 * time.Second, 100 PermitWithoutStream: true, 101 }), 102 ) 103 if err != nil { 104 return nil, cerror.WrapError(cerror.ErrGRPCDialFailed, err) 105 } 106 return conn, nil 107 } 108 109 // getNext gets next available sharedConn, if all conns are not available, scale 110 // the connArray to double size. 111 func (ca *connArray) getNext(ctx context.Context, credential *security.Credential) (*sharedConn, error) { 112 ca.mu.Lock() 113 defer ca.mu.Unlock() 114 115 if len(ca.conns) == 0 { 116 err := ca.resize(ctx, credential, resizeBucketStep) 117 if err != nil { 118 return nil, err 119 } 120 } 121 for current := ca.next; current < ca.next+len(ca.conns); current++ { 122 conn := ca.conns[current%len(ca.conns)] 123 if conn.active < grpcConnCapacity { 124 conn.active++ 125 ca.next = (current + 1) % len(ca.conns) 126 return conn, nil 127 } 128 } 129 130 current := len(ca.conns) 131 // if there is no available conn, increase connArray size by 2. 132 err := ca.resize(ctx, credential, resizeBucketStep) 133 if err != nil { 134 return nil, err 135 } 136 ca.conns[current].active++ 137 ca.next = current + 1 138 return ca.conns[current], nil 139 } 140 141 // recycle removes idle sharedConn, return true if no active gPRC connections remained. 142 func (ca *connArray) recycle() (empty bool) { 143 ca.mu.Lock() 144 defer ca.mu.Unlock() 145 i := 0 146 for _, conn := range ca.conns { 147 if conn.active > 0 { 148 ca.conns[i] = conn 149 i++ 150 } else { 151 // tear down this grpc.ClientConn, we don't use it anymore, the returned 152 // not-nil error can be ignored 153 conn.Close() //nolint:errcheck 154 } 155 } 156 // erasing truncated values 157 for j := i; j < len(ca.conns); j++ { 158 ca.conns[j] = nil 159 } 160 ca.conns = ca.conns[:i] 161 return len(ca.conns) == 0 162 } 163 164 func (ca *connArray) activeCount() (count int64) { 165 ca.mu.Lock() 166 defer ca.mu.Unlock() 167 for _, conn := range ca.conns { 168 count += conn.active 169 } 170 return 171 } 172 173 // close tears down all ClientConns maintained in connArray 174 func (ca *connArray) close() { 175 ca.mu.Lock() 176 defer ca.mu.Unlock() 177 for _, conn := range ca.conns { 178 // tear down this grpc.ClientConn, we don't use it anymore, the returned 179 // not-nil error can be ignored 180 conn.Close() //nolint:errcheck 181 } 182 } 183 184 // GrpcPoolImpl implement GrpcPool interface 185 type GrpcPoolImpl struct { 186 poolMu sync.RWMutex 187 // bucketConns maps from TiKV store address to a connArray, which stores a 188 // a slice of gRPC connections. 189 bucketConns map[string]*connArray 190 191 credential *security.Credential 192 193 // lifecycles of all gPRC connections are bounded to this context 194 ctx context.Context 195 } 196 197 // NewGrpcPoolImpl creates a new GrpcPoolImpl instance 198 func NewGrpcPoolImpl(ctx context.Context, credential *security.Credential) *GrpcPoolImpl { 199 return &GrpcPoolImpl{ 200 credential: credential, 201 bucketConns: make(map[string]*connArray), 202 ctx: ctx, 203 } 204 } 205 206 // GetConn implements GrpcPool.GetConn 207 func (pool *GrpcPoolImpl) GetConn(addr string) (*sharedConn, error) { 208 pool.poolMu.Lock() 209 defer pool.poolMu.Unlock() 210 if _, ok := pool.bucketConns[addr]; !ok { 211 pool.bucketConns[addr] = newConnArray(addr) 212 } 213 return pool.bucketConns[addr].getNext(pool.ctx, pool.credential) 214 } 215 216 // ReleaseConn implements GrpcPool.ReleaseConn 217 func (pool *GrpcPoolImpl) ReleaseConn(sc *sharedConn, addr string) { 218 pool.poolMu.RLock() 219 defer pool.poolMu.RUnlock() 220 if bucket, ok := pool.bucketConns[addr]; !ok { 221 log.Warn("resource is not found in grpc pool", zap.String("addr", addr)) 222 } else { 223 bucket.mu.Lock() 224 sc.active-- 225 bucket.mu.Unlock() 226 } 227 } 228 229 // RecycleConn implements GrpcPool.RecycleConn 230 func (pool *GrpcPoolImpl) RecycleConn(ctx context.Context) { 231 recycleTicker := time.NewTicker(recycleConnInterval) 232 defer recycleTicker.Stop() 233 metricTicker := time.NewTicker(updateMetricInterval) 234 defer metricTicker.Stop() 235 for { 236 select { 237 case <-ctx.Done(): 238 return 239 case <-recycleTicker.C: 240 pool.poolMu.Lock() 241 for addr, bucket := range pool.bucketConns { 242 empty := bucket.recycle() 243 if empty { 244 log.Info("recycle connections in grpc pool", zap.String("address", addr)) 245 delete(pool.bucketConns, addr) 246 grpcPoolStreamGauge.DeleteLabelValues(addr) 247 } 248 } 249 pool.poolMu.Unlock() 250 case <-metricTicker.C: 251 pool.poolMu.RLock() 252 for addr, bucket := range pool.bucketConns { 253 grpcPoolStreamGauge.WithLabelValues(addr).Set(float64(bucket.activeCount())) 254 } 255 pool.poolMu.RUnlock() 256 } 257 } 258 } 259 260 // Close implements GrpcPool.Close 261 func (pool *GrpcPoolImpl) Close() { 262 pool.poolMu.Lock() 263 defer pool.poolMu.Unlock() 264 for _, bucket := range pool.bucketConns { 265 bucket.close() 266 } 267 }