github.com/pingcap/tiflow@v0.0.0-20240520035814-5bf52d54e205/engine/jobmaster/dm/worker_manager.go (about) 1 // Copyright 2022 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 dm 15 16 import ( 17 "context" 18 "sync" 19 "time" 20 21 dmconfig "github.com/pingcap/tiflow/dm/config" 22 "github.com/pingcap/tiflow/engine/framework" 23 "github.com/pingcap/tiflow/engine/framework/logutil" 24 frameModel "github.com/pingcap/tiflow/engine/framework/model" 25 "github.com/pingcap/tiflow/engine/jobmaster/dm/config" 26 "github.com/pingcap/tiflow/engine/jobmaster/dm/metadata" 27 "github.com/pingcap/tiflow/engine/jobmaster/dm/runtime" 28 dmpkg "github.com/pingcap/tiflow/engine/pkg/dm" 29 "github.com/pingcap/tiflow/engine/pkg/dm/ticker" 30 resModel "github.com/pingcap/tiflow/engine/pkg/externalresource/model" 31 "go.uber.org/zap" 32 ) 33 34 var ( 35 // WorkerNormalInterval is check interval when no error returns in tick 36 WorkerNormalInterval = time.Second * 30 37 // WorkerErrorInterval is check interval when any error returns in tick 38 WorkerErrorInterval = time.Second * 10 39 ) 40 41 // WorkerAgent defines an interface for create worker. 42 type WorkerAgent interface { 43 // for create worker 44 CreateWorker( 45 workerType framework.WorkerType, 46 config framework.WorkerConfig, 47 opts ...framework.CreateWorkerOpt, 48 ) (frameModel.WorkerID, error) 49 } 50 51 // CheckpointAgent defines an interface for checkpoint. 52 type CheckpointAgent interface { 53 IsFresh(ctx context.Context, workerType frameModel.WorkerType, taskCfg *metadata.Task) (bool, error) 54 } 55 56 // WorkerManager checks and schedules workers. 57 type WorkerManager struct { 58 *ticker.DefaultTicker 59 60 jobID string 61 jobStore *metadata.JobStore 62 unitStore *metadata.UnitStateStore 63 workerAgent WorkerAgent 64 messageAgent dmpkg.MessageAgent 65 checkpointAgent CheckpointAgent 66 logger *zap.Logger 67 68 storageType resModel.ResourceType 69 70 // workerStatusMap record the runtime worker status 71 // taskID -> WorkerStatus 72 workerStatusMap sync.Map 73 } 74 75 // NewWorkerManager creates a new WorkerManager instance 76 func NewWorkerManager( 77 jobID string, 78 initWorkerStatus []runtime.WorkerStatus, 79 jobStore *metadata.JobStore, 80 unitStore *metadata.UnitStateStore, 81 workerAgent WorkerAgent, 82 messageAgent dmpkg.MessageAgent, 83 checkpointAgent CheckpointAgent, 84 pLogger *zap.Logger, 85 storageType resModel.ResourceType, 86 ) *WorkerManager { 87 workerManager := &WorkerManager{ 88 DefaultTicker: ticker.NewDefaultTicker(WorkerNormalInterval, WorkerErrorInterval), 89 jobID: jobID, 90 jobStore: jobStore, 91 unitStore: unitStore, 92 workerAgent: workerAgent, 93 messageAgent: messageAgent, 94 checkpointAgent: checkpointAgent, 95 logger: pLogger.With(zap.String("component", "worker_manager")), 96 storageType: storageType, 97 } 98 99 workerManager.DefaultTicker.Ticker = workerManager 100 101 for _, workerStatus := range initWorkerStatus { 102 workerManager.UpdateWorkerStatus(workerStatus) 103 } 104 return workerManager 105 } 106 107 // UpdateWorkerStatus is called when receive worker status. 108 func (wm *WorkerManager) UpdateWorkerStatus(workerStatus runtime.WorkerStatus) { 109 wm.logger.Debug("update worker status", zap.String("task_id", workerStatus.TaskID), zap.String(logutil.ConstFieldWorkerKey, workerStatus.ID)) 110 wm.workerStatusMap.Store(workerStatus.TaskID, workerStatus) 111 } 112 113 // WorkerStatus return the worker status. 114 func (wm *WorkerManager) WorkerStatus() map[string]runtime.WorkerStatus { 115 result := make(map[string]runtime.WorkerStatus) 116 wm.workerStatusMap.Range(func(key, value interface{}) bool { 117 result[key.(string)] = value.(runtime.WorkerStatus) 118 return true 119 }) 120 return result 121 } 122 123 // TickImpl remove offline workers. 124 // TickImpl stop unneeded workers. 125 // TickImpl create new workers if needed. 126 func (wm *WorkerManager) TickImpl(ctx context.Context) error { 127 wm.logger.Info("start to schedule workers") 128 wm.removeOfflineWorkers() 129 130 state, err := wm.jobStore.Get(ctx) 131 if err != nil || state.(*metadata.Job).Deleting { 132 wm.logger.Info("on job deleting", zap.Error(err)) 133 if err2 := wm.onJobDel(ctx); err2 != nil { 134 return err2 135 } 136 return err 137 } 138 job := state.(*metadata.Job) 139 140 var recordError error 141 if err := wm.stopUnneededWorkers(ctx, job); err != nil { 142 recordError = err 143 } 144 if err := wm.stopOutdatedWorkers(ctx, job); err != nil { 145 recordError = err 146 } 147 if err := wm.checkAndScheduleWorkers(ctx, job); err != nil { 148 recordError = err 149 } 150 return recordError 151 } 152 153 // remove offline worker status, usually happened when worker is offline. 154 func (wm *WorkerManager) removeOfflineWorkers() { 155 wm.workerStatusMap.Range(func(key, value interface{}) bool { 156 worker := value.(runtime.WorkerStatus) 157 if worker.IsOffline() { 158 wm.logger.Info("remove offline worker status", zap.String("task_id", worker.TaskID)) 159 wm.workerStatusMap.Delete(key) 160 } else if worker.CreateFailed() { 161 wm.logger.Info("remove failed worker status when creating", zap.String("task_id", worker.TaskID)) 162 wm.workerStatusMap.Delete(key) 163 } 164 return true 165 }) 166 } 167 168 // stop all workers, usually happened when delete jobs. 169 func (wm *WorkerManager) onJobDel(ctx context.Context) error { 170 var recordError error 171 wm.workerStatusMap.Range(func(key, value interface{}) bool { 172 workerStatus := value.(runtime.WorkerStatus) 173 if workerStatus.IsTombStone() { 174 return true 175 } 176 wm.logger.Info("stop worker", zap.String("task_id", key.(string)), zap.String(logutil.ConstFieldWorkerKey, value.(runtime.WorkerStatus).ID)) 177 if err := wm.stopWorker(ctx, key.(string), workerStatus.ID); err != nil { 178 recordError = err 179 } 180 return true 181 }) 182 return recordError 183 } 184 185 // stop unneeded workers, usually happened when update-job delete some tasks. 186 func (wm *WorkerManager) stopUnneededWorkers(ctx context.Context, job *metadata.Job) error { 187 var recordError error 188 wm.workerStatusMap.Range(func(key, value interface{}) bool { 189 taskID := key.(string) 190 if _, ok := job.Tasks[taskID]; !ok { 191 workerStatus := value.(runtime.WorkerStatus) 192 if workerStatus.IsTombStone() { 193 return true 194 } 195 wm.logger.Info("stop unneeded worker", zap.String("task_id", taskID), zap.String(logutil.ConstFieldWorkerKey, value.(runtime.WorkerStatus).ID)) 196 if err := wm.stopWorker(ctx, taskID, value.(runtime.WorkerStatus).ID); err != nil { 197 recordError = err 198 } 199 } 200 return true 201 }) 202 return recordError 203 } 204 205 // stop outdated workers, usually happened when update job cfgs. 206 func (wm *WorkerManager) stopOutdatedWorkers(ctx context.Context, job *metadata.Job) error { 207 var recordError error 208 wm.workerStatusMap.Range(func(key, value interface{}) bool { 209 taskID := key.(string) 210 workerStatus := value.(runtime.WorkerStatus) 211 task, ok := job.Tasks[taskID] 212 if !ok || task.Cfg.ModRevision == workerStatus.CfgModRevision { 213 return true 214 } 215 if workerStatus.IsTombStone() { 216 return true 217 } 218 wm.logger.Info("stop outdated worker", zap.String("task_id", taskID), zap.String(logutil.ConstFieldWorkerKey, value.(runtime.WorkerStatus).ID), 219 zap.Uint64("config_modify_revision", workerStatus.CfgModRevision), zap.Uint64("expected_config_modify_revision", task.Cfg.ModRevision)) 220 if err := wm.stopWorker(ctx, taskID, value.(runtime.WorkerStatus).ID); err != nil { 221 recordError = err 222 } 223 return true 224 }) 225 return recordError 226 } 227 228 // checkAndScheduleWorkers check whether a task need a new worker. 229 // If there is no related worker, create a new worker. 230 // If task is finished, check whether need a new worker. 231 // TODO: support incremental -> all mode switch. 232 func (wm *WorkerManager) checkAndScheduleWorkers(ctx context.Context, job *metadata.Job) error { 233 var ( 234 runningWorker runtime.WorkerStatus 235 nextUnit frameModel.WorkerType 236 isFresh bool 237 err error 238 recordError error 239 ) 240 241 // check and schedule workers 242 for taskID, persistentTask := range job.Tasks { 243 worker, ok := wm.workerStatusMap.Load(taskID) 244 if ok { 245 runningWorker = worker.(runtime.WorkerStatus) 246 nextUnit = getNextUnit(persistentTask, runningWorker) 247 isFresh = nextUnit != runningWorker.Unit 248 } else if nextUnit, isFresh, err = wm.getCurrentUnit(ctx, persistentTask); err != nil { 249 wm.logger.Error("get current unit failed", zap.String("task", taskID), zap.Error(err)) 250 recordError = err 251 continue 252 } 253 254 if ok && runningWorker.RunAsExpected() && nextUnit == runningWorker.Unit { 255 wm.logger.Debug("worker status as expected", zap.String("task_id", taskID), zap.Stringer("worker_stage", runningWorker.Stage), zap.Stringer("unit", runningWorker.Unit)) 256 continue 257 } else if !ok { 258 wm.logger.Info("task has no worker", zap.String("task_id", taskID), zap.Stringer("unit", nextUnit)) 259 } else if !runningWorker.RunAsExpected() { 260 wm.logger.Info("unexpected worker status", zap.String("task_id", taskID), zap.Stringer("worker_stage", runningWorker.Stage), zap.Stringer("unit", runningWorker.Unit), zap.Stringer("next_unit", nextUnit)) 261 } else { 262 wm.logger.Info("switch to next unit", zap.String("task_id", taskID), zap.Stringer("next_unit", nextUnit)) 263 } 264 265 var resources []resModel.ResourceID 266 taskCfg := persistentTask.Cfg 267 // first worker don't need local resource. 268 // unfresh sync unit don't need local resource.(if we need to save table checkpoint for loadTableStructureFromDump in future, we can save it before saving global checkpoint.) 269 // TODO: storage should be created/discarded in jobmaster instead of worker. 270 if workerIdxInSeq(persistentTask.Cfg.TaskMode, nextUnit) != 0 && !(nextUnit == frameModel.WorkerDMSync && !isFresh) { 271 resID := NewDMResourceID(wm.jobID, persistentTask.Cfg.Upstreams[0].SourceID, wm.storageType) 272 resources = append(resources, resID) 273 } 274 275 // FIXME: remove this after fix https://github.com/pingcap/tiflow/issues/7304 276 if nextUnit != frameModel.WorkerDMSync || isFresh { 277 taskCfg.NeedExtStorage = true 278 } 279 280 // createWorker should be an asynchronous operation 281 if err := wm.createWorker(ctx, taskID, nextUnit, taskCfg, resources...); err != nil { 282 recordError = err 283 continue 284 } 285 } 286 return recordError 287 } 288 289 var workerSeqMap = map[string][]frameModel.WorkerType{ 290 dmconfig.ModeAll: { 291 frameModel.WorkerDMDump, 292 frameModel.WorkerDMLoad, 293 frameModel.WorkerDMSync, 294 }, 295 dmconfig.ModeFull: { 296 frameModel.WorkerDMDump, 297 frameModel.WorkerDMLoad, 298 }, 299 dmconfig.ModeIncrement: { 300 frameModel.WorkerDMSync, 301 }, 302 dmconfig.ModeDump: { 303 frameModel.WorkerDMDump, 304 }, 305 dmconfig.ModeLoadSync: { 306 frameModel.WorkerDMLoad, 307 frameModel.WorkerDMSync, 308 }, 309 } 310 311 func (wm *WorkerManager) getCurrentUnit(ctx context.Context, task *metadata.Task) (frameModel.WorkerType, bool, error) { 312 workerSeq, ok := workerSeqMap[task.Cfg.TaskMode] 313 if !ok { 314 wm.logger.Panic("Unexpected TaskMode", zap.String("TaskMode", task.Cfg.TaskMode)) 315 } 316 317 for i := len(workerSeq) - 1; i >= 0; i-- { 318 isFresh, err := wm.checkpointAgent.IsFresh(ctx, workerSeq[i], task) 319 if err != nil { 320 return 0, false, err 321 } 322 if !isFresh { 323 return workerSeq[i], false, nil 324 } 325 } 326 327 return workerSeq[0], true, nil 328 } 329 330 func workerIdxInSeq(taskMode string, worker frameModel.WorkerType) int { 331 for i, w := range workerSeqMap[taskMode] { 332 if w == worker { 333 return i 334 } 335 } 336 return -1 337 } 338 339 func nextWorkerIdxAndType(taskMode string, currWorker frameModel.WorkerType) (int, frameModel.WorkerType) { 340 workerSeq := workerSeqMap[taskMode] 341 idx := workerIdxInSeq(taskMode, currWorker) 342 if idx == len(workerSeq)-1 { 343 return idx, workerSeq[idx] 344 } 345 return idx + 1, workerSeq[idx+1] 346 } 347 348 func getNextUnit(task *metadata.Task, worker runtime.WorkerStatus) frameModel.WorkerType { 349 if worker.Stage != runtime.WorkerFinished { 350 return worker.Unit 351 } 352 353 _, workerType := nextWorkerIdxAndType(task.Cfg.TaskMode, worker.Unit) 354 return workerType 355 } 356 357 func (wm *WorkerManager) createWorker( 358 ctx context.Context, 359 taskID string, 360 unit frameModel.WorkerType, 361 taskCfg *config.TaskCfg, 362 resources ...resModel.ResourceID, 363 ) error { 364 wm.logger.Info("start to create worker", zap.String("task_id", taskID), zap.Stringer("unit", unit)) 365 workerID, err := wm.workerAgent.CreateWorker(unit, taskCfg, 366 framework.CreateWorkerWithResourceRequirements(resources...)) 367 if err != nil { 368 wm.logger.Error("failed to create workers", zap.String("task_id", taskID), zap.Stringer("unit", unit), zap.Error(err)) 369 } 370 if len(workerID) != 0 { 371 // There are two mechanisms for create workers status. 372 // 1. create worker status when no error. 373 // It is possible that the worker will be created twice, so the create needs to be idempotent. 374 // 2. create worker status even if there is error. 375 // When create fails, we create it again until the next time we receive WorkerDispatchFailed/WorkerOffline event, so the create interval will be longer. 376 // We need to handle the intermediate state. 377 // We choose the second mechanism now. 378 // If a worker is created but never receives a dispatch/online/offline event(2 ticker?), we should remove it. 379 wm.UpdateWorkerStatus(runtime.InitWorkerStatus(taskID, unit, workerID)) 380 381 // create success, record unit state 382 if err := wm.unitStore.ReadModifyWrite(ctx, func(state *metadata.UnitState) error { 383 wm.logger.Debug("start to update current unit state", zap.String("task", taskID), zap.Stringer("unit", unit)) 384 status, ok := state.CurrentUnitStatus[taskID] 385 if !ok { 386 state.CurrentUnitStatus[taskID] = &metadata.UnitStatus{ 387 Unit: unit, 388 Task: taskID, 389 CreatedTime: time.Now(), 390 } 391 } else { 392 if status.Unit != unit { 393 status.CreatedTime = time.Now() 394 status.Unit = unit 395 } 396 } 397 return nil 398 }); err != nil { 399 wm.logger.Error("update current unit state failed", zap.String("task", taskID), zap.Stringer("unit", unit), zap.Error(err)) 400 return err 401 } 402 } 403 return err 404 } 405 406 func (wm *WorkerManager) stopWorker(ctx context.Context, taskID string, workerID frameModel.WorkerID) error { 407 wm.logger.Info("start to stop worker", zap.String("task_id", taskID), zap.String("worker_id", workerID)) 408 409 msg := &dmpkg.StopWorkerMessage{ 410 Task: taskID, 411 } 412 if err := wm.messageAgent.SendMessage(ctx, taskID, dmpkg.StopWorker, msg); err != nil { 413 wm.logger.Error("failed to stop worker", zap.String("task_id", taskID), zap.String("worker_id", workerID), zap.Error(err)) 414 return err 415 } 416 // workerStatus will be removed when the worker is offline. 417 return nil 418 } 419 420 func (wm *WorkerManager) removeWorkerStatusByWorkerID(workerID frameModel.WorkerID) { 421 wm.workerStatusMap.Range(func(key, value interface{}) bool { 422 if value.(runtime.WorkerStatus).ID == workerID { 423 wm.workerStatusMap.Delete(key) 424 return false 425 } 426 return true 427 }) 428 } 429 430 func (wm *WorkerManager) allTombStone() bool { 431 result := true 432 wm.workerStatusMap.Range(func(key, value interface{}) bool { 433 workerStatus := value.(runtime.WorkerStatus) 434 if !workerStatus.IsTombStone() { 435 result = false 436 return false 437 } 438 return true 439 }) 440 return result 441 }