github.com/pingcap/tiflow@v0.0.0-20240520035814-5bf52d54e205/cdcv2/capture/capture.go (about) 1 // Copyright 2023 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 capture 15 16 import ( 17 "context" 18 "database/sql" 19 "fmt" 20 "io" 21 "sync" 22 23 "github.com/google/uuid" 24 "github.com/pingcap/errors" 25 "github.com/pingcap/log" 26 "github.com/pingcap/tiflow/cdc/capture" 27 "github.com/pingcap/tiflow/cdc/controller" 28 "github.com/pingcap/tiflow/cdc/model" 29 "github.com/pingcap/tiflow/cdc/owner" 30 "github.com/pingcap/tiflow/cdc/processor/sourcemanager/sorter/factory" 31 controllerv2 "github.com/pingcap/tiflow/cdcv2/controller" 32 "github.com/pingcap/tiflow/cdcv2/metadata" 33 msql "github.com/pingcap/tiflow/cdcv2/metadata/sql" 34 ownerv2 "github.com/pingcap/tiflow/cdcv2/owner" 35 "github.com/pingcap/tiflow/pkg/config" 36 cerror "github.com/pingcap/tiflow/pkg/errors" 37 "github.com/pingcap/tiflow/pkg/etcd" 38 "github.com/pingcap/tiflow/pkg/p2p" 39 "github.com/pingcap/tiflow/pkg/upstream" 40 "github.com/pingcap/tiflow/pkg/version" 41 pd "github.com/tikv/pd/client" 42 "go.uber.org/zap" 43 "golang.org/x/sync/errgroup" 44 "golang.org/x/time/rate" 45 ) 46 47 // NewCapture returns a new Capture instance 48 func NewCapture(pdEndpoints []string, 49 etcdClient etcd.CDCEtcdClient, 50 grpcService *p2p.ServerWrapper, 51 sortEngineMangerFactory *factory.SortEngineFactory, 52 pdClient pd.Client, 53 ) capture.Capture { 54 return &captureImpl{ 55 config: config.GetGlobalServerConfig(), 56 liveness: model.LivenessCaptureAlive, 57 EtcdClient: etcdClient, 58 grpcService: grpcService, 59 cancel: func() {}, 60 pdEndpoints: pdEndpoints, 61 info: &model.CaptureInfo{}, 62 sortEngineFactory: sortEngineMangerFactory, 63 pdClient: pdClient, 64 } 65 } 66 67 type captureImpl struct { 68 // captureMu is used to protect the capture info and processorManager. 69 captureMu sync.Mutex 70 info *model.CaptureInfo 71 liveness model.Liveness 72 config *config.ServerConfig 73 74 pdClient pd.Client 75 pdEndpoints []string 76 ownerMu sync.Mutex 77 owner *ownerv2.Owner 78 controller controller.Controller 79 upstreamManager *upstream.Manager 80 81 EtcdClient etcd.CDCEtcdClient 82 83 sortEngineFactory *factory.SortEngineFactory 84 85 // MessageServer is the receiver of the messages from the other nodes. 86 // It should be recreated each time the capture is restarted. 87 MessageServer *p2p.MessageServer 88 89 // MessageRouter manages the clients to send messages to all peers. 90 MessageRouter p2p.MessageRouter 91 92 // grpcService is a wrapper that can hold a MessageServer. 93 // The instance should last for the whole life of the server, 94 // regardless of server restarting. 95 // This design is to solve the problem that grpc-go cannot gracefully 96 // unregister a service. 97 grpcService *p2p.ServerWrapper 98 99 cancel context.CancelFunc 100 101 storage *sql.DB 102 captureObservation metadata.CaptureObservation 103 controllerObserver metadata.ControllerObservation 104 } 105 106 func (c *captureImpl) Run(ctx context.Context) error { 107 defer log.Info("the capture routine has exited") 108 // Limit the frequency of reset capture to avoid frequent recreating of resources 109 rl := rate.NewLimiter(0.05, 2) 110 for { 111 select { 112 case <-ctx.Done(): 113 return nil 114 default: 115 } 116 ctx, cancel := context.WithCancel(ctx) 117 c.cancel = cancel 118 err := rl.Wait(ctx) 119 if err != nil { 120 if errors.Cause(err) == context.Canceled { 121 return nil 122 } 123 return errors.Trace(err) 124 } 125 err = c.run(ctx) 126 // if capture suicided, reset the capture and run again. 127 // if the canceled error throw, there are two possible scenarios: 128 // 1. the internal context canceled, it means some error happened in 129 // the internal, and the routine is exited, we should restart 130 // the capture. 131 // 2. the parent context canceled, it means that the caller of 132 // the capture hope the capture to exit, and this loop will return 133 // in the above `select` block. 134 // if there are some **internal** context deadline exceeded (IO/network 135 // timeout), reset the capture and run again. 136 // 137 // TODO: make sure the internal cancel should return the real error 138 // instead of context.Canceled. 139 if cerror.ErrCaptureSuicide.Equal(err) || 140 context.Canceled == errors.Cause(err) || 141 context.DeadlineExceeded == errors.Cause(err) { 142 log.Info("capture recovered", zap.String("captureID", c.info.ID)) 143 continue 144 } 145 return errors.Trace(err) 146 } 147 } 148 149 func (c *captureImpl) run(stdCtx context.Context) error { 150 err := c.reset(stdCtx) 151 if err != nil { 152 log.Error("reset capture failed", zap.Error(err)) 153 return errors.Trace(err) 154 } 155 defer func() { 156 c.Close() 157 c.grpcService.Reset(nil) 158 }() 159 160 g, stdCtx := errgroup.WithContext(stdCtx) 161 g.Go(func() error { 162 return c.MessageServer.Run(stdCtx, c.MessageRouter.GetLocalChannel()) 163 }) 164 165 g.Go(func() error { 166 return c.captureObservation.Run(stdCtx, 167 func(ctx context.Context, 168 controllerObserver metadata.ControllerObservation, 169 ) error { 170 c.controllerObserver = controllerObserver 171 ctrl := controllerv2.NewController( 172 c.upstreamManager, 173 c.info, controllerObserver, c.captureObservation) 174 c.controller = ctrl 175 return ctrl.Run(ctx) 176 }) 177 }) 178 g.Go(func() error { 179 return c.owner.Run(stdCtx) 180 }) 181 return errors.Trace(g.Wait()) 182 } 183 184 // reset the capture before run it. 185 func (c *captureImpl) reset(ctx context.Context) error { 186 c.captureMu.Lock() 187 defer c.captureMu.Unlock() 188 c.info = &model.CaptureInfo{ 189 ID: uuid.New().String(), 190 AdvertiseAddr: c.config.AdvertiseAddr, 191 Version: version.ReleaseVersion, 192 } 193 194 if c.upstreamManager != nil { 195 c.upstreamManager.Close() 196 } 197 c.upstreamManager = upstream.NewManager(ctx, upstream.CaptureTopologyCfg{ 198 CaptureInfo: c.info, 199 GCServiceID: c.EtcdClient.GetGCServiceID(), 200 SessionTTL: int64(c.config.CaptureSessionTTL), 201 }) 202 _, err := c.upstreamManager.AddDefaultUpstream(c.pdEndpoints, c.config.Security, c.pdClient, c.EtcdClient.GetEtcdClient()) 203 if err != nil { 204 return errors.Trace(err) 205 } 206 207 c.grpcService.Reset(nil) 208 209 if c.MessageRouter != nil { 210 c.MessageRouter.Close() 211 c.MessageRouter = nil 212 } 213 messageServerConfig := c.config.Debug.Messages.ToMessageServerConfig() 214 c.MessageServer = p2p.NewMessageServer(c.info.ID, messageServerConfig) 215 c.grpcService.Reset(c.MessageServer) 216 217 messageClientConfig := c.config.Debug.Messages.ToMessageClientConfig() 218 219 // Puts the advertise-addr of the local node to the client config. 220 // This is for metrics purpose only, so that the receiver knows which 221 // node the connections are from. 222 advertiseAddr := c.config.AdvertiseAddr 223 messageClientConfig.AdvertisedAddr = advertiseAddr 224 225 c.MessageRouter = p2p.NewMessageRouterWithLocalClient(c.info.ID, c.config.Security, messageClientConfig) 226 227 dsnConfig, err := c.config.Debug.CDCV2.MetaStoreConfig.GenDSN() 228 if err != nil { 229 return errors.Trace(err) 230 } 231 c.storage, err = sql.Open("mysql", dsnConfig.FormatDSN()) 232 if err != nil { 233 return errors.Trace(err) 234 } 235 captureDB, err := msql.NewCaptureObservation(c.storage, c.info) 236 c.captureObservation = captureDB 237 if err != nil { 238 return errors.Trace(err) 239 } 240 c.owner = ownerv2.NewOwner(&c.liveness, c.upstreamManager, c.config.Debug.Scheduler, captureDB, captureDB, c.storage) 241 242 log.Info("capture initialized", zap.Any("capture", c.info)) 243 return nil 244 } 245 246 func (c *captureImpl) Close() { 247 defer c.cancel() 248 // Safety: Here we mainly want to stop the owner 249 // and ignore it if the owner does not exist or is not set. 250 o, _ := c.GetOwner() 251 if o != nil { 252 o.AsyncStop() 253 log.Info("owner closed", zap.String("captureID", c.info.ID)) 254 } 255 256 c.captureMu.Lock() 257 defer c.captureMu.Unlock() 258 259 c.grpcService.Reset(nil) 260 if c.MessageRouter != nil { 261 c.MessageRouter.Close() 262 c.MessageRouter = nil 263 } 264 log.Info("message router closed", zap.String("captureID", c.info.ID)) 265 } 266 267 // Drain does nothing for now. 268 func (c *captureImpl) Drain() <-chan struct{} { 269 done := make(chan struct{}) 270 close(done) 271 return done 272 } 273 274 func (c *captureImpl) Liveness() model.Liveness { 275 return c.liveness 276 } 277 278 func (c *captureImpl) GetOwner() (owner.Owner, error) { 279 c.ownerMu.Lock() 280 defer c.ownerMu.Unlock() 281 return c.owner, nil 282 } 283 284 func (c *captureImpl) GetController() (controller.Controller, error) { 285 c.ownerMu.Lock() 286 defer c.ownerMu.Unlock() 287 if c.owner == nil { 288 return nil, cerror.ErrNotOwner.GenWithStackByArgs() 289 } 290 return c.controller, nil 291 } 292 293 func (c *captureImpl) GetControllerCaptureInfo(ctx context.Context) (*model.CaptureInfo, error) { 294 panic("implement me") 295 } 296 297 func (c *captureImpl) IsController() bool { 298 c.captureMu.Lock() 299 defer c.captureMu.Unlock() 300 return c.controller != nil 301 } 302 303 func (c *captureImpl) Info() (model.CaptureInfo, error) { 304 c.captureMu.Lock() 305 defer c.captureMu.Unlock() 306 // when c.reset has not been called yet, c.info is nil. 307 if c.info != nil { 308 return *c.info, nil 309 } 310 return model.CaptureInfo{}, cerror.ErrCaptureNotInitialized.GenWithStackByArgs() 311 } 312 313 func (c *captureImpl) StatusProvider() owner.StatusProvider { 314 c.ownerMu.Lock() 315 defer c.ownerMu.Unlock() 316 if c.owner == nil { 317 return nil 318 } 319 panic("implement me") 320 } 321 322 func (c *captureImpl) WriteDebugInfo(ctx context.Context, w io.Writer) { 323 wait := func(done <-chan error) { 324 var err error 325 select { 326 case <-ctx.Done(): 327 err = ctx.Err() 328 case err = <-done: 329 } 330 if err != nil { 331 log.Warn("write debug info failed", zap.Error(err)) 332 } 333 } 334 // Safety: Because we are mainly outputting information about the owner here, 335 // if the owner does not exist or is not set, the information will not be output. 336 o, _ := c.GetOwner() 337 if o != nil { 338 doneOwner := make(chan error, 1) 339 fmt.Fprintf(w, "\n\n*** owner info ***:\n\n") 340 o.WriteDebugInfo(w, doneOwner) 341 // wait the debug info printed 342 wait(doneOwner) 343 } 344 345 doneM := make(chan error, 1) 346 // wait the debug info printed 347 wait(doneM) 348 } 349 350 func (c *captureImpl) GetUpstreamManager() (*upstream.Manager, error) { 351 if c.upstreamManager == nil { 352 return nil, cerror.ErrUpstreamManagerNotReady 353 } 354 return c.upstreamManager, nil 355 } 356 357 func (c *captureImpl) GetEtcdClient() etcd.CDCEtcdClient { 358 return c.EtcdClient 359 } 360 361 func (c *captureImpl) IsReady() bool { 362 return true 363 } 364 365 func (c *captureImpl) GetUpstreamInfo(ctx context.Context, 366 id model.UpstreamID, 367 s string, 368 ) (*model.UpstreamInfo, error) { 369 panic("implement me") 370 }