github.com/pingcap/ticdc@v0.0.0-20220526033649-485a10ef2652/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 "fmt" 19 "io" 20 "sync" 21 "sync/atomic" 22 "time" 23 24 "github.com/pingcap/errors" 25 "github.com/pingcap/failpoint" 26 "github.com/pingcap/log" 27 "github.com/pingcap/ticdc/cdc/model" 28 cdcContext "github.com/pingcap/ticdc/pkg/context" 29 cerror "github.com/pingcap/ticdc/pkg/errors" 30 "github.com/pingcap/ticdc/pkg/orchestrator" 31 "github.com/pingcap/ticdc/pkg/version" 32 "go.uber.org/zap" 33 ) 34 35 type ownerJobType int 36 37 // All OwnerJob types 38 const ( 39 ownerJobTypeRebalance ownerJobType = iota 40 ownerJobTypeManualSchedule 41 ownerJobTypeAdminJob 42 ownerJobTypeDebugInfo 43 ) 44 45 type ownerJob struct { 46 tp ownerJobType 47 changefeedID model.ChangeFeedID 48 49 // for ManualSchedule only 50 targetCaptureID model.CaptureID 51 // for ManualSchedule only 52 tableID model.TableID 53 54 // for Admin Job only 55 adminJob *model.AdminJob 56 57 // for debug info only 58 debugInfoWriter io.Writer 59 60 done chan struct{} 61 } 62 63 // Owner manages many changefeeds 64 // All public functions are THREAD-SAFE, except for Tick, Tick is only used for etcd worker 65 type Owner struct { 66 changefeeds map[model.ChangeFeedID]*changefeed 67 68 gcManager GcManager 69 70 ownerJobQueueMu sync.Mutex 71 ownerJobQueue []*ownerJob 72 73 lastTickTime time.Time 74 75 closed int32 76 77 newChangefeed func(id model.ChangeFeedID, gcManager GcManager) *changefeed 78 } 79 80 // NewOwner creates a new Owner 81 func NewOwner() *Owner { 82 return &Owner{ 83 changefeeds: make(map[model.ChangeFeedID]*changefeed), 84 gcManager: newGCManager(), 85 lastTickTime: time.Now(), 86 newChangefeed: newChangefeed, 87 } 88 } 89 90 // NewOwner4Test creates a new Owner for test 91 func NewOwner4Test( 92 newDDLPuller func(ctx cdcContext.Context, startTs uint64) (DDLPuller, error), 93 newSink func(ctx cdcContext.Context) (AsyncSink, error)) *Owner { 94 o := NewOwner() 95 o.newChangefeed = func(id model.ChangeFeedID, gcManager GcManager) *changefeed { 96 return newChangefeed4Test(id, gcManager, newDDLPuller, newSink) 97 } 98 return o 99 } 100 101 // Tick implements the Reactor interface 102 func (o *Owner) Tick(stdCtx context.Context, rawState orchestrator.ReactorState) (nextState orchestrator.ReactorState, err error) { 103 failpoint.Inject("owner-run-with-error", func() { 104 failpoint.Return(nil, errors.New("owner run with injected error")) 105 }) 106 failpoint.Inject("sleep-in-owner-tick", nil) 107 ctx := stdCtx.(cdcContext.Context) 108 state := rawState.(*model.GlobalReactorState) 109 o.updateMetrics(state) 110 if !o.clusterVersionConsistent(state.Captures) { 111 // sleep one second to avoid printing too much log 112 time.Sleep(1 * time.Second) 113 return state, nil 114 } 115 err = o.gcManager.updateGCSafePoint(ctx, state) 116 if err != nil { 117 return nil, errors.Trace(err) 118 } 119 o.handleJobs() 120 for changefeedID, changefeedState := range state.Changefeeds { 121 if changefeedState.Info == nil { 122 o.cleanUpChangefeed(changefeedState) 123 continue 124 } 125 ctx = cdcContext.WithChangefeedVars(ctx, &cdcContext.ChangefeedVars{ 126 ID: changefeedID, 127 Info: changefeedState.Info, 128 }) 129 cfReactor, exist := o.changefeeds[changefeedID] 130 if !exist { 131 cfReactor = o.newChangefeed(changefeedID, o.gcManager) 132 o.changefeeds[changefeedID] = cfReactor 133 } 134 cfReactor.Tick(ctx, changefeedState, state.Captures) 135 } 136 if len(o.changefeeds) != len(state.Changefeeds) { 137 for changefeedID, cfReactor := range o.changefeeds { 138 if _, exist := state.Changefeeds[changefeedID]; exist { 139 continue 140 } 141 cfReactor.Close() 142 delete(o.changefeeds, changefeedID) 143 } 144 } 145 if atomic.LoadInt32(&o.closed) != 0 { 146 for _, cfReactor := range o.changefeeds { 147 cfReactor.Close() 148 } 149 return state, cerror.ErrReactorFinished.GenWithStackByArgs() 150 } 151 return state, nil 152 } 153 154 // EnqueueJob enqueues a admin job into a internal queue, and the Owner will handle the job in the next tick 155 func (o *Owner) EnqueueJob(adminJob model.AdminJob) { 156 o.pushOwnerJob(&ownerJob{ 157 tp: ownerJobTypeAdminJob, 158 adminJob: &adminJob, 159 changefeedID: adminJob.CfID, 160 done: make(chan struct{}), 161 }) 162 } 163 164 // TriggerRebalance triggers a rebalance for the specified changefeed 165 func (o *Owner) TriggerRebalance(cfID model.ChangeFeedID) { 166 o.pushOwnerJob(&ownerJob{ 167 tp: ownerJobTypeRebalance, 168 changefeedID: cfID, 169 done: make(chan struct{}), 170 }) 171 } 172 173 // ManualSchedule moves a table from a capture to another capture 174 func (o *Owner) ManualSchedule(cfID model.ChangeFeedID, toCapture model.CaptureID, tableID model.TableID) { 175 o.pushOwnerJob(&ownerJob{ 176 tp: ownerJobTypeManualSchedule, 177 changefeedID: cfID, 178 targetCaptureID: toCapture, 179 tableID: tableID, 180 done: make(chan struct{}), 181 }) 182 } 183 184 // WriteDebugInfo writes debug info into the specified http writer 185 func (o *Owner) WriteDebugInfo(w io.Writer) { 186 timeout := time.Second * 3 187 done := make(chan struct{}) 188 o.pushOwnerJob(&ownerJob{ 189 tp: ownerJobTypeDebugInfo, 190 debugInfoWriter: w, 191 done: done, 192 }) 193 // wait the debug info printed 194 select { 195 case <-done: 196 case <-time.After(timeout): 197 fmt.Fprintf(w, "failed to print debug info for owner\n") 198 } 199 } 200 201 // AsyncStop stops the owner asynchronously 202 func (o *Owner) AsyncStop() { 203 atomic.StoreInt32(&o.closed, 1) 204 } 205 206 func (o *Owner) cleanUpChangefeed(state *model.ChangefeedReactorState) { 207 state.PatchInfo(func(info *model.ChangeFeedInfo) (*model.ChangeFeedInfo, bool, error) { 208 return nil, info != nil, nil 209 }) 210 state.PatchStatus(func(status *model.ChangeFeedStatus) (*model.ChangeFeedStatus, bool, error) { 211 return nil, status != nil, nil 212 }) 213 for captureID := range state.TaskStatuses { 214 state.PatchTaskStatus(captureID, func(status *model.TaskStatus) (*model.TaskStatus, bool, error) { 215 return nil, status != nil, nil 216 }) 217 } 218 for captureID := range state.TaskPositions { 219 state.PatchTaskPosition(captureID, func(position *model.TaskPosition) (*model.TaskPosition, bool, error) { 220 return nil, position != nil, nil 221 }) 222 } 223 for captureID := range state.Workloads { 224 state.PatchTaskWorkload(captureID, func(workload model.TaskWorkload) (model.TaskWorkload, bool, error) { 225 return nil, workload != nil, nil 226 }) 227 } 228 } 229 230 func (o *Owner) updateMetrics(state *model.GlobalReactorState) { 231 // Keep the value of prometheus expression `rate(counter)` = 1 232 // Please also change alert rule in ticdc.rules.yml when change the expression value. 233 now := time.Now() 234 ownershipCounter.Add(float64(now.Sub(o.lastTickTime)) / float64(time.Second)) 235 o.lastTickTime = now 236 237 ownerMaintainTableNumGauge.Reset() 238 changefeedStatusGauge.Reset() 239 for changefeedID, changefeedState := range state.Changefeeds { 240 for captureID, captureInfo := range state.Captures { 241 taskStatus, exist := changefeedState.TaskStatuses[captureID] 242 if !exist { 243 continue 244 } 245 ownerMaintainTableNumGauge.WithLabelValues(changefeedID, captureInfo.AdvertiseAddr, maintainTableTypeTotal).Set(float64(len(taskStatus.Tables))) 246 ownerMaintainTableNumGauge.WithLabelValues(changefeedID, captureInfo.AdvertiseAddr, maintainTableTypeWip).Set(float64(len(taskStatus.Operation))) 247 if changefeedState.Info != nil { 248 changefeedStatusGauge.WithLabelValues(changefeedID).Set(float64(changefeedState.Info.State.ToInt())) 249 } 250 } 251 } 252 } 253 254 func (o *Owner) clusterVersionConsistent(captures map[model.CaptureID]*model.CaptureInfo) bool { 255 myVersion := version.ReleaseVersion 256 for _, capture := range captures { 257 if myVersion != capture.Version { 258 log.Warn("the capture version is different with the owner", zap.Reflect("capture", capture), zap.String("my-version", myVersion)) 259 return false 260 } 261 } 262 return true 263 } 264 265 func (o *Owner) handleJobs() { 266 jobs := o.takeOwnerJobs() 267 for _, job := range jobs { 268 changefeedID := job.changefeedID 269 cfReactor, exist := o.changefeeds[changefeedID] 270 if !exist { 271 log.Warn("changefeed not found when handle a job", zap.Reflect("job", job)) 272 continue 273 } 274 switch job.tp { 275 case ownerJobTypeAdminJob: 276 cfReactor.feedStateManager.PushAdminJob(job.adminJob) 277 case ownerJobTypeManualSchedule: 278 cfReactor.scheduler.MoveTable(job.tableID, job.targetCaptureID) 279 case ownerJobTypeRebalance: 280 cfReactor.scheduler.Rebalance() 281 case ownerJobTypeDebugInfo: 282 // TODO: implement this function 283 } 284 close(job.done) 285 } 286 } 287 288 func (o *Owner) takeOwnerJobs() []*ownerJob { 289 o.ownerJobQueueMu.Lock() 290 defer o.ownerJobQueueMu.Unlock() 291 292 jobs := o.ownerJobQueue 293 o.ownerJobQueue = nil 294 return jobs 295 } 296 297 func (o *Owner) pushOwnerJob(job *ownerJob) { 298 o.ownerJobQueueMu.Lock() 299 defer o.ownerJobQueueMu.Unlock() 300 o.ownerJobQueue = append(o.ownerJobQueue, job) 301 }