github.com/pingcap/ticdc@v0.0.0-20220526033649-485a10ef2652/cdc/capture/capture.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 capture 15 16 import ( 17 "context" 18 "fmt" 19 "io" 20 "sync" 21 "time" 22 23 "github.com/google/uuid" 24 "github.com/pingcap/errors" 25 "github.com/pingcap/failpoint" 26 "github.com/pingcap/log" 27 "github.com/pingcap/ticdc/cdc/kv" 28 "github.com/pingcap/ticdc/cdc/model" 29 "github.com/pingcap/ticdc/cdc/owner" 30 "github.com/pingcap/ticdc/cdc/processor" 31 "github.com/pingcap/ticdc/pkg/config" 32 cdcContext "github.com/pingcap/ticdc/pkg/context" 33 cerror "github.com/pingcap/ticdc/pkg/errors" 34 "github.com/pingcap/ticdc/pkg/orchestrator" 35 "github.com/pingcap/ticdc/pkg/version" 36 tidbkv "github.com/pingcap/tidb/kv" 37 pd "github.com/tikv/pd/client" 38 "go.etcd.io/etcd/clientv3/concurrency" 39 "go.etcd.io/etcd/mvcc" 40 "go.uber.org/zap" 41 "golang.org/x/time/rate" 42 ) 43 44 // Capture represents a Capture server, it monitors the changefeed information in etcd and schedules Task on it. 45 type Capture struct { 46 captureMu sync.Mutex 47 info *model.CaptureInfo 48 49 ownerMu sync.Mutex 50 owner *owner.Owner 51 processorManager *processor.Manager 52 53 // session keeps alive between the capture and etcd 54 session *concurrency.Session 55 election *concurrency.Election 56 57 pdClient pd.Client 58 kvStorage tidbkv.Storage 59 etcdClient *kv.CDCEtcdClient 60 grpcPool kv.GrpcPool 61 62 cancel context.CancelFunc 63 64 newProcessorManager func() *processor.Manager 65 newOwner func() *owner.Owner 66 } 67 68 // NewCapture returns a new Capture instance 69 func NewCapture(pdClient pd.Client, kvStorage tidbkv.Storage, etcdClient *kv.CDCEtcdClient) *Capture { 70 return &Capture{ 71 pdClient: pdClient, 72 kvStorage: kvStorage, 73 etcdClient: etcdClient, 74 cancel: func() {}, 75 76 newProcessorManager: processor.NewManager, 77 newOwner: owner.NewOwner, 78 } 79 } 80 81 func (c *Capture) reset(ctx context.Context) error { 82 c.captureMu.Lock() 83 defer c.captureMu.Unlock() 84 conf := config.GetGlobalServerConfig() 85 c.info = &model.CaptureInfo{ 86 ID: uuid.New().String(), 87 AdvertiseAddr: conf.AdvertiseAddr, 88 Version: version.ReleaseVersion, 89 } 90 c.processorManager = c.newProcessorManager() 91 if c.session != nil { 92 c.session.Close() //nolint:errcheck 93 } 94 sess, err := concurrency.NewSession(c.etcdClient.Client.Unwrap(), 95 concurrency.WithTTL(conf.CaptureSessionTTL)) 96 if err != nil { 97 return errors.Annotate(cerror.WrapError(cerror.ErrNewCaptureFailed, err), "create capture session") 98 } 99 c.session = sess 100 c.election = concurrency.NewElection(sess, kv.CaptureOwnerKey) 101 if c.grpcPool != nil { 102 c.grpcPool.Close() 103 } 104 c.grpcPool = kv.NewGrpcPoolImpl(ctx, conf.Security) 105 log.Info("init capture", zap.String("capture-id", c.info.ID), zap.String("capture-addr", c.info.AdvertiseAddr)) 106 return nil 107 } 108 109 // Run runs the capture 110 func (c *Capture) Run(ctx context.Context) error { 111 defer log.Info("the capture routine has exited") 112 // Limit the frequency of reset capture to avoid frequent recreating of resources 113 rl := rate.NewLimiter(0.05, 2) 114 for { 115 select { 116 case <-ctx.Done(): 117 return nil 118 default: 119 } 120 ctx, cancel := context.WithCancel(ctx) 121 c.cancel = cancel 122 err := rl.Wait(ctx) 123 if err != nil { 124 if errors.Cause(err) == context.Canceled { 125 return nil 126 } 127 return errors.Trace(err) 128 } 129 err = c.reset(ctx) 130 if err != nil { 131 return errors.Trace(err) 132 } 133 err = c.run(ctx) 134 // if capture suicided, reset the capture and run again. 135 // if the canceled error throw, there are two possible scenarios: 136 // 1. the internal context canceled, it means some error happened in the internal, and the routine is exited, we should restart the capture 137 // 2. the parent context canceled, it means that the caller of the capture hope the capture to exit, and this loop will return in the above `select` block 138 // TODO: make sure the internal cancel should return the real error instead of context.Canceled 139 if cerror.ErrCaptureSuicide.Equal(err) || context.Canceled == errors.Cause(err) { 140 log.Info("capture recovered", zap.String("capture-id", c.info.ID)) 141 continue 142 } 143 return errors.Trace(err) 144 } 145 } 146 147 func (c *Capture) run(stdCtx context.Context) error { 148 ctx := cdcContext.NewContext(stdCtx, &cdcContext.GlobalVars{ 149 PDClient: c.pdClient, 150 KVStorage: c.kvStorage, 151 CaptureInfo: c.info, 152 EtcdClient: c.etcdClient, 153 GrpcPool: c.grpcPool, 154 }) 155 err := c.register(ctx) 156 if err != nil { 157 return errors.Trace(err) 158 } 159 defer func() { 160 timeoutCtx, cancel := context.WithTimeout(context.Background(), 5*time.Second) 161 if err := ctx.GlobalVars().EtcdClient.DeleteCaptureInfo(timeoutCtx, c.info.ID); err != nil { 162 log.Warn("failed to delete capture info when capture exited", zap.Error(err)) 163 } 164 cancel() 165 }() 166 wg := new(sync.WaitGroup) 167 wg.Add(3) 168 var ownerErr, processorErr error 169 go func() { 170 defer wg.Done() 171 defer c.AsyncClose() 172 // when the campaignOwner returns an error, it means that the the owner throws an unrecoverable serious errors 173 // (recoverable errors are intercepted in the owner tick) 174 // so we should also stop the processor and let capture restart or exit 175 ownerErr = c.campaignOwner(ctx) 176 log.Info("the owner routine has exited", zap.Error(ownerErr)) 177 }() 178 go func() { 179 defer wg.Done() 180 defer c.AsyncClose() 181 conf := config.GetGlobalServerConfig() 182 processorFlushInterval := time.Duration(conf.ProcessorFlushInterval) 183 // when the etcd worker of processor returns an error, it means that the the processor throws an unrecoverable serious errors 184 // (recoverable errors are intercepted in the processor tick) 185 // so we should also stop the owner and let capture restart or exit 186 processorErr = c.runEtcdWorker(ctx, c.processorManager, model.NewGlobalState(), processorFlushInterval) 187 log.Info("the processor routine has exited", zap.Error(processorErr)) 188 }() 189 go func() { 190 defer wg.Done() 191 c.grpcPool.RecycleConn(ctx) 192 }() 193 wg.Wait() 194 if ownerErr != nil { 195 return errors.Annotate(ownerErr, "owner exited with error") 196 } 197 if processorErr != nil { 198 return errors.Annotate(processorErr, "processor exited with error") 199 } 200 return nil 201 } 202 203 // Info gets the capture info 204 func (c *Capture) Info() model.CaptureInfo { 205 c.captureMu.Lock() 206 defer c.captureMu.Unlock() 207 return *c.info 208 } 209 210 func (c *Capture) campaignOwner(ctx cdcContext.Context) error { 211 // In most failure cases, we don't return error directly, just run another 212 // campaign loop. We treat campaign loop as a special background routine. 213 conf := config.GetGlobalServerConfig() 214 ownerFlushInterval := time.Duration(conf.OwnerFlushInterval) 215 failpoint.Inject("ownerFlushIntervalInject", func(val failpoint.Value) { 216 ownerFlushInterval = time.Millisecond * time.Duration(val.(int)) 217 }) 218 // Limit the frequency of elections to avoid putting too much pressure on the etcd server 219 rl := rate.NewLimiter(0.05, 2) 220 for { 221 select { 222 case <-ctx.Done(): 223 return nil 224 default: 225 } 226 err := rl.Wait(ctx) 227 if err != nil { 228 if errors.Cause(err) == context.Canceled { 229 return nil 230 } 231 return errors.Trace(err) 232 } 233 // Campaign to be an owner, it blocks until it becomes the owner 234 if err := c.campaign(ctx); err != nil { 235 switch errors.Cause(err) { 236 case context.Canceled: 237 return nil 238 case mvcc.ErrCompacted: 239 // the revision we requested is compacted, just retry 240 continue 241 } 242 log.Warn("campaign owner failed", zap.Error(err)) 243 // if campaign owner failed, restart capture 244 return cerror.ErrCaptureSuicide.GenWithStackByArgs() 245 } 246 247 log.Info("campaign owner successfully", zap.String("capture-id", c.info.ID)) 248 owner := c.newOwner() 249 c.setOwner(owner) 250 err = c.runEtcdWorker(ctx, owner, model.NewGlobalState(), ownerFlushInterval) 251 c.setOwner(nil) 252 log.Info("run owner exited", zap.Error(err)) 253 // if owner exits, resign the owner key 254 if resignErr := c.resign(ctx); resignErr != nil { 255 // if resigning owner failed, return error to let capture exits 256 return errors.Annotatef(resignErr, "resign owner failed, capture: %s", c.info.ID) 257 } 258 if err != nil { 259 // for errors, return error and let capture exits or restart 260 return errors.Trace(err) 261 } 262 // if owner exits normally, continue the campaign loop and try to election owner again 263 } 264 } 265 266 func (c *Capture) runEtcdWorker(ctx cdcContext.Context, reactor orchestrator.Reactor, reactorState orchestrator.ReactorState, timerInterval time.Duration) error { 267 etcdWorker, err := orchestrator.NewEtcdWorker(ctx.GlobalVars().EtcdClient.Client, kv.EtcdKeyBase, reactor, reactorState) 268 if err != nil { 269 return errors.Trace(err) 270 } 271 if err := etcdWorker.Run(ctx, c.session, timerInterval); err != nil { 272 // We check ttl of lease instead of check `session.Done`, because 273 // `session.Done` is only notified when etcd client establish a 274 // new keepalive request, there could be a time window as long as 275 // 1/3 of session ttl that `session.Done` can't be triggered even 276 // the lease is already revoked. 277 switch { 278 case cerror.ErrEtcdSessionDone.Equal(err), 279 cerror.ErrLeaseExpired.Equal(err): 280 log.Warn("session is disconnected", zap.Error(err)) 281 return cerror.ErrCaptureSuicide.GenWithStackByArgs() 282 } 283 lease, inErr := ctx.GlobalVars().EtcdClient.Client.TimeToLive(ctx, c.session.Lease()) 284 if inErr != nil { 285 return cerror.WrapError(cerror.ErrPDEtcdAPIError, inErr) 286 } 287 if lease.TTL == int64(-1) { 288 log.Warn("session is disconnected", zap.Error(err)) 289 return cerror.ErrCaptureSuicide.GenWithStackByArgs() 290 } 291 return errors.Trace(err) 292 } 293 return nil 294 } 295 296 func (c *Capture) setOwner(owner *owner.Owner) { 297 c.ownerMu.Lock() 298 defer c.ownerMu.Unlock() 299 c.owner = owner 300 } 301 302 // OperateOwnerUnderLock operates the owner with lock 303 func (c *Capture) OperateOwnerUnderLock(fn func(*owner.Owner) error) error { 304 c.ownerMu.Lock() 305 defer c.ownerMu.Unlock() 306 if c.owner == nil { 307 return cerror.ErrNotOwner.GenWithStackByArgs() 308 } 309 return fn(c.owner) 310 } 311 312 // Campaign to be an owner 313 func (c *Capture) campaign(ctx cdcContext.Context) error { 314 failpoint.Inject("capture-campaign-compacted-error", func() { 315 failpoint.Return(errors.Trace(mvcc.ErrCompacted)) 316 }) 317 return cerror.WrapError(cerror.ErrCaptureCampaignOwner, c.election.Campaign(ctx, c.info.ID)) 318 } 319 320 // Resign lets a owner start a new election. 321 func (c *Capture) resign(ctx cdcContext.Context) error { 322 failpoint.Inject("capture-resign-failed", func() { 323 failpoint.Return(errors.New("capture resign failed")) 324 }) 325 return cerror.WrapError(cerror.ErrCaptureResignOwner, c.election.Resign(ctx)) 326 } 327 328 // register registers the capture information in etcd 329 func (c *Capture) register(ctx cdcContext.Context) error { 330 err := ctx.GlobalVars().EtcdClient.PutCaptureInfo(ctx, c.info, c.session.Lease()) 331 if err != nil { 332 return cerror.WrapError(cerror.ErrCaptureRegister, err) 333 } 334 return nil 335 } 336 337 // AsyncClose closes the capture by unregistering it from etcd 338 func (c *Capture) AsyncClose() { 339 defer c.cancel() 340 c.OperateOwnerUnderLock(func(o *owner.Owner) error { //nolint:errcheck 341 o.AsyncStop() 342 return nil 343 }) 344 c.captureMu.Lock() 345 defer c.captureMu.Unlock() 346 if c.processorManager != nil { 347 c.processorManager.AsyncClose() 348 } 349 if c.grpcPool != nil { 350 c.grpcPool.Close() 351 } 352 } 353 354 // WriteDebugInfo writes the debug info into writer. 355 func (c *Capture) WriteDebugInfo(w io.Writer) { 356 c.OperateOwnerUnderLock(func(o *owner.Owner) error { //nolint:errcheck 357 fmt.Fprintf(w, "\n\n*** owner info ***:\n\n") 358 o.WriteDebugInfo(w) 359 return nil 360 }) 361 c.captureMu.Lock() 362 defer c.captureMu.Unlock() 363 if c.processorManager != nil { 364 fmt.Fprintf(w, "\n\n*** processors info ***:\n\n") 365 c.processorManager.WriteDebugInfo(w) 366 } 367 } 368 369 // IsOwner returns whether the capture is an owner 370 func (c *Capture) IsOwner() bool { 371 return c.OperateOwnerUnderLock(func(o *owner.Owner) error { 372 return nil 373 }) == nil 374 }