github.com/pingcap/tiflow@v0.0.0-20240520035814-5bf52d54e205/engine/executor/dm/worker.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 "encoding/json" 19 "sync" 20 "time" 21 22 "github.com/BurntSushi/toml" 23 "github.com/pingcap/log" 24 dmconfig "github.com/pingcap/tiflow/dm/config" 25 "github.com/pingcap/tiflow/dm/pb" 26 "github.com/pingcap/tiflow/dm/pkg/backoff" 27 "github.com/pingcap/tiflow/dm/worker" 28 "github.com/pingcap/tiflow/engine/framework" 29 "github.com/pingcap/tiflow/engine/framework/logutil" 30 frameModel "github.com/pingcap/tiflow/engine/framework/model" 31 "github.com/pingcap/tiflow/engine/framework/registry" 32 "github.com/pingcap/tiflow/engine/jobmaster/dm" 33 "github.com/pingcap/tiflow/engine/jobmaster/dm/config" 34 "github.com/pingcap/tiflow/engine/jobmaster/dm/metadata" 35 "github.com/pingcap/tiflow/engine/jobmaster/dm/runtime" 36 dcontext "github.com/pingcap/tiflow/engine/pkg/context" 37 dmpkg "github.com/pingcap/tiflow/engine/pkg/dm" 38 "github.com/pingcap/tiflow/engine/pkg/dm/ticker" 39 "github.com/pingcap/tiflow/engine/pkg/externalresource/broker" 40 "github.com/pingcap/tiflow/engine/pkg/p2p" 41 "github.com/pingcap/tiflow/pkg/errors" 42 "go.uber.org/zap" 43 "google.golang.org/grpc/codes" 44 "google.golang.org/grpc/status" 45 ) 46 47 // RegisterWorker is used to register dm task to global registry 48 func RegisterWorker() { 49 registry.GlobalWorkerRegistry().MustRegisterWorkerType(frameModel.WorkerDMDump, newWorkerFactory(frameModel.WorkerDMDump)) 50 registry.GlobalWorkerRegistry().MustRegisterWorkerType(frameModel.WorkerDMLoad, newWorkerFactory(frameModel.WorkerDMLoad)) 51 registry.GlobalWorkerRegistry().MustRegisterWorkerType(frameModel.WorkerDMSync, newWorkerFactory(frameModel.WorkerDMSync)) 52 } 53 54 // workerFactory create dm task 55 type workerFactory struct { 56 workerType frameModel.WorkerType 57 } 58 59 // newWorkerFactory creates abstractFactory 60 func newWorkerFactory(workerType frameModel.WorkerType) *workerFactory { 61 return &workerFactory{workerType: workerType} 62 } 63 64 // DeserializeConfig implements WorkerFactory.DeserializeConfig 65 func (f workerFactory) DeserializeConfig(configBytes []byte) (registry.WorkerConfig, error) { 66 cfg := &config.TaskCfg{} 67 _, err := toml.Decode(string(configBytes), cfg) 68 return cfg, err 69 } 70 71 // NewWorkerImpl implements WorkerFactory.NewWorkerImpl 72 func (f workerFactory) NewWorkerImpl(ctx *dcontext.Context, workerID frameModel.WorkerID, masterID frameModel.MasterID, conf framework.WorkerConfig) (framework.WorkerImpl, error) { 73 cfg := conf.(*config.TaskCfg) 74 log.Info("new dm worker", zap.String(logutil.ConstFieldJobKey, masterID), zap.Stringer("worker_type", f.workerType), zap.String(logutil.ConstFieldWorkerKey, workerID), zap.Any("task_config", cfg)) 75 return newDMWorker(ctx, masterID, f.workerType, cfg) 76 } 77 78 // IsRetryableError implements WorkerFactory.IsRetryableError 79 func (f workerFactory) IsRetryableError(err error) bool { 80 return true 81 } 82 83 var ( 84 workerNormalInterval = time.Second * 30 85 workerErrorInterval = time.Second * 10 86 ) 87 88 // dmWorker implements methods for framework.WorkerImpl 89 type dmWorker struct { 90 framework.BaseWorker 91 *ticker.DefaultTicker 92 93 unitHolder unitHolder 94 messageAgent dmpkg.MessageAgent 95 autoResume *worker.AutoResumeInfo 96 97 mu sync.RWMutex 98 cfg *dmconfig.SubTaskConfig 99 storageWriteHandle broker.Handle 100 stage metadata.TaskStage 101 workerType frameModel.WorkerType 102 taskID string 103 masterID frameModel.MasterID 104 messageHandlerManager p2p.MessageHandlerManager 105 106 cfgModRevision uint64 107 needExtStorage bool 108 } 109 110 func newDMWorker( 111 ctx *dcontext.Context, 112 masterID frameModel.MasterID, 113 workerType framework.WorkerType, 114 cfg *config.TaskCfg, 115 ) (*dmWorker, error) { 116 // TODO: support config later 117 // nolint:errcheck 118 bf, _ := backoff.NewBackoff(dmconfig.DefaultBackoffFactor, dmconfig.DefaultBackoffJitter, dmconfig.DefaultBackoffMin, dmconfig.DefaultBackoffMax) 119 autoResume := &worker.AutoResumeInfo{Backoff: bf, LatestPausedTime: time.Now(), LatestResumeTime: time.Now()} 120 dmSubtaskCfg := cfg.ToDMSubTaskCfg(masterID) 121 err := dmSubtaskCfg.Adjust(true) 122 if err != nil { 123 return nil, err 124 } 125 w := &dmWorker{ 126 DefaultTicker: ticker.NewDefaultTicker(workerNormalInterval, workerErrorInterval), 127 cfg: dmSubtaskCfg, 128 stage: metadata.StageInit, 129 workerType: workerType, 130 taskID: dmSubtaskCfg.SourceID, 131 masterID: masterID, 132 unitHolder: newUnitHolderImpl(workerType, dmSubtaskCfg), 133 autoResume: autoResume, 134 cfgModRevision: cfg.ModRevision, 135 needExtStorage: cfg.NeedExtStorage, 136 } 137 w.DefaultTicker.Ticker = w 138 139 // nolint:errcheck 140 ctx.Deps().Construct(func(m p2p.MessageHandlerManager) (p2p.MessageHandlerManager, error) { 141 w.messageHandlerManager = m 142 return m, nil 143 }) 144 return w, nil 145 } 146 147 // InitImpl implements lib.WorkerImpl.InitImpl 148 func (w *dmWorker) InitImpl(ctx context.Context) error { 149 w.Logger().Info("initializing the dm worker", zap.String("task-id", w.taskID)) 150 w.messageAgent = dmpkg.NewMessageAgentImpl(w.taskID, w, w.messageHandlerManager, w.Logger()) 151 // register jobmaster client 152 if err := w.messageAgent.UpdateClient(w.masterID, w); err != nil { 153 return err 154 } 155 // for dump/load&sync mode task, we needn't to setup external storage 156 // these two tasks will directly read/write data from/to user specified external storage without executor's management 157 // for all/full mode task, the dump/load units run on a same executor, so they can access the s3 data under a same executor 158 // but for dump/load&sync mode task, import API needs a clear S3 URI without exector's prefix, 159 // what's more, dump/load units may not be executed on a same executor, 160 // so we choose to use user's own external storage and don't set up here. 161 if (w.cfg.Mode == dmconfig.ModeAll || w.cfg.Mode == dmconfig.ModeFull) && w.needExtStorage { 162 if err := w.setupStorage(ctx); err != nil { 163 return err 164 } 165 } 166 w.cfg.MetricsFactory = w.MetricFactory() 167 w.cfg.FrameworkLogger = w.Logger() 168 return w.unitHolder.Init(ctx) 169 } 170 171 // Tick implements lib.WorkerImpl.Tick 172 // Do not do heavy work in Tick, it will block the message processing. 173 func (w *dmWorker) Tick(ctx context.Context) error { 174 if err := w.checkAndAutoResume(ctx); err != nil { 175 return err 176 } 177 if err := w.updateStatusWhenStageChange(ctx); err != nil { 178 return err 179 } 180 // update unit status periodically to update metrics 181 w.unitHolder.CheckAndUpdateStatus() 182 w.discardResource4Syncer(ctx) 183 w.DoTick(ctx) 184 return w.messageAgent.Tick(ctx) 185 } 186 187 func (w *dmWorker) TickImpl(ctx context.Context) error { 188 status := w.workerStatus(ctx) 189 return w.UpdateStatus(ctx, status) 190 } 191 192 // OnMasterMessage implements lib.WorkerImpl.OnMasterMessage 193 func (w *dmWorker) OnMasterMessage(ctx context.Context, topic p2p.Topic, message p2p.MessageValue) error { 194 w.Logger().Info("dmworker.OnMasterMessage", zap.String("topic", topic), zap.Any("message", message)) 195 return nil 196 } 197 198 // CloseImpl implements lib.WorkerImpl.CloseImpl 199 func (w *dmWorker) CloseImpl(ctx context.Context) { 200 w.Logger().Info("close the dm worker", zap.String("task-id", w.taskID)) 201 202 if err := w.unitHolder.Close(ctx); err != nil { 203 w.Logger().Error("fail to close unit holder", zap.Error(err)) 204 } 205 206 if w.messageAgent == nil { 207 return 208 } 209 if err := w.messageAgent.UpdateClient(w.masterID, nil); err != nil { 210 w.Logger().Error("failed to update message client", zap.Error(err)) 211 } 212 if err := w.messageAgent.Close(ctx); err != nil { 213 w.Logger().Error("failed to close message client", zap.Error(err)) 214 } 215 } 216 217 // setupStorage opens and configs external storage 218 func (w *dmWorker) setupStorage(ctx context.Context) error { 219 rid := dm.NewDMResourceID(w.cfg.Name, w.cfg.SourceID, dm.GetDMStorageType(w.GetEnabledBucketStorage())) 220 opts := []broker.OpenStorageOption{} 221 if w.workerType == frameModel.WorkerDMDump { 222 // always use an empty storage for dumpling task 223 opts = append(opts, broker.WithCleanBeforeOpen()) 224 } 225 226 h, err := w.OpenStorage(ctx, rid, opts...) 227 for status.Code(err) == codes.Unavailable { 228 w.Logger().Info("simple retry", zap.Error(err)) 229 time.Sleep(time.Second) 230 h, err = w.OpenStorage(ctx, rid, opts...) 231 } 232 if err != nil { 233 return errors.Trace(err) 234 } 235 w.storageWriteHandle = h 236 w.cfg.ExtStorage = h.BrExternalStorage() 237 return nil 238 } 239 240 // persistStorage persists storage. 241 func (w *dmWorker) persistStorage(ctx context.Context) error { 242 return w.storageWriteHandle.Persist(ctx) 243 } 244 245 // updateStatusWhenStageChange updates status when task stage changed. 246 func (w *dmWorker) updateStatusWhenStageChange(ctx context.Context) error { 247 currentStage, _ := w.unitHolder.Stage() 248 previousStage := w.getStage() 249 if currentStage == previousStage { 250 return nil 251 } 252 w.Logger().Info("task stage changed", zap.String("task-id", w.taskID), zap.Stringer("from", previousStage), zap.Stringer("to", currentStage)) 253 w.setStage(currentStage) 254 255 status := w.workerStatus(ctx) 256 if currentStage != metadata.StageFinished { 257 w.Logger().Info("update status", zap.String("task-id", w.taskID), zap.String("status", string(status.ExtBytes))) 258 return w.UpdateStatus(ctx, status) 259 } 260 261 // now we are in StageFinished 262 // for all and full mode, resource is managed by engine, we need to discard them 263 if w.cfg.Mode == dmconfig.ModeAll || w.cfg.Mode == dmconfig.ModeFull { 264 switch w.workerType { 265 case frameModel.WorkerDMDump: 266 if err := w.persistStorage(ctx); err != nil { 267 w.Logger().Error("failed to persist storage", zap.Error(err)) 268 // persist in next tick 269 return nil 270 } 271 case frameModel.WorkerDMLoad: 272 if w.cfg.Mode != dmconfig.ModeFull { 273 break 274 } 275 if err := w.storageWriteHandle.Discard(ctx); err != nil { 276 w.Logger().Error("failed to discard storage", zap.Error(err)) 277 // discard in next tick 278 return nil 279 } 280 } 281 } 282 283 if err := w.Exit(ctx, framework.ExitReasonFinished, nil, status.ExtBytes); err != nil { 284 return err 285 } 286 287 return errors.ErrWorkerFinish.FastGenByArgs() 288 } 289 290 // workerStatus gets worker status. 291 func (w *dmWorker) workerStatus(ctx context.Context) frameModel.WorkerStatus { 292 var ( 293 stage = w.getStage() 294 code frameModel.WorkerState 295 taskStatus = &runtime.TaskStatus{ 296 Unit: w.workerType, 297 Task: w.taskID, 298 Stage: stage, 299 StageUpdatedTime: time.Now(), 300 CfgModRevision: w.cfgModRevision, 301 } 302 finalStatus any 303 ) 304 if stage == metadata.StageFinished { 305 code = frameModel.WorkerStateFinished 306 _, result := w.unitHolder.Stage() 307 status := w.unitHolder.Status(ctx) 308 // nolint:errcheck 309 statusBytes, _ := json.Marshal(status) 310 finalStatus = &runtime.FinishedTaskStatus{ 311 TaskStatus: *taskStatus, 312 Result: result, 313 Status: statusBytes, 314 } 315 } else { 316 code = frameModel.WorkerStateNormal 317 finalStatus = taskStatus 318 } 319 // nolint:errcheck 320 statusBytes, _ := json.Marshal(finalStatus) 321 return frameModel.WorkerStatus{ 322 State: code, 323 ExtBytes: statusBytes, 324 } 325 } 326 327 // getStage gets stage. 328 func (w *dmWorker) getStage() metadata.TaskStage { 329 w.mu.RLock() 330 defer w.mu.RUnlock() 331 return w.stage 332 } 333 334 func (w *dmWorker) setStage(stage metadata.TaskStage) { 335 w.mu.Lock() 336 defer w.mu.Unlock() 337 w.stage = stage 338 } 339 340 func (w *dmWorker) checkAndAutoResume(ctx context.Context) error { 341 stage, result := w.unitHolder.Stage() 342 if stage != metadata.StageError { 343 return nil 344 } 345 346 subtaskStage := &pb.SubTaskStatus{ 347 Stage: pb.Stage_Paused, 348 Result: result, 349 } 350 strategy := w.autoResume.CheckResumeSubtask(subtaskStage, dmconfig.DefaultBackoffRollback) 351 352 previousStage := w.getStage() 353 if stage != previousStage { 354 w.Logger().Error("task runs with error", zap.String("task-id", w.taskID), zap.Any("error msg", result.Errors)) 355 w.Logger().Info("got auto resume strategy", zap.String("task-id", w.taskID), zap.Stringer("strategy", strategy)) 356 } 357 358 if strategy == worker.ResumeDispatch { 359 w.Logger().Info("dispatch auto resume task", zap.String("task-id", w.taskID)) 360 err := w.unitHolder.Resume(ctx) 361 if err == nil { 362 w.autoResume.LatestResumeTime = time.Now() 363 w.autoResume.Backoff.BoundaryForward() 364 } 365 return err 366 } 367 return nil 368 } 369 370 func (w *dmWorker) discardResource4Syncer(ctx context.Context) { 371 if w.storageWriteHandle == nil || w.workerType != frameModel.WorkerDMSync || !w.needExtStorage { 372 return 373 } 374 impl, ok := w.unitHolder.(*unitHolderImpl) 375 if !ok { 376 return 377 } 378 isFresh, err := impl.unit.IsFreshTask(ctx) 379 if err != nil { 380 w.Logger().Warn("failed to check if task is fresh", zap.Error(err)) 381 return 382 } 383 if isFresh { 384 return 385 } 386 387 if err := w.storageWriteHandle.Discard(ctx); err != nil { 388 w.Logger().Error("failed to discard storage", zap.Error(err)) 389 return 390 } 391 w.needExtStorage = false 392 }