github.com/pingcap/tiflow@v0.0.0-20240520035814-5bf52d54e205/cdc/server/server.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 server 15 16 import ( 17 "context" 18 "fmt" 19 "net" 20 "net/http" 21 "os" 22 "path/filepath" 23 "time" 24 25 "github.com/dustin/go-humanize" 26 "github.com/gin-gonic/gin" 27 "github.com/pingcap/errors" 28 "github.com/pingcap/kvproto/pkg/diagnosticspb" 29 "github.com/pingcap/log" 30 "github.com/pingcap/sysutil" 31 "github.com/pingcap/tidb/pkg/util/gctuner" 32 "github.com/pingcap/tiflow/cdc" 33 "github.com/pingcap/tiflow/cdc/capture" 34 "github.com/pingcap/tiflow/cdc/processor/sourcemanager/sorter/factory" 35 capturev2 "github.com/pingcap/tiflow/cdcv2/capture" 36 "github.com/pingcap/tiflow/pkg/config" 37 cerror "github.com/pingcap/tiflow/pkg/errors" 38 "github.com/pingcap/tiflow/pkg/etcd" 39 "github.com/pingcap/tiflow/pkg/fsutil" 40 clogutil "github.com/pingcap/tiflow/pkg/logutil" 41 "github.com/pingcap/tiflow/pkg/p2p" 42 "github.com/pingcap/tiflow/pkg/pdutil" 43 "github.com/pingcap/tiflow/pkg/tcpserver" 44 p2pProto "github.com/pingcap/tiflow/proto/p2p" 45 pd "github.com/tikv/pd/client" 46 "go.uber.org/zap" 47 "golang.org/x/net/netutil" 48 "golang.org/x/sync/errgroup" 49 "google.golang.org/grpc" 50 "google.golang.org/grpc/backoff" 51 ) 52 53 const ( 54 defaultDataDir = "/tmp/cdc_data" 55 // dataDirThreshold is used to warn if the free space of the specified data-dir is lower than it, unit is GB 56 dataDirThreshold = 500 57 // maxHTTPConnection is used to limit the max concurrent connections of http server. 58 maxHTTPConnection = 1000 59 // httpConnectionTimeout is used to limit a connection max alive time of http server. 60 httpConnectionTimeout = 10 * time.Minute 61 // maxGcTunerMemory is used to limit the max memory usage of cdc server. if the memory is larger than it, gc tuner will be disabled 62 maxGcTunerMemory = 512 * 1024 * 1024 * 1024 63 ) 64 65 // Server is the interface for the TiCDC server 66 type Server interface { 67 // Run runs the server. 68 Run(ctx context.Context) error 69 // Close closes the server. 70 Close() 71 // Drain removes tables in the current TiCDC instance. 72 // It's part of graceful shutdown, should be called before Close. 73 Drain() <-chan struct{} 74 } 75 76 // server implement the TiCDC Server interface 77 // TODO: we need to make server more unit testable and add more test cases. 78 // Especially we need to decouple the HTTPServer out of server. 79 type server struct { 80 capture capture.Capture 81 tcpServer tcpserver.TCPServer 82 grpcService *p2p.ServerWrapper 83 diagnosticsService *sysutil.DiagnosticsServer 84 statusServer *http.Server 85 etcdClient etcd.CDCEtcdClient 86 // pdClient is the default upstream PD client. 87 // The PD acts as a metadata management service for TiCDC. 88 pdClient pd.Client 89 pdAPIClient pdutil.PDAPIClient 90 pdEndpoints []string 91 sortEngineFactory *factory.SortEngineFactory 92 } 93 94 // New creates a server instance. 95 func New(pdEndpoints []string) (*server, error) { 96 conf := config.GetGlobalServerConfig() 97 98 // This is to make communication between nodes possible. 99 // In other words, the nodes have to trust each other. 100 if len(conf.Security.CertAllowedCN) != 0 { 101 err := conf.Security.AddSelfCommonName() 102 if err != nil { 103 return nil, errors.Trace(err) 104 } 105 } 106 107 // tcpServer is the unified frontend of the CDC server that serves 108 // both RESTful APIs and gRPC APIs. 109 // Note that we pass the TLS config to the tcpServer, so there is no need to 110 // configure TLS elsewhere. 111 tcpServer, err := tcpserver.NewTCPServer(conf.Addr, conf.Security) 112 if err != nil { 113 return nil, errors.Trace(err) 114 } 115 116 debugConfig := config.GetGlobalServerConfig().Debug 117 s := &server{ 118 pdEndpoints: pdEndpoints, 119 grpcService: p2p.NewServerWrapper(debugConfig.Messages.ToMessageServerConfig()), 120 diagnosticsService: sysutil.NewDiagnosticsServer(conf.LogFile), 121 tcpServer: tcpServer, 122 } 123 124 log.Info("CDC server created", 125 zap.Strings("pd", pdEndpoints), zap.Stringer("config", conf)) 126 127 return s, nil 128 } 129 130 func (s *server) prepare(ctx context.Context) error { 131 conf := config.GetGlobalServerConfig() 132 grpcTLSOption, err := conf.Security.ToGRPCDialOption() 133 if err != nil { 134 return errors.Trace(err) 135 } 136 log.Info("create pd client", zap.Strings("endpoints", s.pdEndpoints)) 137 s.pdClient, err = pd.NewClientWithContext( 138 ctx, s.pdEndpoints, conf.Security.PDSecurityOption(), 139 // the default `timeout` is 3s, maybe too small if the pd is busy, 140 // set to 10s to avoid frequent timeout. 141 pd.WithCustomTimeoutOption(10*time.Second), 142 pd.WithGRPCDialOptions( 143 grpcTLSOption, 144 grpc.WithBlock(), 145 grpc.WithConnectParams(grpc.ConnectParams{ 146 Backoff: backoff.Config{ 147 BaseDelay: time.Second, 148 Multiplier: 1.1, 149 Jitter: 0.1, 150 MaxDelay: 3 * time.Second, 151 }, 152 MinConnectTimeout: 3 * time.Second, 153 }), 154 ), 155 pd.WithForwardingOption(config.EnablePDForwarding)) 156 if err != nil { 157 return errors.Trace(err) 158 } 159 s.pdAPIClient, err = pdutil.NewPDAPIClient(s.pdClient, conf.Security) 160 if err != nil { 161 return errors.Trace(err) 162 } 163 log.Info("create etcdCli", zap.Strings("endpoints", s.pdEndpoints)) 164 // we do not pass a `context` to create a the etcd client, 165 // to prevent it's cancelled when the server is closing. 166 // For example, when the non-owner node goes offline, 167 // it would resign the campaign key which was put by call `campaign`, 168 // if this is not done due to the passed context cancelled, 169 // the key will be kept for the lease TTL, which is 10 seconds, 170 // then cause the new owner cannot be elected immediately after the old owner offline. 171 // see https://github.com/etcd-io/etcd/blob/525d53bd41/client/v3/concurrency/election.go#L98 172 etcdCli, err := etcd.CreateRawEtcdClient(conf.Security, grpcTLSOption, s.pdEndpoints...) 173 if err != nil { 174 return errors.Trace(err) 175 } 176 177 cdcEtcdClient, err := etcd.NewCDCEtcdClient(ctx, etcdCli, conf.ClusterID) 178 if err != nil { 179 return errors.Trace(err) 180 } 181 s.etcdClient = cdcEtcdClient 182 183 // Collect all endpoints from pd here to make the server more robust. 184 // Because in some scenarios, the deployer may only provide one pd endpoint, 185 // this will cause the TiCDC server to fail to restart when some pd node is down. 186 allPDEndpoints, err := s.pdAPIClient.CollectMemberEndpoints(ctx) 187 if err != nil { 188 return errors.Trace(err) 189 } 190 s.pdEndpoints = append(s.pdEndpoints, allPDEndpoints...) 191 192 err = s.initDir(ctx) 193 if err != nil { 194 return errors.Trace(err) 195 } 196 197 s.createSortEngineFactory() 198 s.setMemoryLimit() 199 200 if conf.Debug.CDCV2.Enable { 201 s.capture = capturev2.NewCapture(s.pdEndpoints, cdcEtcdClient, 202 s.grpcService, s.sortEngineFactory, s.pdClient) 203 } else { 204 s.capture = capture.NewCapture(s.pdEndpoints, cdcEtcdClient, 205 s.grpcService, s.sortEngineFactory, s.pdClient) 206 } 207 return nil 208 } 209 210 func (s *server) setMemoryLimit() { 211 conf := config.GetGlobalServerConfig() 212 if conf.GcTunerMemoryThreshold > maxGcTunerMemory { 213 // If total memory is larger than 512GB, we will not set memory limit. 214 // Because the memory limit is not accurate, and it is not necessary to set memory limit. 215 log.Info("total memory is larger than 512GB, skip setting memory limit", 216 zap.Uint64("bytes", conf.GcTunerMemoryThreshold), 217 zap.String("memory", humanize.IBytes(conf.GcTunerMemoryThreshold)), 218 ) 219 return 220 } 221 if conf.GcTunerMemoryThreshold > 0 { 222 gctuner.EnableGOGCTuner.Store(true) 223 gctuner.Tuning(conf.GcTunerMemoryThreshold) 224 log.Info("enable gctuner, set memory limit", 225 zap.Uint64("bytes", conf.GcTunerMemoryThreshold), 226 zap.String("memory", humanize.IBytes(conf.GcTunerMemoryThreshold)), 227 ) 228 } 229 } 230 231 func (s *server) createSortEngineFactory() { 232 conf := config.GetGlobalServerConfig() 233 if s.sortEngineFactory != nil { 234 if err := s.sortEngineFactory.Close(); err != nil { 235 log.Error("fails to close sort engine manager", zap.Error(err)) 236 } 237 s.sortEngineFactory = nil 238 } 239 240 // Sorter dir has been set and checked when server starts. 241 // See https://github.com/pingcap/tiflow/blob/9dad09/cdc/server.go#L275 242 sortDir := config.GetGlobalServerConfig().Sorter.SortDir 243 memInBytes := conf.Sorter.CacheSizeInMB * uint64(1<<20) 244 s.sortEngineFactory = factory.NewForPebble(sortDir, memInBytes, conf.Debug.DB) 245 log.Info("sorter engine memory limit", 246 zap.Uint64("bytes", memInBytes), 247 zap.String("memory", humanize.IBytes(memInBytes)), 248 ) 249 } 250 251 // Run runs the server. 252 func (s *server) Run(serverCtx context.Context) error { 253 if err := s.prepare(serverCtx); err != nil { 254 return err 255 } 256 257 err := s.startStatusHTTP(s.tcpServer.HTTP1Listener()) 258 if err != nil { 259 return err 260 } 261 262 return s.run(serverCtx) 263 } 264 265 // startStatusHTTP starts the HTTP server. 266 // `lis` is a listener that gives us plain-text HTTP requests. 267 // TODO: can we decouple the HTTP server from the capture server? 268 func (s *server) startStatusHTTP(lis net.Listener) error { 269 // LimitListener returns a Listener that accepts at most n simultaneous 270 // connections from the provided Listener. Connections that exceed the 271 // limit will wait in a queue and no new goroutines will be created until 272 // a connection is processed. 273 // We use it here to limit the max concurrent connections of statusServer. 274 lis = netutil.LimitListener(lis, maxHTTPConnection) 275 conf := config.GetGlobalServerConfig() 276 277 logWritter := clogutil.InitGinLogWritter() 278 router := gin.New() 279 // add gin.RecoveryWithWriter() to handle unexpected panic (logging and 280 // returning status code 500) 281 router.Use(gin.RecoveryWithWriter(logWritter)) 282 // router. 283 // Register APIs. 284 cdc.RegisterRoutes(router, s.capture, registry) 285 286 // No need to configure TLS because it is already handled by `s.tcpServer`. 287 // Add ReadTimeout and WriteTimeout to avoid some abnormal connections never close. 288 s.statusServer = &http.Server{ 289 Handler: router, 290 ReadTimeout: httpConnectionTimeout, 291 WriteTimeout: httpConnectionTimeout, 292 } 293 294 go func() { 295 log.Info("http server is running", zap.String("addr", conf.Addr)) 296 err := s.statusServer.Serve(lis) 297 if err != nil && err != http.ErrServerClosed { 298 log.Error("http server error", zap.Error(cerror.WrapError(cerror.ErrServeHTTP, err))) 299 } 300 }() 301 return nil 302 } 303 304 func (s *server) upstreamPDHealthChecker(ctx context.Context) error { 305 ticker := time.NewTicker(time.Second * 3) 306 defer ticker.Stop() 307 308 for { 309 select { 310 case <-ctx.Done(): 311 return ctx.Err() 312 case <-ticker.C: 313 endpoints, err := s.pdAPIClient.CollectMemberEndpoints(ctx) 314 if err != nil { 315 log.Warn("etcd health check: cannot collect all members", zap.Error(err)) 316 continue 317 } 318 for _, endpoint := range endpoints { 319 start := time.Now() 320 ctx, cancel := context.WithTimeout(ctx, 5*time.Second) 321 if err := s.pdAPIClient.Healthy(ctx, endpoint); err != nil { 322 log.Warn("etcd health check error", 323 zap.String("endpoint", endpoint), zap.Error(err)) 324 } 325 etcdHealthCheckDuration.WithLabelValues(endpoint). 326 Observe(time.Since(start).Seconds()) 327 cancel() 328 } 329 ctx, cancel := context.WithTimeout(ctx, 5*time.Second) 330 _, err = s.etcdClient.GetEtcdClient().Unwrap().MemberList(ctx) 331 cancel() 332 if err != nil { 333 log.Warn("etcd health check error, fail to list etcd members", zap.Error(err)) 334 } 335 } 336 } 337 } 338 339 func (s *server) run(ctx context.Context) (err error) { 340 ctx, cancel := context.WithCancel(ctx) 341 defer cancel() 342 defer s.pdAPIClient.Close() 343 344 eg, egCtx := errgroup.WithContext(ctx) 345 346 eg.Go(func() error { 347 return s.upstreamPDHealthChecker(egCtx) 348 }) 349 350 eg.Go(func() error { 351 return s.tcpServer.Run(egCtx) 352 }) 353 354 grpcServer := grpc.NewServer(s.grpcService.ServerOptions()...) 355 p2pProto.RegisterCDCPeerToPeerServer(grpcServer, s.grpcService) 356 diagnosticspb.RegisterDiagnosticsServer(grpcServer, s.diagnosticsService) 357 358 eg.Go(func() error { 359 return grpcServer.Serve(s.tcpServer.GrpcListener()) 360 }) 361 eg.Go(func() error { 362 <-egCtx.Done() 363 grpcServer.Stop() 364 return nil 365 }) 366 367 eg.Go(func() error { 368 return s.capture.Run(egCtx) 369 }) 370 371 return eg.Wait() 372 } 373 374 // Drain removes tables in the current TiCDC instance. 375 // It's part of graceful shutdown, should be called before Close. 376 func (s *server) Drain() <-chan struct{} { 377 if s.capture == nil { 378 done := make(chan struct{}) 379 close(done) 380 return done 381 } 382 return s.capture.Drain() 383 } 384 385 // Close closes the server. 386 func (s *server) Close() { 387 if s.capture != nil { 388 s.capture.Close() 389 } 390 // Close the sort engine factory after capture closed to avoid 391 // puller send data to closed sort engine. 392 s.closeSortEngineFactory() 393 394 if s.statusServer != nil { 395 err := s.statusServer.Close() 396 if err != nil { 397 log.Error("close status server", zap.Error(err)) 398 } 399 s.statusServer = nil 400 } 401 if s.tcpServer != nil { 402 err := s.tcpServer.Close() 403 if err != nil { 404 log.Error("close tcp server", zap.Error(err)) 405 } 406 s.tcpServer = nil 407 } 408 409 if s.pdClient != nil { 410 s.pdClient.Close() 411 } 412 } 413 414 func (s *server) closeSortEngineFactory() { 415 start := time.Now() 416 if s.sortEngineFactory != nil { 417 if err := s.sortEngineFactory.Close(); err != nil { 418 log.Error("fails to close sort engine manager", zap.Error(err)) 419 } 420 log.Info("sort engine manager closed", zap.Duration("duration", time.Since(start))) 421 } 422 } 423 424 func (s *server) initDir(ctx context.Context) error { 425 if err := s.setUpDir(ctx); err != nil { 426 return errors.Trace(err) 427 } 428 conf := config.GetGlobalServerConfig() 429 // Ensure data dir exists and read-writable. 430 diskInfo, err := checkDir(conf.DataDir) 431 if err != nil { 432 return errors.Trace(err) 433 } 434 log.Info(fmt.Sprintf("%s is set as data-dir (%dGB available), sort-dir=%s. "+ 435 "It is recommended that the disk for data-dir at least have %dGB available space", 436 conf.DataDir, diskInfo.Avail, conf.Sorter.SortDir, dataDirThreshold)) 437 438 // Ensure sorter dir exists and read-writable. 439 _, err = checkDir(conf.Sorter.SortDir) 440 if err != nil { 441 return errors.Trace(err) 442 } 443 return nil 444 } 445 446 func (s *server) setUpDir(ctx context.Context) error { 447 conf := config.GetGlobalServerConfig() 448 if conf.DataDir != "" { 449 conf.Sorter.SortDir = filepath.Join(conf.DataDir, config.DefaultSortDir) 450 config.StoreGlobalServerConfig(conf) 451 452 return nil 453 } 454 455 // data-dir will be decided by exist changefeed for backward compatibility 456 allInfo, err := s.etcdClient.GetAllChangeFeedInfo(ctx) 457 if err != nil { 458 return errors.Trace(err) 459 } 460 461 candidates := make([]string, 0, len(allInfo)) 462 for _, info := range allInfo { 463 if info.SortDir != "" { 464 candidates = append(candidates, info.SortDir) 465 } 466 } 467 468 conf.DataDir = defaultDataDir 469 best, ok := findBestDataDir(candidates) 470 if ok { 471 conf.DataDir = best 472 } 473 474 conf.Sorter.SortDir = filepath.Join(conf.DataDir, config.DefaultSortDir) 475 config.StoreGlobalServerConfig(conf) 476 return nil 477 } 478 479 func checkDir(dir string) (*fsutil.DiskInfo, error) { 480 err := os.MkdirAll(dir, 0o700) 481 if err != nil { 482 return nil, errors.Trace(err) 483 } 484 if err := fsutil.IsDirReadWritable(dir); err != nil { 485 return nil, errors.Trace(err) 486 } 487 return fsutil.GetDiskInfo(dir) 488 } 489 490 // try to find the best data dir by rules 491 // at the moment, only consider available disk space 492 func findBestDataDir(candidates []string) (result string, ok bool) { 493 var low uint64 = 0 494 495 for _, dir := range candidates { 496 info, err := checkDir(dir) 497 if err != nil { 498 log.Warn("check the availability of dir", zap.String("dir", dir), zap.Error(err)) 499 continue 500 } 501 if info.Avail > low { 502 result = dir 503 low = info.Avail 504 ok = true 505 } 506 } 507 508 if !ok && len(candidates) != 0 { 509 log.Warn("try to find directory for data-dir failed, use `/tmp/cdc_data` as data-dir", zap.Strings("candidates", candidates)) 510 } 511 512 return result, ok 513 }