github.com/pingcap/tiflow@v0.0.0-20240520035814-5bf52d54e205/pkg/etcd/client.go (about) 1 // Copyright 2020 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 etcd 15 16 import ( 17 "context" 18 "crypto/tls" 19 "fmt" 20 "sync" 21 "time" 22 23 "github.com/benbjohnson/clock" 24 "github.com/pingcap/errors" 25 "github.com/pingcap/log" 26 cerror "github.com/pingcap/tiflow/pkg/errors" 27 "github.com/pingcap/tiflow/pkg/errorutil" 28 "github.com/pingcap/tiflow/pkg/retry" 29 "github.com/pingcap/tiflow/pkg/security" 30 "github.com/pingcap/tiflow/pkg/util" 31 "github.com/prometheus/client_golang/prometheus" 32 "github.com/tikv/pd/pkg/errs" 33 "go.etcd.io/etcd/api/v3/v3rpc/rpctypes" 34 v3rpc "go.etcd.io/etcd/api/v3/v3rpc/rpctypes" 35 "go.etcd.io/etcd/client/pkg/v3/logutil" 36 clientV3 "go.etcd.io/etcd/client/v3" 37 clientv3 "go.etcd.io/etcd/client/v3" 38 "go.uber.org/zap" 39 "go.uber.org/zap/zapcore" 40 "google.golang.org/grpc" 41 "google.golang.org/grpc/backoff" 42 "google.golang.org/grpc/codes" 43 "google.golang.org/grpc/keepalive" 44 ) 45 46 // etcd operation names 47 const ( 48 EtcdPut = "Put" 49 EtcdGet = "Get" 50 EtcdTxn = "Txn" 51 EtcdDel = "Del" 52 EtcdGrant = "Grant" 53 EtcdRevoke = "Revoke" 54 ) 55 56 const ( 57 backoffBaseDelayInMs = 500 58 // in previous/backoff retry pkg, the DefaultMaxInterval = 60 * time.Second 59 backoffMaxDelayInMs = 60 * 1000 60 // If no msg comes from an etcd watchCh for etcdWatchChTimeoutDuration long, 61 // we should cancel the watchCh and request a new watchCh from etcd client 62 etcdWatchChTimeoutDuration = 10 * time.Second 63 // If no msg comes from an etcd watchCh for etcdRequestProgressDuration long, 64 // we should call RequestProgress of etcd client 65 etcdRequestProgressDuration = 1 * time.Second 66 // etcdWatchChBufferSize is arbitrarily specified, it will be modified in the future 67 etcdWatchChBufferSize = 16 68 // etcdClientTimeoutDuration represents the timeout duration for 69 // etcd client to execute a remote call 70 etcdClientTimeoutDuration = 30 * time.Second 71 ) 72 73 var ( 74 txnEmptyCmps = []clientV3.Cmp{} 75 txnEmptyOpsThen = []clientV3.Op{} 76 // TxnEmptyOpsElse is a no-op operation. 77 TxnEmptyOpsElse = []clientV3.Op{} 78 ) 79 80 // set to var instead of const for mocking the value to speedup test 81 var maxTries uint64 = 12 82 83 // Client is a simple wrapper that adds retry to etcd RPC 84 type Client struct { 85 cli *clientV3.Client 86 metrics map[string]prometheus.Counter 87 // clock is for making it easier to mock time-related data structures in unit tests 88 clock clock.Clock 89 } 90 91 // Wrap warps a clientV3.Client that provides etcd APIs required by TiCDC. 92 func Wrap(cli *clientV3.Client, metrics map[string]prometheus.Counter) *Client { 93 return &Client{cli: cli, metrics: metrics, clock: clock.New()} 94 } 95 96 // Unwrap returns a clientV3.Client 97 func (c *Client) Unwrap() *clientV3.Client { 98 return c.cli 99 } 100 101 func retryRPC(rpcName string, metric prometheus.Counter, etcdRPC func() error) error { 102 // By default, PD etcd sets [3s, 6s) for election timeout. 103 // Some rpc could fail due to etcd errors, like "proposal dropped". 104 // Retry at least two election timeout to handle the case that two PDs restarted 105 // (the first election maybe failed). 106 // 16s = \sum_{n=0}^{6} 0.5*1.5^n 107 return retry.Do(context.Background(), func() error { 108 err := etcdRPC() 109 if err != nil && errors.Cause(err) != context.Canceled { 110 log.Warn("etcd RPC failed", zap.String("RPC", rpcName), zap.Error(err)) 111 } 112 if metric != nil { 113 metric.Inc() 114 } 115 return err 116 }, retry.WithBackoffBaseDelay(backoffBaseDelayInMs), 117 retry.WithBackoffMaxDelay(backoffMaxDelayInMs), 118 retry.WithMaxTries(maxTries), 119 retry.WithIsRetryableErr(isRetryableError(rpcName))) 120 } 121 122 // Put delegates request to clientV3.KV.Put 123 func (c *Client) Put( 124 ctx context.Context, key, val string, opts ...clientV3.OpOption, 125 ) (resp *clientV3.PutResponse, err error) { 126 putCtx, cancel := context.WithTimeout(ctx, etcdClientTimeoutDuration) 127 defer cancel() 128 err = retryRPC(EtcdPut, c.metrics[EtcdPut], func() error { 129 var inErr error 130 resp, inErr = c.cli.Put(putCtx, key, val, opts...) 131 return inErr 132 }) 133 return 134 } 135 136 // Get delegates request to clientV3.KV.Get 137 func (c *Client) Get( 138 ctx context.Context, key string, opts ...clientV3.OpOption, 139 ) (resp *clientV3.GetResponse, err error) { 140 getCtx, cancel := context.WithTimeout(ctx, etcdClientTimeoutDuration) 141 defer cancel() 142 err = retryRPC(EtcdGet, c.metrics[EtcdGet], func() error { 143 var inErr error 144 resp, inErr = c.cli.Get(getCtx, key, opts...) 145 return inErr 146 }) 147 return 148 } 149 150 // Delete delegates request to clientV3.KV.Delete 151 func (c *Client) Delete( 152 ctx context.Context, key string, opts ...clientV3.OpOption, 153 ) (resp *clientV3.DeleteResponse, err error) { 154 if metric, ok := c.metrics[EtcdDel]; ok { 155 metric.Inc() 156 } 157 delCtx, cancel := context.WithTimeout(ctx, etcdClientTimeoutDuration) 158 defer cancel() 159 // We don't retry on delete operation. It's dangerous. 160 return c.cli.Delete(delCtx, key, opts...) 161 } 162 163 // Txn delegates request to clientV3.KV.Txn. The error returned can only be a non-retryable error, 164 // such as context.Canceled, context.DeadlineExceeded, errors.ErrReachMaxTry. 165 func (c *Client) Txn( 166 ctx context.Context, cmps []clientV3.Cmp, opsThen, opsElse []clientV3.Op, 167 ) (resp *clientV3.TxnResponse, err error) { 168 txnCtx, cancel := context.WithTimeout(ctx, etcdClientTimeoutDuration) 169 defer cancel() 170 err = retryRPC(EtcdTxn, c.metrics[EtcdTxn], func() error { 171 var inErr error 172 resp, inErr = c.cli.Txn(txnCtx).If(cmps...).Then(opsThen...).Else(opsElse...).Commit() 173 return inErr 174 }) 175 return 176 } 177 178 // Grant delegates request to clientV3.Lease.Grant 179 func (c *Client) Grant( 180 ctx context.Context, ttl int64, 181 ) (resp *clientV3.LeaseGrantResponse, err error) { 182 grantCtx, cancel := context.WithTimeout(ctx, etcdClientTimeoutDuration) 183 defer cancel() 184 err = retryRPC(EtcdGrant, c.metrics[EtcdGrant], func() error { 185 var inErr error 186 resp, inErr = c.cli.Grant(grantCtx, ttl) 187 return inErr 188 }) 189 return 190 } 191 192 // Revoke delegates request to clientV3.Lease.Revoke 193 func (c *Client) Revoke( 194 ctx context.Context, id clientV3.LeaseID, 195 ) (resp *clientV3.LeaseRevokeResponse, err error) { 196 revokeCtx, cancel := context.WithTimeout(ctx, etcdClientTimeoutDuration) 197 defer cancel() 198 err = retryRPC(EtcdRevoke, c.metrics[EtcdRevoke], func() error { 199 var inErr error 200 resp, inErr = c.cli.Revoke(revokeCtx, id) 201 return inErr 202 }) 203 return 204 } 205 206 // TimeToLive delegates request to clientV3.Lease.TimeToLive 207 func (c *Client) TimeToLive( 208 ctx context.Context, lease clientV3.LeaseID, opts ...clientV3.LeaseOption, 209 ) (resp *clientV3.LeaseTimeToLiveResponse, err error) { 210 timeToLiveCtx, cancel := context.WithTimeout(ctx, etcdClientTimeoutDuration) 211 defer cancel() 212 err = retryRPC(EtcdRevoke, c.metrics[EtcdRevoke], func() error { 213 var inErr error 214 resp, inErr = c.cli.TimeToLive(timeToLiveCtx, lease, opts...) 215 return inErr 216 }) 217 return 218 } 219 220 // Watch delegates request to clientV3.Watcher.Watch 221 func (c *Client) Watch( 222 ctx context.Context, key string, role string, opts ...clientV3.OpOption, 223 ) clientV3.WatchChan { 224 watchCh := make(chan clientV3.WatchResponse, etcdWatchChBufferSize) 225 go c.WatchWithChan(ctx, watchCh, key, role, opts...) 226 return watchCh 227 } 228 229 // WatchWithChan maintains a watchCh and sends all msg from the watchCh to outCh 230 func (c *Client) WatchWithChan( 231 ctx context.Context, outCh chan<- clientV3.WatchResponse, 232 key string, role string, opts ...clientV3.OpOption, 233 ) { 234 // get initial revision from opts to avoid revision fall back 235 lastRevision := getRevisionFromWatchOpts(opts...) 236 watchCtx, cancel := context.WithCancel(ctx) 237 watchCh := c.cli.Watch(watchCtx, key, opts...) 238 239 ticker := c.clock.Ticker(etcdRequestProgressDuration) 240 lastReceivedResponseTime := c.clock.Now() 241 242 defer func() { 243 // Using closures to handle changes to the cancel function 244 ticker.Stop() 245 cancel() 246 close(outCh) 247 248 log.Info("WatchWithChan exited", zap.String("role", role)) 249 }() 250 251 for { 252 select { 253 case <-ctx.Done(): 254 return 255 case response := <-watchCh: 256 lastReceivedResponseTime = c.clock.Now() 257 if response.Err() == nil && !response.IsProgressNotify() { 258 lastRevision = response.Header.Revision 259 } 260 261 Loop: 262 // we must loop here until the response is sent to outCh 263 // or otherwise the response will be lost 264 for { 265 select { 266 case <-ctx.Done(): 267 return 268 case outCh <- response: // it may block here 269 break Loop 270 case <-ticker.C: 271 if c.clock.Since(lastReceivedResponseTime) >= etcdWatchChTimeoutDuration { 272 log.Warn("etcd client outCh blocking too long, the etcdWorker may be stuck", 273 zap.Duration("duration", c.clock.Since(lastReceivedResponseTime)), 274 zap.String("role", role)) 275 } 276 } 277 } 278 279 ticker.Reset(etcdRequestProgressDuration) 280 case <-ticker.C: 281 if err := c.RequestProgress(ctx); err != nil { 282 log.Warn("failed to request progress for etcd watcher", zap.Error(err)) 283 } 284 if c.clock.Since(lastReceivedResponseTime) >= etcdWatchChTimeoutDuration { 285 // cancel the last cancel func to reset it 286 log.Warn("etcd client watchCh blocking too long, reset the watchCh", 287 zap.Duration("duration", c.clock.Since(lastReceivedResponseTime)), 288 zap.Stack("stack"), 289 zap.String("role", role)) 290 cancel() 291 watchCtx, cancel = context.WithCancel(ctx) 292 // to avoid possible context leak warning from govet 293 _ = cancel 294 watchCh = c.cli.Watch(watchCtx, key, 295 clientV3.WithPrefix(), clientV3.WithRev(lastRevision)) 296 // we need to reset lastReceivedResponseTime after reset Watch 297 lastReceivedResponseTime = c.clock.Now() 298 } 299 } 300 } 301 } 302 303 // RequestProgress requests a progress notify response be sent in all watch channels. 304 func (c *Client) RequestProgress(ctx context.Context) error { 305 return c.cli.RequestProgress(ctx) 306 } 307 308 func isRetryableError(rpcName string) retry.IsRetryable { 309 return func(err error) bool { 310 if !cerror.IsRetryableError(err) { 311 return false 312 } 313 314 switch rpcName { 315 case EtcdRevoke: 316 if etcdErr, ok := err.(v3rpc.EtcdError); ok && etcdErr.Code() == codes.NotFound { 317 // It means the etcd lease is already expired or revoked 318 return false 319 } 320 case EtcdTxn: 321 return errorutil.IsRetryableEtcdError(err) 322 default: 323 // For other types of operation, we retry directly without handling errors 324 } 325 326 return true 327 } 328 } 329 330 // The following code is mainly copied from: 331 // https://github.com/tikv/pd/blob/master/pkg/utils/etcdutil/etcdutil.go 332 const ( 333 // defaultEtcdClientTimeout is the default timeout for etcd client. 334 defaultEtcdClientTimeout = 5 * time.Second 335 // defaultDialKeepAliveTime is the time after which client pings the server to see if transport is alive. 336 defaultDialKeepAliveTime = 10 * time.Second 337 // defaultDialKeepAliveTimeout is the time that the client waits for a response for the 338 // keep-alive probe. If the response is not received in this time, the connection is closed. 339 defaultDialKeepAliveTimeout = 3 * time.Second 340 // etcdServerOfflineTimeout is the timeout for an unhealthy etcd endpoint to be offline from healthy checker. 341 etcdServerOfflineTimeout = 30 * time.Minute 342 // etcdServerDisconnectedTimeout is the timeout for an unhealthy etcd endpoint to be disconnected from healthy checker. 343 etcdServerDisconnectedTimeout = 1 * time.Minute 344 // healthyPath is the path to check etcd health. 345 healthyPath = "health" 346 ) 347 348 func newClient(tlsConfig *tls.Config, grpcDialOption grpc.DialOption, endpoints ...string) (*clientv3.Client, error) { 349 if len(endpoints) == 0 { 350 return nil, errors.New("empty endpoints") 351 } 352 logConfig := logutil.DefaultZapLoggerConfig 353 logConfig.Level = zap.NewAtomicLevelAt(zapcore.ErrorLevel) 354 355 lgc := zap.NewProductionConfig() 356 lgc.Encoding = log.ZapEncodingName 357 client, err := clientv3.New(clientv3.Config{ 358 Endpoints: endpoints, 359 TLS: tlsConfig, 360 LogConfig: &logConfig, 361 DialTimeout: defaultEtcdClientTimeout, 362 DialKeepAliveTime: defaultDialKeepAliveTime, 363 DialKeepAliveTimeout: defaultDialKeepAliveTimeout, 364 DialOptions: []grpc.DialOption{ 365 grpcDialOption, 366 grpc.WithBlock(), 367 grpc.WithConnectParams(grpc.ConnectParams{ 368 Backoff: backoff.Config{ 369 BaseDelay: time.Second, 370 Multiplier: 1.1, 371 Jitter: 0.1, 372 MaxDelay: 3 * time.Second, 373 }, 374 MinConnectTimeout: 3 * time.Second, 375 }), 376 grpc.WithKeepaliveParams(keepalive.ClientParameters{ 377 Time: 10 * time.Second, 378 Timeout: 20 * time.Second, 379 }), 380 }, 381 }) 382 if err != nil { 383 return nil, errors.Trace(err) 384 } 385 return client, nil 386 } 387 388 // CreateRawEtcdClient creates etcd v3 client with detecting endpoints. 389 // It will check the health of endpoints periodically, and update endpoints if needed. 390 func CreateRawEtcdClient(securityConf *security.Credential, grpcDialOption grpc.DialOption, endpoints ...string) (*clientv3.Client, error) { 391 log.Info("create etcdCli", zap.Strings("endpoints", endpoints)) 392 393 tlsConfig, err := securityConf.ToTLSConfig() 394 if err != nil { 395 return nil, err 396 } 397 398 client, err := newClient(tlsConfig, grpcDialOption, endpoints...) 399 if err != nil { 400 return nil, err 401 } 402 403 tickerInterval := defaultDialKeepAliveTime 404 405 checker := &healthyChecker{ 406 tlsConfig: tlsConfig, 407 grpcDialOption: grpcDialOption, 408 } 409 eps := syncUrls(client) 410 checker.update(eps) 411 412 // Create a goroutine to check the health of etcd endpoints periodically. 413 go func(client *clientv3.Client) { 414 ticker := time.NewTicker(tickerInterval) 415 defer ticker.Stop() 416 lastAvailable := time.Now() 417 for { 418 select { 419 case <-client.Ctx().Done(): 420 log.Info("etcd client is closed, exit health check goroutine") 421 checker.Range(func(key, value interface{}) bool { 422 client := value.(*healthyClient) 423 client.Close() 424 return true 425 }) 426 return 427 case <-ticker.C: 428 usedEps := client.Endpoints() 429 healthyEps := checker.patrol(client.Ctx()) 430 if len(healthyEps) == 0 { 431 // when all endpoints are unhealthy, try to reset endpoints to update connect 432 // rather than delete them to avoid there is no any endpoint in client. 433 // Note: reset endpoints will trigger subconn closed, and then trigger reconnect. 434 // otherwise, the subconn will be retrying in grpc layer and use exponential backoff, 435 // and it cannot recover as soon as possible. 436 if time.Since(lastAvailable) > etcdServerDisconnectedTimeout { 437 log.Info("no available endpoint, try to reset endpoints", zap.Strings("lastEndpoints", usedEps)) 438 client.SetEndpoints([]string{}...) 439 client.SetEndpoints(usedEps...) 440 } 441 } else { 442 if !util.AreStringSlicesEquivalent(healthyEps, usedEps) { 443 client.SetEndpoints(healthyEps...) 444 change := fmt.Sprintf("%d->%d", len(usedEps), len(healthyEps)) 445 etcdStateGauge.WithLabelValues("endpoints").Set(float64(len(healthyEps))) 446 log.Info("update endpoints", zap.String("numChange", change), 447 zap.Strings("lastEndpoints", usedEps), zap.Strings("endpoints", client.Endpoints())) 448 } 449 lastAvailable = time.Now() 450 } 451 } 452 } 453 }(client) 454 455 // Notes: use another goroutine to update endpoints to avoid blocking health check in the first goroutine. 456 go func(client *clientv3.Client) { 457 ticker := time.NewTicker(tickerInterval) 458 defer ticker.Stop() 459 for { 460 select { 461 case <-client.Ctx().Done(): 462 log.Info("etcd client is closed, exit update endpoint goroutine") 463 return 464 case <-ticker.C: 465 eps := syncUrls(client) 466 checker.update(eps) 467 } 468 } 469 }(client) 470 471 return client, nil 472 } 473 474 type healthyClient struct { 475 *clientv3.Client 476 lastHealth time.Time 477 } 478 479 type healthyChecker struct { 480 sync.Map // map[string]*healthyClient 481 tlsConfig *tls.Config 482 grpcDialOption grpc.DialOption 483 } 484 485 func (checker *healthyChecker) patrol(ctx context.Context) []string { 486 // See https://github.com/etcd-io/etcd/blob/85b640cee793e25f3837c47200089d14a8392dc7/etcdctl/ctlv3/command/ep_command.go#L105-L145 487 var wg sync.WaitGroup 488 count := 0 489 checker.Range(func(key, value interface{}) bool { 490 count++ 491 return true 492 }) 493 hch := make(chan string, count) 494 healthyList := make([]string, 0, count) 495 checker.Range(func(key, value interface{}) bool { 496 wg.Add(1) 497 go func(key, value interface{}) { 498 defer wg.Done() 499 ep := key.(string) 500 client := value.(*healthyClient) 501 if IsHealthy(ctx, client.Client) { 502 hch <- ep 503 checker.Store(ep, &healthyClient{ 504 Client: client.Client, 505 lastHealth: time.Now(), 506 }) 507 return 508 } 509 }(key, value) 510 return true 511 }) 512 wg.Wait() 513 close(hch) 514 for h := range hch { 515 healthyList = append(healthyList, h) 516 } 517 return healthyList 518 } 519 520 func (checker *healthyChecker) update(eps []string) { 521 for _, ep := range eps { 522 // check if client exists, if not, create one, if exists, check if it's offline or disconnected. 523 if client, ok := checker.Load(ep); ok { 524 lastHealthy := client.(*healthyClient).lastHealth 525 if time.Since(lastHealthy) > etcdServerOfflineTimeout { 526 log.Info("some etcd server maybe offline", zap.String("endpoint", ep)) 527 checker.Delete(ep) 528 } 529 if time.Since(lastHealthy) > etcdServerDisconnectedTimeout { 530 // try to reset client endpoint to trigger reconnect 531 client.(*healthyClient).Client.SetEndpoints([]string{}...) 532 client.(*healthyClient).Client.SetEndpoints(ep) 533 } 534 continue 535 } 536 checker.addClient(ep, time.Now()) 537 } 538 } 539 540 func (checker *healthyChecker) addClient(ep string, lastHealth time.Time) { 541 client, err := newClient(checker.tlsConfig, checker.grpcDialOption, ep) 542 if err != nil { 543 log.Error("failed to create etcd healthy client", zap.Error(err)) 544 return 545 } 546 checker.Store(ep, &healthyClient{ 547 Client: client, 548 lastHealth: lastHealth, 549 }) 550 } 551 552 func syncUrls(client *clientv3.Client) []string { 553 // See https://github.com/etcd-io/etcd/blob/85b640cee793e25f3837c47200089d14a8392dc7/clientv3/client.go#L170-L183 554 ctx, cancel := context.WithTimeout(clientv3.WithRequireLeader(client.Ctx()), 555 etcdClientTimeoutDuration) 556 defer cancel() 557 mresp, err := client.MemberList(ctx) 558 if err != nil { 559 log.Error("failed to list members", errs.ZapError(err)) 560 return []string{} 561 } 562 var eps []string 563 for _, m := range mresp.Members { 564 if len(m.Name) != 0 && !m.IsLearner { 565 eps = append(eps, m.ClientURLs...) 566 } 567 } 568 return eps 569 } 570 571 // IsHealthy checks if the etcd is healthy. 572 func IsHealthy(ctx context.Context, client *clientv3.Client) bool { 573 timeout := etcdClientTimeoutDuration 574 ctx, cancel := context.WithTimeout(clientv3.WithRequireLeader(ctx), timeout) 575 defer cancel() 576 _, err := client.Get(ctx, healthyPath) 577 // permission denied is OK since proposal goes through consensus to get it 578 // See: https://github.com/etcd-io/etcd/blob/85b640cee793e25f3837c47200089d14a8392dc7/etcdctl/ctlv3/command/ep_command.go#L124 579 return err == nil || err == rpctypes.ErrPermissionDenied 580 }