github.com/pingcap/tiflow@v0.0.0-20240520035814-5bf52d54e205/engine/framework/base_jobmaster.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 framework 15 16 import ( 17 "context" 18 "sync" 19 "time" 20 21 "github.com/gin-gonic/gin" 22 "github.com/pingcap/log" 23 runtime "github.com/pingcap/tiflow/engine/executor/worker" 24 "github.com/pingcap/tiflow/engine/framework/internal/eventloop" 25 frameModel "github.com/pingcap/tiflow/engine/framework/model" 26 dcontext "github.com/pingcap/tiflow/engine/pkg/context" 27 "github.com/pingcap/tiflow/engine/pkg/errctx" 28 resModel "github.com/pingcap/tiflow/engine/pkg/externalresource/model" 29 metaModel "github.com/pingcap/tiflow/engine/pkg/meta/model" 30 "github.com/pingcap/tiflow/engine/pkg/p2p" 31 "github.com/pingcap/tiflow/engine/pkg/promutil" 32 "github.com/pingcap/tiflow/pkg/errors" 33 "github.com/pingcap/tiflow/pkg/logutil" 34 "go.uber.org/zap" 35 ) 36 37 // BaseJobMaster defines an interface that can work as a job master, it embeds 38 // a Worker interface which can run on dataflow engine runtime, and also provides 39 // some utility methods. 40 type BaseJobMaster interface { 41 Worker 42 43 // MetaKVClient return business metastore kv client with job-level isolation 44 MetaKVClient() metaModel.KVClient 45 46 // MetricFactory return a promethus factory with some underlying labels(e.g. job-id, work-id) 47 MetricFactory() promutil.Factory 48 49 // Logger return a zap logger with some underlying fields(e.g. job-id) 50 Logger() *zap.Logger 51 52 // GetWorkers return the handle of all workers, from which we can get the worker status、worker id and 53 // the method for sending message to specific worker 54 GetWorkers() map[frameModel.WorkerID]WorkerHandle 55 56 // CreateWorker requires the framework to dispatch a new worker. 57 // If the worker needs to access certain file system resources, it must pass 58 // resource ID via CreateWorkerOpt 59 CreateWorker( 60 workerType frameModel.WorkerType, 61 config WorkerConfig, 62 opts ...CreateWorkerOpt, 63 ) (frameModel.WorkerID, error) 64 65 // UpdateJobStatus updates jobmaster(worker of jobmanager) status and 66 // sends a 'status updated' message to jobmanager 67 UpdateJobStatus(ctx context.Context, status frameModel.WorkerStatus) error 68 69 // CurrentEpoch return the epoch of current job 70 CurrentEpoch() frameModel.Epoch 71 72 // SendMessage sends a message of specific topic to jobmanager in a blocking or nonblocking way 73 SendMessage(ctx context.Context, topic p2p.Topic, message interface{}, nonblocking bool) error 74 75 // Exit should be called when jobmaster (in user logic) wants to exit. 76 // exitReason: ExitReasonFinished/ExitReasonCanceled/ExitReasonFailed 77 Exit(ctx context.Context, exitReason ExitReason, err error, detail []byte) error 78 79 // IsMasterReady returns whether the master has received heartbeats for all 80 // workers after a fail-over. If this is the first time the JobMaster started up, 81 // the return value is always true. 82 IsMasterReady() bool 83 84 // IsBaseJobMaster is an empty function used to prevent accidental implementation 85 // of this interface. 86 IsBaseJobMaster() 87 88 // GetEnabledBucketStorage returns whether the bucket storage is enabled and the corresponding resource type 89 // if the bucket exists 90 GetEnabledBucketStorage() (bool, resModel.ResourceType) 91 } 92 93 // BaseJobMasterExt extends BaseJobMaster with some extra methods. 94 // These methods are used by framework and is not visible to JobMasterImpl. 95 type BaseJobMasterExt interface { 96 // TriggerOpenAPIInitialize is used to trigger the initialization of openapi handler. 97 // It just delegates to the JobMasterImpl.OnOpenAPIInitialized. 98 TriggerOpenAPIInitialize(apiGroup *gin.RouterGroup) 99 100 // IsBaseJobMasterExt is an empty function used to prevent accidental implementation 101 // of this interface. 102 IsBaseJobMasterExt() 103 } 104 105 var ( 106 _ BaseJobMaster = (*DefaultBaseJobMaster)(nil) 107 _ BaseJobMasterExt = (*DefaultBaseJobMaster)(nil) 108 ) 109 110 // DefaultBaseJobMaster implements BaseJobMaster interface 111 type DefaultBaseJobMaster struct { 112 master *DefaultBaseMaster 113 worker *DefaultBaseWorker 114 impl JobMasterImpl 115 errCenter *errctx.ErrCenter 116 closeOnce sync.Once 117 } 118 119 // JobMasterImpl is the implementation of a job master of dataflow engine. 120 // the implementation struct must embed the framework.BaseJobMaster interface, this 121 // interface will be initialized by the framework. 122 type JobMasterImpl interface { 123 MasterImpl 124 125 // OnCancel is triggered when a cancel message is received. It can be 126 // triggered multiple times. 127 // TODO: when it returns error, framework should close this jobmaster. 128 OnCancel(ctx context.Context) error 129 // OnOpenAPIInitialized is called as the first callback function of the JobMasterImpl 130 // instance, the business logic should only register the OpenAPI handler in it. 131 // The implementation must not retain the apiGroup. 132 // Note: this function is called before Init(). 133 // Concurrent safety: 134 // - this function is called as the first callback function of an JobMasterImpl 135 // instance, and it's not concurrent with other callbacks. 136 OnOpenAPIInitialized(apiGroup *gin.RouterGroup) 137 138 // IsJobMasterImpl is an empty function used to prevent accidental implementation 139 // of this interface. 140 IsJobMasterImpl() 141 } 142 143 // NewBaseJobMaster creates a new DefaultBaseJobMaster instance 144 func NewBaseJobMaster( 145 ctx *dcontext.Context, 146 jobMasterImpl JobMasterImpl, 147 masterID frameModel.MasterID, 148 workerID frameModel.WorkerID, 149 tp frameModel.WorkerType, 150 workerEpoch frameModel.Epoch, 151 ) BaseJobMaster { 152 // master-worker pair: job manager <-> job master(`baseWorker` following) 153 // master-worker pair: job master(`baseMaster` following) <-> real workers 154 // `masterID` here is the ID of `JobManager` 155 // `workerID` here is the ID of Job. It remains unchanged in the job lifecycle. 156 baseMaster := NewBaseMaster( 157 ctx, &jobMasterImplAsMasterImpl{jobMasterImpl}, workerID, tp) 158 baseWorker := NewBaseWorker( 159 ctx, &jobMasterImplAsWorkerImpl{inner: jobMasterImpl}, workerID, masterID, tp, workerEpoch) 160 errCenter := errctx.NewErrCenter() 161 baseMaster.(*DefaultBaseMaster).errCenter = errCenter 162 baseWorker.(*DefaultBaseWorker).errCenter = errCenter 163 return &DefaultBaseJobMaster{ 164 master: baseMaster.(*DefaultBaseMaster), 165 worker: baseWorker.(*DefaultBaseWorker), 166 impl: jobMasterImpl, 167 errCenter: errCenter, 168 } 169 } 170 171 // MetaKVClient implements BaseJobMaster.MetaKVClient 172 func (d *DefaultBaseJobMaster) MetaKVClient() metaModel.KVClient { 173 return d.master.MetaKVClient() 174 } 175 176 // MetricFactory implements BaseJobMaster.MetricFactory 177 func (d *DefaultBaseJobMaster) MetricFactory() promutil.Factory { 178 return d.master.MetricFactory() 179 } 180 181 // Logger implements BaseJobMaster.Logger 182 func (d *DefaultBaseJobMaster) Logger() *zap.Logger { 183 return d.master.logger 184 } 185 186 // Init implements BaseJobMaster.Init 187 func (d *DefaultBaseJobMaster) Init(ctx context.Context) error { 188 // Note this context must not be held in any resident goroutine. 189 ctx, cancel := d.errCenter.WithCancelOnFirstError(ctx) 190 defer cancel() 191 192 if err := d.worker.doPreInit(ctx); err != nil { 193 return errors.Trace(err) 194 } 195 196 isFirstStartUp, err := d.master.doInit(ctx) 197 if err != nil { 198 return errors.Trace(err) 199 } 200 201 if isFirstStartUp { 202 if err := d.impl.InitImpl(ctx); err != nil { 203 // Currently we only pass error to error center when calling busniess 204 // API returns error. Business API is also known as XxxImpl. 205 d.errCenter.OnError(err) 206 return errors.Trace(err) 207 } 208 if err := d.master.markStateInMetadata(ctx, frameModel.MasterStateInit); err != nil { 209 return errors.Trace(err) 210 } 211 } else { 212 if err := d.impl.OnMasterRecovered(ctx); err != nil { 213 d.errCenter.OnError(err) 214 return errors.Trace(err) 215 } 216 } 217 218 if err := d.worker.doPostInit(ctx); err != nil { 219 return errors.Trace(err) 220 } 221 222 return nil 223 } 224 225 // Poll implements BaseJobMaster.Poll 226 func (d *DefaultBaseJobMaster) Poll(ctx context.Context) error { 227 ctx, cancel := d.errCenter.WithCancelOnFirstError(ctx) 228 defer cancel() 229 230 if err := d.master.doPoll(ctx); err != nil { 231 return errors.Trace(err) 232 } 233 if err := d.worker.doPoll(ctx); err != nil { 234 if !errors.Is(err, errors.ErrWorkerHalfExit) { 235 return errors.Trace(err) 236 } 237 return nil 238 } 239 if err := d.impl.Tick(ctx); err != nil { 240 d.errCenter.OnError(err) 241 return errors.Trace(err) 242 } 243 return nil 244 } 245 246 // GetWorkers implements BaseJobMaster.GetWorkers 247 func (d *DefaultBaseJobMaster) GetWorkers() map[frameModel.WorkerID]WorkerHandle { 248 return d.master.GetWorkers() 249 } 250 251 // Close implements BaseJobMaster.Close 252 func (d *DefaultBaseJobMaster) Close(ctx context.Context) error { 253 d.closeOnce.Do(func() { 254 d.impl.CloseImpl(ctx) 255 }) 256 257 d.master.persistMetaError() 258 d.master.doClose() 259 d.worker.doClose() 260 return nil 261 } 262 263 // Stop implements BaseJobMaster.Stop 264 func (d *DefaultBaseJobMaster) Stop(ctx context.Context) error { 265 ctx, cancel := context.WithTimeout(ctx, time.Second*5) 266 defer cancel() 267 268 d.impl.StopImpl(ctx) 269 d.master.doClose() 270 d.worker.doClose() 271 return nil 272 } 273 274 // NotifyExit implements BaseJobMaster interface 275 func (d *DefaultBaseJobMaster) NotifyExit(ctx context.Context, errIn error) (retErr error) { 276 if eventloop.IsTerminatedError(errIn) { 277 // In terminate scenario job master should call StopImpl, and we don't 278 // call NotifyExit to advance the calling of StopImpl. The drawback of 279 // this choice is job manager has to hearbeat timeout of this job master. 280 return nil 281 } 282 283 d.closeOnce.Do(func() { 284 d.impl.CloseImpl(ctx) 285 }) 286 287 startTime := time.Now() 288 defer func() { 289 duration := time.Since(startTime) 290 d.Logger().Info("job master finished exiting", 291 zap.NamedError("caused", errIn), 292 zap.Duration("duration", duration), 293 logutil.ShortError(retErr)) 294 }() 295 296 d.Logger().Info("worker start exiting", zap.NamedError("cause", errIn)) 297 return d.worker.masterClient.WaitClosed(ctx) 298 } 299 300 // CreateWorker implements BaseJobMaster.CreateWorker 301 func (d *DefaultBaseJobMaster) CreateWorker( 302 workerType frameModel.WorkerType, 303 config WorkerConfig, 304 opts ...CreateWorkerOpt, 305 ) (frameModel.WorkerID, error) { 306 return d.master.CreateWorker(workerType, config, opts...) 307 } 308 309 // UpdateStatus delegates the UpdateStatus of inner worker 310 func (d *DefaultBaseJobMaster) UpdateStatus(ctx context.Context, status frameModel.WorkerStatus) error { 311 ctx, cancel := d.errCenter.WithCancelOnFirstError(ctx) 312 defer cancel() 313 314 return d.worker.UpdateStatus(ctx, status) 315 } 316 317 // ID delegates the ID of inner worker 318 func (d *DefaultBaseJobMaster) ID() runtime.RunnableID { 319 // JobMaster is a combination of 'master' and 'worker' 320 // d.master.MasterID() == d.worker.ID() == JobID 321 return d.worker.ID() 322 } 323 324 // UpdateJobStatus implements BaseJobMaster.UpdateJobStatus 325 func (d *DefaultBaseJobMaster) UpdateJobStatus(ctx context.Context, status frameModel.WorkerStatus) error { 326 ctx, cancel := d.errCenter.WithCancelOnFirstError(ctx) 327 defer cancel() 328 329 return d.worker.UpdateStatus(ctx, status) 330 } 331 332 // CurrentEpoch implements BaseJobMaster.CurrentEpoch 333 func (d *DefaultBaseJobMaster) CurrentEpoch() frameModel.Epoch { 334 return d.master.currentEpoch.Load() 335 } 336 337 // IsBaseJobMaster implements BaseJobMaster.IsBaseJobMaster 338 func (d *DefaultBaseJobMaster) IsBaseJobMaster() { 339 } 340 341 // GetEnabledBucketStorage implements BaseJobMaster.GetEnabledBucketStorage 342 func (d *DefaultBaseJobMaster) GetEnabledBucketStorage() (bool, resModel.ResourceType) { 343 return d.worker.GetEnabledBucketStorage() 344 } 345 346 // SendMessage delegates the SendMessage or inner worker 347 func (d *DefaultBaseJobMaster) SendMessage(ctx context.Context, topic p2p.Topic, message interface{}, nonblocking bool) error { 348 ctx, cancel := d.errCenter.WithCancelOnFirstError(ctx) 349 defer cancel() 350 351 // master will use WorkerHandle to send message 352 return d.worker.SendMessage(ctx, topic, message, nonblocking) 353 } 354 355 // IsMasterReady implements BaseJobMaster.IsMasterReady 356 func (d *DefaultBaseJobMaster) IsMasterReady() bool { 357 return d.master.IsMasterReady() 358 } 359 360 // Exit implements BaseJobMaster.Exit 361 func (d *DefaultBaseJobMaster) Exit(ctx context.Context, exitReason ExitReason, err error, detail []byte) error { 362 ctx, cancel := d.errCenter.WithCancelOnFirstError(ctx) 363 defer cancel() 364 365 // Don't set error center for master to make worker.Exit work well 366 if errTmp := d.master.exitWithoutSetErrCenter(ctx, exitReason, err, detail); errTmp != nil { 367 return errTmp 368 } 369 370 return d.worker.Exit(ctx, exitReason, err, detail) 371 } 372 373 // TriggerOpenAPIInitialize implements BaseJobMasterExt.TriggerOpenAPIInitialize. 374 func (d *DefaultBaseJobMaster) TriggerOpenAPIInitialize(apiGroup *gin.RouterGroup) { 375 d.impl.OnOpenAPIInitialized(apiGroup) 376 } 377 378 // IsBaseJobMasterExt implements BaseJobMaster.IsBaseJobMasterExt. 379 func (d *DefaultBaseJobMaster) IsBaseJobMasterExt() {} 380 381 type jobMasterImplAsWorkerImpl struct { 382 inner JobMasterImpl 383 onCancelCalled bool 384 } 385 386 func (j *jobMasterImplAsWorkerImpl) InitImpl(ctx context.Context) error { 387 log.Panic("unexpected Init call") 388 return nil 389 } 390 391 func (j *jobMasterImplAsWorkerImpl) Tick(ctx context.Context) error { 392 log.Panic("unexpected Poll call") 393 return nil 394 } 395 396 func (j *jobMasterImplAsWorkerImpl) OnMasterMessage( 397 ctx context.Context, topic p2p.Topic, message interface{}, 398 ) error { 399 switch msg := message.(type) { 400 case *frameModel.StatusChangeRequest: 401 switch msg.ExpectState { 402 case frameModel.WorkerStateStopped: 403 if !j.onCancelCalled { 404 j.onCancelCalled = true 405 return j.inner.OnCancel(ctx) 406 } 407 default: 408 log.Info("Ignore status change state", zap.Int32("state", int32(msg.ExpectState))) 409 } 410 default: 411 log.Info("unsupported message", zap.Any("message", message)) 412 } 413 return nil 414 } 415 416 func (j *jobMasterImplAsWorkerImpl) CloseImpl(ctx context.Context) { 417 log.Panic("unexpected Close call") 418 } 419 420 type jobMasterImplAsMasterImpl struct { 421 inner JobMasterImpl 422 } 423 424 func (j *jobMasterImplAsMasterImpl) OnWorkerStatusUpdated(worker WorkerHandle, newStatus *frameModel.WorkerStatus) error { 425 return j.inner.OnWorkerStatusUpdated(worker, newStatus) 426 } 427 428 func (j *jobMasterImplAsMasterImpl) Tick(ctx context.Context) error { 429 log.Panic("unexpected poll call") 430 return nil 431 } 432 433 func (j *jobMasterImplAsMasterImpl) InitImpl(ctx context.Context) error { 434 log.Panic("unexpected init call") 435 return nil 436 } 437 438 func (j *jobMasterImplAsMasterImpl) OnMasterRecovered(ctx context.Context) error { 439 return j.inner.OnMasterRecovered(ctx) 440 } 441 442 func (j *jobMasterImplAsMasterImpl) OnWorkerDispatched(worker WorkerHandle, result error) error { 443 return j.inner.OnWorkerDispatched(worker, result) 444 } 445 446 func (j *jobMasterImplAsMasterImpl) OnWorkerOnline(worker WorkerHandle) error { 447 return j.inner.OnWorkerOnline(worker) 448 } 449 450 func (j *jobMasterImplAsMasterImpl) OnWorkerOffline(worker WorkerHandle, reason error) error { 451 return j.inner.OnWorkerOffline(worker, reason) 452 } 453 454 func (j *jobMasterImplAsMasterImpl) OnWorkerMessage(worker WorkerHandle, topic p2p.Topic, message interface{}) error { 455 return j.inner.OnWorkerMessage(worker, topic, message) 456 } 457 458 func (j *jobMasterImplAsMasterImpl) CloseImpl(ctx context.Context) { 459 log.Panic("unexpected Close call") 460 } 461 462 func (j *jobMasterImplAsMasterImpl) StopImpl(ctx context.Context) { 463 log.Panic("unexpected StopImpl call") 464 }