github.com/pingcap/tiflow@v0.0.0-20240520035814-5bf52d54e205/cdc/owner/owner.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 owner 15 16 import ( 17 "context" 18 "io" 19 "sync" 20 "sync/atomic" 21 "time" 22 23 "github.com/pingcap/errors" 24 "github.com/pingcap/failpoint" 25 "github.com/pingcap/log" 26 "github.com/pingcap/tiflow/cdc/model" 27 "github.com/pingcap/tiflow/cdc/scheduler" 28 "github.com/pingcap/tiflow/cdc/vars" 29 "github.com/pingcap/tiflow/pkg/config" 30 cerror "github.com/pingcap/tiflow/pkg/errors" 31 "github.com/pingcap/tiflow/pkg/etcd" 32 "github.com/pingcap/tiflow/pkg/orchestrator" 33 "github.com/pingcap/tiflow/pkg/upstream" 34 "github.com/pingcap/tiflow/pkg/version" 35 "go.uber.org/zap" 36 "golang.org/x/time/rate" 37 ) 38 39 type ownerJobType int 40 41 // All OwnerJob types 42 const ( 43 ownerJobTypeRebalance ownerJobType = iota 44 ownerJobTypeScheduleTable 45 ownerJobTypeDrainCapture 46 ownerJobTypeAdminJob 47 ownerJobTypeDebugInfo 48 ownerJobTypeQuery 49 ) 50 51 // versionInconsistentLogRate represents the rate of log output when there are 52 // captures with versions different from that of the owner 53 const versionInconsistentLogRate = 1 54 55 // Export field names for pretty printing. 56 type ownerJob struct { 57 Tp ownerJobType 58 ChangefeedID model.ChangeFeedID 59 60 // for ScheduleTable only 61 TargetCaptureID model.CaptureID 62 // for ScheduleTable only 63 TableID model.TableID 64 65 // for Admin Job only 66 AdminJob *model.AdminJob 67 68 // for debug info only 69 debugInfoWriter io.Writer 70 71 // for status provider 72 query *Query 73 74 // for scheduler related jobs 75 scheduleQuery *scheduler.Query 76 77 done chan<- error 78 } 79 80 // Owner managers TiCDC cluster. 81 // 82 // The interface is thread-safe, except for Tick, it's only used by etcd worker. 83 type Owner interface { 84 EnqueueJob(adminJob model.AdminJob, done chan<- error) 85 RebalanceTables(cfID model.ChangeFeedID, done chan<- error) 86 ScheduleTable( 87 cfID model.ChangeFeedID, toCapture model.CaptureID, 88 tableID model.TableID, done chan<- error, 89 ) 90 DrainCapture(query *scheduler.Query, done chan<- error) 91 WriteDebugInfo(w io.Writer, done chan<- error) 92 Query(query *Query, done chan<- error) 93 AsyncStop() 94 UpdateChangefeedAndUpstream(ctx context.Context, 95 upstreamInfo *model.UpstreamInfo, 96 changeFeedInfo *model.ChangeFeedInfo, 97 ) error 98 UpdateChangefeed(ctx context.Context, 99 changeFeedInfo *model.ChangeFeedInfo) error 100 } 101 102 type ownerImpl struct { 103 changefeeds map[model.ChangeFeedID]*changefeed 104 captures map[model.CaptureID]*model.CaptureInfo 105 upstreamManager *upstream.Manager 106 ownerJobQueue struct { 107 sync.Mutex 108 queue []*ownerJob 109 } 110 // logLimiter controls cluster version check log output rate 111 logLimiter *rate.Limiter 112 lastTickTime time.Time 113 closed int32 114 115 // changefeedTicked specifies whether changefeeds have been ticked. 116 // NOTICE: Do not use it in a method other than tick unexpectedly, 117 // as it is not a thread-safe value. 118 changefeedTicked bool 119 120 etcdClient etcd.CDCEtcdClient 121 122 newChangefeed func( 123 id model.ChangeFeedID, 124 cfInfo *model.ChangeFeedInfo, 125 cfStatus *model.ChangeFeedStatus, 126 feedStateManager FeedStateManager, 127 up *upstream.Upstream, 128 cfg *config.SchedulerConfig, 129 globalVars *vars.GlobalVars, 130 ) *changefeed 131 cfg *config.SchedulerConfig 132 133 globalVars *vars.GlobalVars 134 } 135 136 var ( 137 _ orchestrator.Reactor = &ownerImpl{} 138 _ Owner = &ownerImpl{} 139 ) 140 141 // NewOwner creates a new Owner 142 func NewOwner( 143 upstreamManager *upstream.Manager, 144 cfg *config.SchedulerConfig, 145 globalVars *vars.GlobalVars, 146 ) Owner { 147 return &ownerImpl{ 148 upstreamManager: upstreamManager, 149 changefeeds: make(map[model.ChangeFeedID]*changefeed), 150 lastTickTime: time.Now(), 151 newChangefeed: NewChangefeed, 152 logLimiter: rate.NewLimiter(versionInconsistentLogRate, versionInconsistentLogRate), 153 cfg: cfg, 154 etcdClient: globalVars.EtcdClient, 155 globalVars: globalVars, 156 } 157 } 158 159 // Tick implements the Reactor interface 160 func (o *ownerImpl) Tick(stdCtx context.Context, rawState orchestrator.ReactorState) (nextState orchestrator.ReactorState, err error) { 161 failpoint.Inject("owner-run-with-error", func() { 162 failpoint.Return(nil, errors.New("owner run with injected error")) 163 }) 164 failpoint.Inject("sleep-in-owner-tick", nil) 165 state := rawState.(*orchestrator.GlobalReactorState) 166 167 o.captures = state.Captures 168 o.updateMetrics() 169 170 // handleJobs() should be called before clusterVersionConsistent(), because 171 // when there are different versions of cdc nodes in the cluster, 172 // the admin job may not be processed all the time. And http api relies on 173 // admin job, which will cause all http api unavailable. 174 o.handleJobs(stdCtx) 175 176 // Tick all changefeeds. 177 for changefeedID, changefeedState := range state.Changefeeds { 178 // check if we are the changefeed owner to handle this changefeed 179 if !o.shouldHandleChangefeed(changefeedState) { 180 continue 181 } 182 if changefeedState.Info == nil { 183 o.cleanUpChangefeed(changefeedState) 184 if cfReactor, ok := o.changefeeds[changefeedID]; ok { 185 cfReactor.isRemoved = true 186 } 187 continue 188 } 189 cfReactor, exist := o.changefeeds[changefeedID] 190 if !exist { 191 up, ok := o.upstreamManager.Get(changefeedState.Info.UpstreamID) 192 if !ok { 193 upstreamInfo := state.Upstreams[changefeedState.Info.UpstreamID] 194 up = o.upstreamManager.AddUpstream(upstreamInfo) 195 } 196 cfReactor = o.newChangefeed(changefeedID, changefeedState.Info, changefeedState.Status, 197 NewFeedStateManager(up, changefeedState), 198 up, o.cfg, o.globalVars) 199 o.changefeeds[changefeedID] = cfReactor 200 } 201 changefeedState.CheckCaptureAlive(o.globalVars.CaptureInfo.ID) 202 captures := o.getChangefeedCaptures(changefeedState, state) 203 if !preflightCheck(changefeedState, captures) { 204 continue 205 } 206 checkpointTs, minTableBarrierTs := cfReactor.Tick(stdCtx, changefeedState.Info, changefeedState.Status, captures) 207 updateStatus(changefeedState, checkpointTs, minTableBarrierTs) 208 } 209 o.changefeedTicked = true 210 211 // Cleanup changefeeds that are not in the state. 212 if len(o.changefeeds) != len(state.Changefeeds) { 213 for changefeedID, reactor := range o.changefeeds { 214 if _, exist := state.Changefeeds[changefeedID]; exist { 215 continue 216 } 217 reactor.Close(stdCtx) 218 delete(o.changefeeds, changefeedID) 219 } 220 } 221 222 // Close and cleanup all changefeeds. 223 if atomic.LoadInt32(&o.closed) != 0 { 224 for _, reactor := range o.changefeeds { 225 reactor.Close(stdCtx) 226 } 227 return state, cerror.ErrReactorFinished.GenWithStackByArgs() 228 } 229 230 if err := o.upstreamManager.Tick(stdCtx, state); err != nil { 231 return state, errors.Trace(err) 232 } 233 return state, nil 234 } 235 236 // preflightCheck makes sure that the metadata in Etcd is complete enough to run the tick. 237 // If the metadata is not complete, such as when the ChangeFeedStatus is nil, 238 // this function will reconstruct the lost metadata and skip this tick. 239 func preflightCheck(changefeed *orchestrator.ChangefeedReactorState, 240 captures map[model.CaptureID]*model.CaptureInfo, 241 ) (ok bool) { 242 ok = true 243 if changefeed.Status == nil { 244 // complete the changefeed status when it is just created. 245 changefeed.PatchStatus( 246 func(status *model.ChangeFeedStatus) (*model.ChangeFeedStatus, bool, error) { 247 if status == nil { 248 status = &model.ChangeFeedStatus{ 249 // changefeed status is nil when the changefeed has just created. 250 CheckpointTs: changefeed.Info.StartTs, 251 MinTableBarrierTs: changefeed.Info.StartTs, 252 AdminJobType: model.AdminNone, 253 } 254 return status, true, nil 255 } 256 return status, false, nil 257 }) 258 ok = false 259 } else if changefeed.Status.MinTableBarrierTs == 0 { 260 // complete the changefeed status when the TiCDC cluster is 261 // upgraded from an old version(less than v6.7.0). 262 changefeed.PatchStatus( 263 func(status *model.ChangeFeedStatus) (*model.ChangeFeedStatus, bool, error) { 264 if status != nil { 265 if status.MinTableBarrierTs == 0 { 266 status.MinTableBarrierTs = status.CheckpointTs 267 } 268 return status, true, nil 269 } 270 return status, false, nil 271 }) 272 ok = false 273 } 274 275 // clean stale capture task positions 276 for captureID := range changefeed.TaskPositions { 277 if _, exist := captures[captureID]; !exist { 278 changefeed.PatchTaskPosition(captureID, func(position *model.TaskPosition) (*model.TaskPosition, bool, error) { 279 return nil, position != nil, nil 280 }) 281 ok = false 282 } 283 } 284 if !ok { 285 log.Info("changefeed preflight check failed, will skip this tick", 286 zap.String("namespace", changefeed.ID.Namespace), 287 zap.String("changefeed", changefeed.ID.ID), 288 zap.Any("status", changefeed.Status), zap.Bool("ok", ok), 289 ) 290 } 291 292 return 293 } 294 295 func updateStatus(changefeed *orchestrator.ChangefeedReactorState, 296 checkpointTs, minTableBarrierTs model.Ts, 297 ) { 298 if checkpointTs == 0 || minTableBarrierTs == 0 { 299 return 300 } 301 changefeed.PatchStatus( 302 func(status *model.ChangeFeedStatus) (*model.ChangeFeedStatus, bool, error) { 303 changed := false 304 if status == nil { 305 return nil, changed, nil 306 } 307 if status.CheckpointTs != checkpointTs { 308 status.CheckpointTs = checkpointTs 309 changed = true 310 } 311 if status.MinTableBarrierTs != minTableBarrierTs { 312 status.MinTableBarrierTs = minTableBarrierTs 313 changed = true 314 } 315 return status, changed, nil 316 }) 317 } 318 319 // shouldHandleChangefeed returns whether the owner should handle the changefeed. 320 func (o *ownerImpl) shouldHandleChangefeed(_ *orchestrator.ChangefeedReactorState) bool { 321 return true 322 } 323 324 // getChangefeedCaptures returns the captures to run the changefeed. 325 func (o *ownerImpl) getChangefeedCaptures(_ *orchestrator.ChangefeedReactorState, 326 globalStates *orchestrator.GlobalReactorState, 327 ) map[model.CaptureID]*model.CaptureInfo { 328 return globalStates.Captures 329 } 330 331 // EnqueueJob enqueues an admin job into an internal queue, 332 // and the Owner will handle the job in the next tick 333 // `done` must be buffered to prevent blocking owner. 334 func (o *ownerImpl) EnqueueJob(adminJob model.AdminJob, done chan<- error) { 335 o.pushOwnerJob(&ownerJob{ 336 Tp: ownerJobTypeAdminJob, 337 AdminJob: &adminJob, 338 ChangefeedID: adminJob.CfID, 339 done: done, 340 }) 341 } 342 343 // RebalanceTables triggers a rebalance for the specified changefeed 344 // `done` must be buffered to prevent blocking owner. 345 func (o *ownerImpl) RebalanceTables(cfID model.ChangeFeedID, done chan<- error) { 346 o.pushOwnerJob(&ownerJob{ 347 Tp: ownerJobTypeRebalance, 348 ChangefeedID: cfID, 349 done: done, 350 }) 351 } 352 353 // ScheduleTable moves a table from a capture to another capture 354 // `done` must be buffered to prevent blocking owner. 355 func (o *ownerImpl) ScheduleTable( 356 cfID model.ChangeFeedID, toCapture model.CaptureID, tableID model.TableID, 357 done chan<- error, 358 ) { 359 o.pushOwnerJob(&ownerJob{ 360 Tp: ownerJobTypeScheduleTable, 361 ChangefeedID: cfID, 362 TargetCaptureID: toCapture, 363 TableID: tableID, 364 done: done, 365 }) 366 } 367 368 // DrainCapture removes all tables at the target capture 369 // `done` must be buffered to prevent blocking owner. 370 func (o *ownerImpl) DrainCapture(query *scheduler.Query, done chan<- error) { 371 o.pushOwnerJob(&ownerJob{ 372 Tp: ownerJobTypeDrainCapture, 373 scheduleQuery: query, 374 done: done, 375 }) 376 } 377 378 // WriteDebugInfo writes debug info into the specified http writer 379 func (o *ownerImpl) WriteDebugInfo(w io.Writer, done chan<- error) { 380 o.pushOwnerJob(&ownerJob{ 381 Tp: ownerJobTypeDebugInfo, 382 debugInfoWriter: w, 383 done: done, 384 }) 385 } 386 387 // Query queries owner internal information. 388 func (o *ownerImpl) Query(query *Query, done chan<- error) { 389 o.pushOwnerJob(&ownerJob{ 390 Tp: ownerJobTypeQuery, 391 query: query, 392 done: done, 393 }) 394 } 395 396 // AsyncStop stops the owner asynchronously 397 func (o *ownerImpl) AsyncStop() { 398 atomic.StoreInt32(&o.closed, 1) 399 // Must be called after setting closed. 400 o.cleanupOwnerJob() 401 o.cleanStaleMetrics() 402 } 403 404 func (o *ownerImpl) UpdateChangefeedAndUpstream(ctx context.Context, 405 upstreamInfo *model.UpstreamInfo, 406 changeFeedInfo *model.ChangeFeedInfo, 407 ) error { 408 return o.etcdClient.UpdateChangefeedAndUpstream(ctx, upstreamInfo, changeFeedInfo) 409 } 410 411 func (o *ownerImpl) UpdateChangefeed(ctx context.Context, 412 changeFeedInfo *model.ChangeFeedInfo, 413 ) error { 414 return o.etcdClient.SaveChangeFeedInfo(ctx, changeFeedInfo, model.ChangeFeedID{ 415 Namespace: changeFeedInfo.Namespace, 416 ID: changeFeedInfo.ID, 417 }) 418 } 419 420 func (o *ownerImpl) cleanUpChangefeed(state *orchestrator.ChangefeedReactorState) { 421 state.PatchInfo(func(info *model.ChangeFeedInfo) (*model.ChangeFeedInfo, bool, error) { 422 return nil, info != nil, nil 423 }) 424 state.PatchStatus(func(status *model.ChangeFeedStatus) (*model.ChangeFeedStatus, bool, error) { 425 return nil, status != nil, nil 426 }) 427 for captureID := range state.TaskPositions { 428 state.PatchTaskPosition(captureID, func(position *model.TaskPosition) (*model.TaskPosition, bool, error) { 429 return nil, position != nil, nil 430 }) 431 } 432 } 433 434 func (o *ownerImpl) cleanStaleMetrics() { 435 // The gauge metrics of the Owner should be reset 436 // each time a new owner is launched, in case the previous owner 437 // has crashed and has not cleaned up the stale metrics values. 438 changefeedCheckpointTsGauge.Reset() 439 changefeedCheckpointTsLagGauge.Reset() 440 changefeedResolvedTsGauge.Reset() 441 changefeedResolvedTsLagGauge.Reset() 442 changefeedStatusGauge.Reset() 443 } 444 445 func (o *ownerImpl) updateMetrics() { 446 // Keep the value of prometheus expression `rate(counter)` = 1 447 // Please also change alert rule in ticdc.rules.yml when change the expression value. 448 now := time.Now() 449 ownershipCounter.Add(float64(now.Sub(o.lastTickTime)) / float64(time.Second)) 450 o.lastTickTime = now 451 452 for cfID, cf := range o.changefeeds { 453 if cf.latestInfo != nil { 454 changefeedStatusGauge.WithLabelValues(cfID.Namespace, cfID.ID). 455 Set(float64(cf.latestInfo.State.ToInt())) 456 } 457 } 458 } 459 460 func (o *ownerImpl) clusterVersionConsistent(captures map[model.CaptureID]*model.CaptureInfo) bool { 461 versions := make(map[string]struct{}, len(captures)) 462 for _, capture := range captures { 463 versions[capture.Version] = struct{}{} 464 } 465 466 if err := version.CheckTiCDCVersion(versions); err != nil { 467 if o.logLimiter.Allow() { 468 log.Warn("TiCDC cluster versions not allowed", 469 zap.String("ownerVer", version.ReleaseVersion), 470 zap.Any("captures", captures), zap.Error(err)) 471 } 472 return false 473 } 474 return true 475 } 476 477 func (o *ownerImpl) handleDrainCaptures(ctx context.Context, query *scheduler.Query, done chan<- error) { 478 if err := o.upstreamManager.Visit(func(upstream *upstream.Upstream) error { 479 if err := version.CheckStoreVersion(ctx, upstream.PDClient, 0); err != nil { 480 return errors.Trace(err) 481 } 482 return nil 483 }); err != nil { 484 log.Info("owner handle drain capture failed, since check upstream store version failed", 485 zap.String("target", query.CaptureID), zap.Error(err)) 486 query.Resp = &model.DrainCaptureResp{CurrentTableCount: 0} 487 done <- err 488 close(done) 489 return 490 } 491 492 var ( 493 changefeedWithTableCount int 494 totalTableCount int 495 err error 496 ) 497 for _, changefeed := range o.changefeeds { 498 // Only count normal changefeed. 499 state := changefeed.latestInfo.State 500 if state != model.StateNormal { 501 log.Info("skip drain changefeed", 502 zap.String("state", string(state)), 503 zap.String("target", query.CaptureID), 504 zap.String("namespace", changefeed.id.Namespace), 505 zap.String("changefeed", changefeed.id.ID)) 506 continue 507 } 508 if changefeed.scheduler == nil { 509 // Scheduler is created lazily, it is nil before initialization. 510 log.Info("drain a changefeed without scheduler", 511 zap.String("state", string(state)), 512 zap.String("target", query.CaptureID), 513 zap.String("namespace", changefeed.id.Namespace), 514 zap.String("changefeed", changefeed.id.ID)) 515 // To prevent a changefeed being considered drained, 516 // we increase totalTableCount. 517 totalTableCount++ 518 continue 519 } 520 count, e := changefeed.scheduler.DrainCapture(query.CaptureID) 521 if e != nil { 522 err = e 523 break 524 } 525 if count > 0 { 526 changefeedWithTableCount++ 527 } 528 totalTableCount += count 529 } 530 531 query.Resp = &model.DrainCaptureResp{ 532 CurrentTableCount: totalTableCount, 533 } 534 535 if err != nil { 536 log.Info("owner handle drain capture failed", 537 zap.String("target", query.CaptureID), zap.Error(err)) 538 done <- err 539 close(done) 540 return 541 } 542 543 log.Info("owner handle drain capture", 544 zap.String("target", query.CaptureID), 545 zap.Int("changefeedWithTableCount", changefeedWithTableCount), 546 zap.Int("totalTableCount", totalTableCount)) 547 close(done) 548 } 549 550 func (o *ownerImpl) handleJobs(ctx context.Context) { 551 jobs := o.takeOwnerJobs() 552 for _, job := range jobs { 553 changefeedID := job.ChangefeedID 554 cfReactor, exist := o.changefeeds[changefeedID] 555 if !exist && (job.Tp != ownerJobTypeQuery && job.Tp != ownerJobTypeDrainCapture) { 556 log.Warn("changefeed not found when handle a job", zap.Any("job", job)) 557 job.done <- cerror.ErrChangeFeedNotExists.FastGenByArgs(job.ChangefeedID) 558 close(job.done) 559 continue 560 } 561 switch job.Tp { 562 case ownerJobTypeAdminJob: 563 cfReactor.feedStateManager.PushAdminJob(job.AdminJob) 564 case ownerJobTypeScheduleTable: 565 // Scheduler is created lazily, it is nil before initialization. 566 if cfReactor.scheduler != nil { 567 cfReactor.scheduler.MoveTable(job.TableID, job.TargetCaptureID) 568 } 569 case ownerJobTypeDrainCapture: 570 o.handleDrainCaptures(ctx, job.scheduleQuery, job.done) 571 continue // continue here to prevent close the done channel twice 572 case ownerJobTypeRebalance: 573 // Scheduler is created lazily, it is nil before initialization. 574 if cfReactor.scheduler != nil { 575 cfReactor.scheduler.Rebalance() 576 } 577 case ownerJobTypeQuery: 578 job.done <- o.handleQueries(job.query) 579 case ownerJobTypeDebugInfo: 580 // TODO: implement this function 581 } 582 close(job.done) 583 } 584 } 585 586 func (o *ownerImpl) handleQueries(query *Query) error { 587 switch query.Tp { 588 case QueryChangeFeedStatuses: 589 cfReactor, ok := o.changefeeds[query.ChangeFeedID] 590 if !ok { 591 query.Data = nil 592 return nil 593 } 594 ret := &model.ChangeFeedStatusForAPI{} 595 ret.ResolvedTs = cfReactor.resolvedTs 596 ret.CheckpointTs = cfReactor.latestStatus.CheckpointTs 597 query.Data = ret 598 case QueryChangeFeedSyncedStatus: 599 cfReactor, ok := o.changefeeds[query.ChangeFeedID] 600 if !ok { 601 query.Data = nil 602 return nil 603 } 604 ret := &model.ChangeFeedSyncedStatusForAPI{} 605 ret.LastSyncedTs = cfReactor.lastSyncedTs 606 ret.CheckpointTs = cfReactor.latestStatus.CheckpointTs 607 ret.PullerResolvedTs = cfReactor.pullerResolvedTs 608 609 if cfReactor.latestInfo == nil { 610 ret.CheckpointInterval = 0 611 ret.SyncedCheckInterval = 0 612 } else { 613 ret.CheckpointInterval = cfReactor.latestInfo.Config.SyncedStatus.CheckpointInterval 614 ret.SyncedCheckInterval = cfReactor.latestInfo.Config.SyncedStatus.SyncedCheckInterval 615 } 616 query.Data = ret 617 case QueryChangefeedInfo: 618 cfReactor, ok := o.changefeeds[query.ChangeFeedID] 619 if !ok { 620 query.Data = nil 621 return nil 622 } 623 if cfReactor.latestInfo == nil { 624 query.Data = &model.ChangeFeedInfo{} 625 } else { 626 var err error 627 query.Data, err = cfReactor.latestInfo.Clone() 628 if err != nil { 629 return errors.Trace(err) 630 } 631 } 632 case QueryAllTaskStatuses: 633 cfReactor, ok := o.changefeeds[query.ChangeFeedID] 634 if !ok { 635 return cerror.ErrChangeFeedNotExists.GenWithStackByArgs(query.ChangeFeedID) 636 } 637 638 var ret map[model.CaptureID]*model.TaskStatus 639 provider := cfReactor.GetInfoProvider() 640 if provider == nil { 641 // The scheduler has not been initialized yet. 642 return cerror.ErrChangeFeedNotExists.GenWithStackByArgs(query.ChangeFeedID) 643 } 644 645 var err error 646 ret, err = provider.GetTaskStatuses() 647 if err != nil { 648 return errors.Trace(err) 649 } 650 query.Data = ret 651 case QueryProcessors: 652 var ret []*model.ProcInfoSnap 653 for cfID, cfReactor := range o.changefeeds { 654 provider := cfReactor.GetInfoProvider() 655 if provider == nil { 656 // The scheduler has not been initialized yet. 657 continue 658 } 659 660 statuses, err := provider.GetTaskStatuses() 661 if err != nil { 662 return errors.Trace(err) 663 } 664 for captureID := range statuses { 665 ret = append(ret, &model.ProcInfoSnap{ 666 CfID: cfID, 667 CaptureID: captureID, 668 }) 669 } 670 } 671 query.Data = ret 672 case QueryCaptures: 673 var ret []*model.CaptureInfo 674 for _, captureInfo := range o.captures { 675 ret = append(ret, &model.CaptureInfo{ 676 ID: captureInfo.ID, 677 AdvertiseAddr: captureInfo.AdvertiseAddr, 678 Version: captureInfo.Version, 679 }) 680 } 681 query.Data = ret 682 case QueryHealth: 683 query.Data = o.isHealthy() 684 case QueryOwner: 685 _, exist := o.changefeeds[query.ChangeFeedID] 686 query.Data = exist 687 } 688 return nil 689 } 690 691 func (o *ownerImpl) isHealthy() bool { 692 if !o.changefeedTicked { 693 // Owner has not yet tick changefeeds, some changefeeds may be not 694 // initialized. 695 log.Warn("owner is not healthy since changefeeds are not ticked") 696 return false 697 } 698 if !o.clusterVersionConsistent(o.captures) { 699 return false 700 } 701 for _, changefeed := range o.changefeeds { 702 if changefeed.latestInfo == nil { 703 continue 704 } 705 if changefeed.latestInfo.State != model.StateNormal { 706 log.Warn("isHealthy: changefeed not normal", 707 zap.String("namespace", changefeed.id.Namespace), 708 zap.String("changefeed", changefeed.id.ID), 709 zap.Any("state", changefeed.latestInfo.State)) 710 continue 711 } 712 713 provider := changefeed.GetInfoProvider() 714 if provider == nil || !provider.IsInitialized() { 715 // The scheduler has not been initialized yet, it is considered 716 // unhealthy, because owner can not schedule tables for now. 717 log.Warn("isHealthy: changefeed is not initialized", 718 zap.String("namespace", changefeed.id.Namespace), 719 zap.String("changefeed", changefeed.id.ID)) 720 return false 721 } 722 } 723 return true 724 } 725 726 func (o *ownerImpl) takeOwnerJobs() []*ownerJob { 727 o.ownerJobQueue.Lock() 728 defer o.ownerJobQueue.Unlock() 729 730 jobs := o.ownerJobQueue.queue 731 o.ownerJobQueue.queue = nil 732 return jobs 733 } 734 735 func (o *ownerImpl) pushOwnerJob(job *ownerJob) { 736 o.ownerJobQueue.Lock() 737 defer o.ownerJobQueue.Unlock() 738 if atomic.LoadInt32(&o.closed) != 0 { 739 log.Info("reject owner job as owner has been closed", 740 zap.Int("jobType", int(job.Tp))) 741 select { 742 case job.done <- cerror.ErrOwnerNotFound.GenWithStackByArgs(): 743 default: 744 } 745 close(job.done) 746 return 747 } 748 o.ownerJobQueue.queue = append(o.ownerJobQueue.queue, job) 749 } 750 751 func (o *ownerImpl) cleanupOwnerJob() { 752 log.Info("cleanup owner jobs as owner has been closed") 753 jobs := o.takeOwnerJobs() 754 for _, job := range jobs { 755 select { 756 case job.done <- cerror.ErrOwnerNotFound.GenWithStackByArgs(): 757 default: 758 } 759 close(job.done) 760 } 761 } 762 763 // StatusProvider returns a StatusProvider 764 func (o *ownerImpl) StatusProvider() StatusProvider { 765 return &ownerStatusProvider{owner: o} 766 }