github.com/pingcap/tiflow@v0.0.0-20240520035814-5bf52d54e205/engine/framework/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 framework
    15  
    16  import (
    17  	"context"
    18  	"sync"
    19  	"time"
    20  
    21  	"github.com/pingcap/log"
    22  	runtime "github.com/pingcap/tiflow/engine/executor/worker"
    23  	"github.com/pingcap/tiflow/engine/framework/config"
    24  	frameErrors "github.com/pingcap/tiflow/engine/framework/internal/errors"
    25  	"github.com/pingcap/tiflow/engine/framework/internal/worker"
    26  	frameLog "github.com/pingcap/tiflow/engine/framework/logutil"
    27  	"github.com/pingcap/tiflow/engine/framework/metadata"
    28  	frameModel "github.com/pingcap/tiflow/engine/framework/model"
    29  	"github.com/pingcap/tiflow/engine/framework/statusutil"
    30  	"github.com/pingcap/tiflow/engine/pkg/clock"
    31  	dcontext "github.com/pingcap/tiflow/engine/pkg/context"
    32  	"github.com/pingcap/tiflow/engine/pkg/errctx"
    33  	"github.com/pingcap/tiflow/engine/pkg/externalresource/broker"
    34  	resModel "github.com/pingcap/tiflow/engine/pkg/externalresource/model"
    35  	"github.com/pingcap/tiflow/engine/pkg/meta"
    36  	metaModel "github.com/pingcap/tiflow/engine/pkg/meta/model"
    37  	pkgOrm "github.com/pingcap/tiflow/engine/pkg/orm"
    38  	"github.com/pingcap/tiflow/engine/pkg/p2p"
    39  	"github.com/pingcap/tiflow/engine/pkg/promutil"
    40  	"github.com/pingcap/tiflow/engine/pkg/tenant"
    41  	"github.com/pingcap/tiflow/pkg/errors"
    42  	"github.com/pingcap/tiflow/pkg/logutil"
    43  	"github.com/pingcap/tiflow/pkg/workerpool"
    44  	"go.uber.org/dig"
    45  	"go.uber.org/zap"
    46  )
    47  
    48  // Worker defines an interface that provides all methods that will be used in
    49  // runtime(runner container)
    50  type Worker interface {
    51  	Init(ctx context.Context) error
    52  	Poll(ctx context.Context) error
    53  	ID() runtime.RunnableID
    54  	Close(ctx context.Context) error
    55  	Stop(ctx context.Context) error
    56  	NotifyExit(ctx context.Context, errIn error) error
    57  }
    58  
    59  // WorkerImpl is the implementation of a worker of dataflow engine.
    60  // the implementation struct must embed the framework.BaseWorker interface, this
    61  // interface will be initialized by the framework.
    62  type WorkerImpl interface {
    63  	// InitImpl is called as the consequence of CreateWorker from jobmaster or failover,
    64  	// business logic is expected to do initialization here.
    65  	// Return:
    66  	// - error to let the framework call CloseImpl.
    67  	// Concurrent safety:
    68  	// - this function is called as the first callback function of an WorkerImpl
    69  	//   instance, and it's not concurrent with other callbacks.
    70  	InitImpl(ctx context.Context) error
    71  
    72  	// Tick is called on a fixed interval after WorkerImpl is initialized, business
    73  	// logic can do some periodic tasks here.
    74  	// Return:
    75  	// - error to let the framework call CloseImpl.
    76  	// Concurrent safety:
    77  	// - this function may be concurrently called with OnMasterMessage.
    78  	Tick(ctx context.Context) error
    79  
    80  	// OnMasterMessage is called when worker receives master message, business developer
    81  	// does not need to implement it.
    82  	// TODO: move it out of WorkerImpl and should not be concurrent with CloseImpl.
    83  	OnMasterMessage(ctx context.Context, topic p2p.Topic, message p2p.MessageValue) error
    84  
    85  	// CloseImpl is called as the consequence of returning error from InitImpl or
    86  	// Tick, the Tick will be stopped after entering this function. Business logic
    87  	// is expected to release resources here, but business developer should be aware
    88  	// that when the runtime is crashed, CloseImpl has no time to be called.
    89  	// CloseImpl will only be called for once.
    90  	// TODO: no other callbacks will be called after CloseImpl
    91  	// Concurrent safety:
    92  	// - this function may be concurrently called with OnMasterMessage.
    93  	CloseImpl(ctx context.Context)
    94  }
    95  
    96  // BaseWorker defines the worker interface, it embeds a Worker interface and adds
    97  // more utility methods
    98  // TODO: decouple the BaseWorker and WorkerService(for business)
    99  type BaseWorker interface {
   100  	Worker
   101  
   102  	// MetaKVClient return business metastore kv client with job-level isolation
   103  	MetaKVClient() metaModel.KVClient
   104  
   105  	// MetricFactory return a promethus factory with some underlying labels(e.g. job-id, work-id)
   106  	MetricFactory() promutil.Factory
   107  
   108  	// Logger return a zap logger with some underlying fields(e.g. job-id)
   109  	Logger() *zap.Logger
   110  
   111  	// UpdateStatus persists the status to framework metastore if worker status is changed and
   112  	// sends 'status updated message' to master.
   113  	UpdateStatus(ctx context.Context, status frameModel.WorkerStatus) error
   114  
   115  	// SendMessage sends a message of specific topic to master in a blocking or nonblocking way
   116  	SendMessage(ctx context.Context, topic p2p.Topic, message interface{}, nonblocking bool) error
   117  
   118  	// OpenStorage creates a resource and return the resource handle
   119  	OpenStorage(
   120  		ctx context.Context, resourcePath resModel.ResourceID, opts ...broker.OpenStorageOption,
   121  	) (broker.Handle, error)
   122  
   123  	// GetEnabledBucketStorage returns whether the bucket storage is enabled and
   124  	// the resource type if bucket exists
   125  	GetEnabledBucketStorage() (bool, resModel.ResourceType)
   126  
   127  	// Exit should be called when worker (in user logic) wants to exit.
   128  	// exitReason: ExitReasonFinished/ExitReasonCanceled/ExitReasonFailed
   129  	Exit(ctx context.Context, exitReason ExitReason, err error, extBytes []byte) error
   130  }
   131  
   132  // DefaultBaseWorker implements BaseWorker interface, it also embeds an Impl
   133  // which implements the WorkerImpl interface and passed from business logic.
   134  type DefaultBaseWorker struct {
   135  	Impl WorkerImpl
   136  
   137  	messageHandlerManager p2p.MessageHandlerManager
   138  	messageSender         p2p.MessageSender
   139  	// framework metastore client
   140  	frameMetaClient pkgOrm.Client
   141  	resourceBroker  broker.Broker
   142  
   143  	masterClient *worker.MasterClient
   144  	masterID     frameModel.MasterID
   145  
   146  	workerMetaClient *metadata.WorkerStatusClient
   147  	statusSender     *statusutil.Writer
   148  	workerStatus     *frameModel.WorkerStatus
   149  	messageRouter    *MessageRouter
   150  
   151  	id            frameModel.WorkerID
   152  	timeoutConfig config.TimeoutConfig
   153  
   154  	pool workerpool.AsyncPool
   155  
   156  	wg        sync.WaitGroup
   157  	errCenter *errctx.ErrCenter
   158  
   159  	cancelMu      sync.Mutex
   160  	cancelBgTasks context.CancelFunc
   161  	cancelPool    context.CancelFunc
   162  
   163  	closeImplOnce sync.Once
   164  
   165  	clock clock.Clock
   166  
   167  	// business metastore kvclient with namespace
   168  	businessMetaKVClient metaModel.KVClient
   169  
   170  	// metricFactory can produce metric with underlying project info and job info
   171  	metricFactory promutil.Factory
   172  
   173  	// logger is the zap logger with underlying project info and job info
   174  	logger *zap.Logger
   175  
   176  	// tenant/project information
   177  	projectInfo tenant.ProjectInfo
   178  }
   179  
   180  type workerParams struct {
   181  	dig.In
   182  
   183  	MessageHandlerManager p2p.MessageHandlerManager
   184  	MessageSender         p2p.MessageSender
   185  	FrameMetaClient       pkgOrm.Client
   186  	BusinessClientConn    metaModel.ClientConn
   187  	ResourceBroker        broker.Broker
   188  }
   189  
   190  // NewBaseWorker creates a new BaseWorker instance
   191  func NewBaseWorker(
   192  	ctx *dcontext.Context,
   193  	impl WorkerImpl,
   194  	workerID frameModel.WorkerID,
   195  	masterID frameModel.MasterID,
   196  	tp frameModel.WorkerType,
   197  	epoch frameModel.Epoch,
   198  ) BaseWorker {
   199  	var params workerParams
   200  	if err := ctx.Deps().Fill(&params); err != nil {
   201  		log.Panic("Failed to fill dependencies for BaseWorker",
   202  			zap.Error(err))
   203  	}
   204  
   205  	logger := logutil.FromContext(*ctx)
   206  
   207  	cli, err := meta.NewKVClientWithNamespace(params.BusinessClientConn, ctx.ProjectInfo.UniqueID(), masterID)
   208  	if err != nil {
   209  		// TODO more elegant error handling
   210  		log.Panic("failed to create business kvclient", zap.Error(err))
   211  	}
   212  
   213  	return &DefaultBaseWorker{
   214  		Impl:                  impl,
   215  		messageHandlerManager: params.MessageHandlerManager,
   216  		messageSender:         params.MessageSender,
   217  		frameMetaClient:       params.FrameMetaClient,
   218  		resourceBroker:        params.ResourceBroker,
   219  
   220  		masterID:    masterID,
   221  		id:          workerID,
   222  		projectInfo: ctx.ProjectInfo,
   223  		workerStatus: &frameModel.WorkerStatus{
   224  			ProjectID: ctx.ProjectInfo.UniqueID(),
   225  			JobID:     masterID,
   226  			ID:        workerID,
   227  			Type:      tp,
   228  			Epoch:     epoch,
   229  		},
   230  		timeoutConfig: config.DefaultTimeoutConfig(),
   231  
   232  		pool: workerpool.NewDefaultAsyncPool(1),
   233  
   234  		errCenter:            errctx.NewErrCenter(),
   235  		clock:                clock.New(),
   236  		businessMetaKVClient: cli,
   237  		metricFactory:        promutil.NewFactory4Worker(ctx.ProjectInfo, MustConvertWorkerType2JobType(tp), masterID, workerID),
   238  		logger:               frameLog.WithWorkerID(frameLog.WithMasterID(logger, masterID), workerID),
   239  	}
   240  }
   241  
   242  // Init implements BaseWorker.Init
   243  func (w *DefaultBaseWorker) Init(ctx context.Context) error {
   244  	// Note this context must not be held in any resident goroutine.
   245  	ctx, cancel := w.errCenter.WithCancelOnFirstError(ctx)
   246  	defer cancel()
   247  
   248  	if err := w.doPreInit(ctx); err != nil {
   249  		return errors.Trace(err)
   250  	}
   251  
   252  	if err := w.Impl.InitImpl(ctx); err != nil {
   253  		return errors.Trace(err)
   254  	}
   255  
   256  	if err := w.doPostInit(ctx); err != nil {
   257  		return errors.Trace(err)
   258  	}
   259  
   260  	return nil
   261  }
   262  
   263  // NotifyExit implements BaseWorker.NotifyExit
   264  func (w *DefaultBaseWorker) NotifyExit(ctx context.Context, errIn error) (retErr error) {
   265  	w.closeImplOnce.Do(func() {
   266  		// Must ensure that the business logic is
   267  		// notified before closing.
   268  		w.callCloseImpl()
   269  	})
   270  
   271  	startTime := time.Now()
   272  	defer func() {
   273  		duration := time.Since(startTime)
   274  		w.logger.Info("worker finished exiting",
   275  			zap.NamedError("caused", errIn),
   276  			zap.Duration("duration", duration),
   277  			logutil.ShortError(retErr))
   278  	}()
   279  
   280  	w.logger.Info("worker start exiting", zap.NamedError("cause", errIn))
   281  	return w.masterClient.WaitClosed(ctx)
   282  }
   283  
   284  func (w *DefaultBaseWorker) doPreInit(ctx context.Context) (retErr error) {
   285  	defer func() {
   286  		if retErr != nil {
   287  			// Wraps the error as FailFast because errors
   288  			// that occurred during doPreInit may indicate
   289  			// a failure that's severe enough that it is not
   290  			// possible for the worker to correctly communicate
   291  			// with the master.
   292  			retErr = frameErrors.FailFast(retErr)
   293  		}
   294  	}()
   295  	// poolCtx will be held in background goroutines, and it won't be canceled
   296  	// until DefaultBaseWorker.Close is called.
   297  	poolCtx, cancelPool := context.WithCancel(context.Background())
   298  	w.cancelMu.Lock()
   299  	w.cancelPool = cancelPool
   300  	w.cancelMu.Unlock()
   301  
   302  	w.wg.Add(1)
   303  	go func() {
   304  		defer w.wg.Done()
   305  		err := w.pool.Run(poolCtx)
   306  		w.Logger().Info("workerpool exited",
   307  			zap.String("worker-id", w.id),
   308  			zap.Error(err))
   309  	}()
   310  
   311  	initTime := w.clock.Mono()
   312  	rctx, ok := runtime.ToRuntimeCtx(ctx)
   313  	if ok {
   314  		initTime = rctx.SubmitTime()
   315  	}
   316  
   317  	w.masterClient = worker.NewMasterClient(
   318  		w.masterID,
   319  		w.id,
   320  		w.messageSender,
   321  		w.frameMetaClient,
   322  		initTime,
   323  		w.clock,
   324  		w.workerStatus.Epoch,
   325  	)
   326  
   327  	w.workerMetaClient = metadata.NewWorkerStatusClient(w.masterID, w.frameMetaClient)
   328  
   329  	w.statusSender = statusutil.NewWriter(
   330  		w.frameMetaClient, w.messageSender, w.masterClient, w.id)
   331  	w.messageRouter = NewMessageRouter(w.id, w.pool, defaultMessageRouterBufferSize,
   332  		func(topic p2p.Topic, msg p2p.MessageValue) error {
   333  			return w.Impl.OnMasterMessage(poolCtx, topic, msg)
   334  		},
   335  	)
   336  
   337  	w.startBackgroundTasks()
   338  
   339  	if err := w.initMessageHandlers(ctx); err != nil {
   340  		return errors.Trace(err)
   341  	}
   342  
   343  	if err := w.masterClient.InitMasterInfoFromMeta(ctx); err != nil {
   344  		return errors.Trace(err)
   345  	}
   346  
   347  	return nil
   348  }
   349  
   350  func (w *DefaultBaseWorker) doPostInit(ctx context.Context) error {
   351  	// Upsert the worker to ensure we have created the worker info
   352  	if err := w.frameMetaClient.UpsertWorker(ctx, w.workerStatus); err != nil {
   353  		return errors.Trace(err)
   354  	}
   355  
   356  	w.workerStatus.State = frameModel.WorkerStateInit
   357  	if err := w.statusSender.UpdateStatus(ctx, w.workerStatus); err != nil {
   358  		return errors.Trace(err)
   359  	}
   360  
   361  	return nil
   362  }
   363  
   364  func (w *DefaultBaseWorker) doPoll(ctx context.Context) error {
   365  	if err := w.errCenter.CheckError(); err != nil {
   366  		return err
   367  	}
   368  
   369  	if err := w.messageHandlerManager.CheckError(ctx); err != nil {
   370  		return err
   371  	}
   372  
   373  	return w.messageRouter.Tick(ctx)
   374  }
   375  
   376  // Poll implements BaseWorker.Poll
   377  func (w *DefaultBaseWorker) Poll(ctx context.Context) error {
   378  	ctx, cancel := w.errCenter.WithCancelOnFirstError(ctx)
   379  	defer cancel()
   380  
   381  	if err := w.doPoll(ctx); err != nil {
   382  		return err
   383  	}
   384  
   385  	if err := w.Impl.Tick(ctx); err != nil {
   386  		w.errCenter.OnError(err)
   387  		return err
   388  	}
   389  	return nil
   390  }
   391  
   392  func (w *DefaultBaseWorker) doClose() {
   393  	if w.resourceBroker != nil {
   394  		// Closing the resource broker here will release all temporary file
   395  		// resources created by the worker.
   396  		w.resourceBroker.OnWorkerClosed(context.Background(), w.id, w.masterID)
   397  	}
   398  
   399  	w.cancelMu.Lock()
   400  	if w.cancelBgTasks != nil {
   401  		w.cancelBgTasks()
   402  	}
   403  	if w.cancelPool != nil {
   404  		w.cancelPool()
   405  	}
   406  	w.cancelMu.Unlock()
   407  
   408  	closeCtx, cancel := context.WithTimeout(context.Background(), time.Second*3)
   409  	defer cancel()
   410  
   411  	if err := w.messageHandlerManager.Clean(closeCtx); err != nil {
   412  		w.Logger().Warn("cleaning message handlers failed",
   413  			zap.Error(err))
   414  	}
   415  
   416  	w.wg.Wait()
   417  	promutil.UnregisterWorkerMetrics(w.id)
   418  	w.businessMetaKVClient.Close()
   419  }
   420  
   421  // Close implements BaseWorker.Close
   422  // TODO remove the return value from the signature.
   423  func (w *DefaultBaseWorker) Close(ctx context.Context) error {
   424  	w.closeImplOnce.Do(func() {
   425  		w.callCloseImpl()
   426  	})
   427  
   428  	w.doClose()
   429  	return nil
   430  }
   431  
   432  func (w *DefaultBaseWorker) callCloseImpl() {
   433  	closeCtx, cancel := context.WithTimeout(
   434  		context.Background(), w.timeoutConfig.CloseWorkerTimeout)
   435  	defer cancel()
   436  
   437  	w.Impl.CloseImpl(closeCtx)
   438  }
   439  
   440  // Stop implements Worker.Stop, works the same as Worker.Close
   441  func (w *DefaultBaseWorker) Stop(ctx context.Context) error {
   442  	return w.Close(ctx)
   443  }
   444  
   445  // ID implements BaseWorker.ID
   446  func (w *DefaultBaseWorker) ID() runtime.RunnableID {
   447  	return w.id
   448  }
   449  
   450  // MetaKVClient implements BaseWorker.MetaKVClient
   451  func (w *DefaultBaseWorker) MetaKVClient() metaModel.KVClient {
   452  	return w.businessMetaKVClient
   453  }
   454  
   455  // MetricFactory implements BaseWorker.MetricFactory
   456  func (w *DefaultBaseWorker) MetricFactory() promutil.Factory {
   457  	return w.metricFactory
   458  }
   459  
   460  // Logger implements BaseMaster.Logger
   461  func (w *DefaultBaseWorker) Logger() *zap.Logger {
   462  	return w.logger
   463  }
   464  
   465  // UpdateStatus updates the worker's status and tries to notify the master.
   466  // The status is persisted if State or ErrorMsg has changed. Refer to (*WorkerState).HasSignificantChange.
   467  //
   468  // If UpdateStatus returns without an error, then the status must have been persisted,
   469  // but there is no guarantee that the master has received a notification.
   470  // Note that if the master cannot handle the notifications fast enough, notifications
   471  // can be lost.
   472  func (w *DefaultBaseWorker) UpdateStatus(ctx context.Context, status frameModel.WorkerStatus) error {
   473  	ctx, cancel := w.errCenter.WithCancelOnFirstError(ctx)
   474  	defer cancel()
   475  
   476  	w.workerStatus.State = status.State
   477  	w.workerStatus.ErrorMsg = status.ErrorMsg
   478  	w.workerStatus.ExtBytes = status.ExtBytes
   479  	err := w.statusSender.UpdateStatus(ctx, w.workerStatus)
   480  	if err != nil {
   481  		return errors.Trace(err)
   482  	}
   483  	return nil
   484  }
   485  
   486  // SendMessage implements BaseWorker.SendMessage
   487  func (w *DefaultBaseWorker) SendMessage(
   488  	ctx context.Context,
   489  	topic p2p.Topic,
   490  	message interface{},
   491  	nonblocking bool,
   492  ) error {
   493  	var err error
   494  	ctx, cancel := w.errCenter.WithCancelOnFirstError(ctx)
   495  	defer cancel()
   496  	if nonblocking {
   497  		_, err = w.messageSender.SendToNode(ctx, w.masterClient.MasterNode(), topic, message)
   498  	} else {
   499  		err = w.messageSender.SendToNodeB(ctx, w.masterClient.MasterNode(), topic, message)
   500  	}
   501  	return err
   502  }
   503  
   504  // OpenStorage implements BaseWorker.OpenStorage
   505  func (w *DefaultBaseWorker) OpenStorage(
   506  	ctx context.Context, resourcePath resModel.ResourceID, opts ...broker.OpenStorageOption,
   507  ) (broker.Handle, error) {
   508  	ctx, cancel := w.errCenter.WithCancelOnFirstError(ctx)
   509  	defer cancel()
   510  	return w.resourceBroker.OpenStorage(ctx, w.projectInfo, w.id, w.masterID, resourcePath, opts...)
   511  }
   512  
   513  // GetEnabledBucketStorage implements BaseWorker.GetEnabledBucketStorage
   514  func (w *DefaultBaseWorker) GetEnabledBucketStorage() (bool, resModel.ResourceType) {
   515  	return w.resourceBroker.GetEnabledBucketStorage()
   516  }
   517  
   518  // Exit implements BaseWorker.Exit
   519  func (w *DefaultBaseWorker) Exit(ctx context.Context, exitReason ExitReason, err error, extBytes []byte) (errRet error) {
   520  	// Set the errCenter to prevent user from forgetting to return directly after calling 'Exit'
   521  	defer func() {
   522  		// keep the original error or ErrWorkerFinish in error center
   523  		if err == nil {
   524  			err = errors.ErrWorkerFinish.FastGenByArgs()
   525  		}
   526  		w.onError(err)
   527  	}()
   528  
   529  	switch exitReason {
   530  	case ExitReasonFinished:
   531  		w.workerStatus.State = frameModel.WorkerStateFinished
   532  	case ExitReasonCanceled:
   533  		// TODO: replace stop with cancel
   534  		w.workerStatus.State = frameModel.WorkerStateStopped
   535  	case ExitReasonFailed:
   536  		// TODO: replace error with failed
   537  		w.workerStatus.State = frameModel.WorkerStateError
   538  	default:
   539  		w.workerStatus.State = frameModel.WorkerStateError
   540  	}
   541  
   542  	if err != nil {
   543  		w.workerStatus.ErrorMsg = err.Error()
   544  	}
   545  	w.workerStatus.ExtBytes = extBytes
   546  	return w.statusSender.UpdateStatus(ctx, w.workerStatus)
   547  }
   548  
   549  func (w *DefaultBaseWorker) startBackgroundTasks() {
   550  	ctx, cancel := context.WithCancel(context.Background())
   551  
   552  	w.cancelMu.Lock()
   553  	w.cancelBgTasks = cancel
   554  	w.cancelMu.Unlock()
   555  
   556  	w.wg.Add(1)
   557  	go func() {
   558  		defer w.wg.Done()
   559  		if err := w.runHeartbeatWorker(ctx); err != nil {
   560  			w.onError(err)
   561  		}
   562  	}()
   563  
   564  	w.wg.Add(1)
   565  	go func() {
   566  		defer w.wg.Done()
   567  		if err := w.runWatchDog(ctx); err != nil {
   568  			w.onError(err)
   569  		}
   570  	}()
   571  }
   572  
   573  func (w *DefaultBaseWorker) runHeartbeatWorker(ctx context.Context) error {
   574  	ticker := w.clock.Ticker(w.timeoutConfig.WorkerHeartbeatInterval)
   575  	defer ticker.Stop()
   576  
   577  	for {
   578  		select {
   579  		case <-ctx.Done():
   580  			return errors.Trace(ctx.Err())
   581  		case <-ticker.C:
   582  			if err := w.masterClient.SendHeartBeat(ctx); err != nil {
   583  				return errors.Trace(err)
   584  			}
   585  		}
   586  	}
   587  }
   588  
   589  func (w *DefaultBaseWorker) runWatchDog(ctx context.Context) error {
   590  	ticker := w.clock.Ticker(w.timeoutConfig.WorkerHeartbeatInterval)
   591  	for {
   592  		select {
   593  		case <-ctx.Done():
   594  			return errors.Trace(ctx.Err())
   595  		case <-ticker.C:
   596  		}
   597  
   598  		isNormal, err := w.masterClient.CheckMasterTimeout()
   599  		if err != nil {
   600  			return errors.Trace(err)
   601  		}
   602  		if !isNormal {
   603  			errOut := errors.ErrWorkerSuicide.GenWithStackByArgs(w.masterClient.MasterID())
   604  			return errOut
   605  		}
   606  	}
   607  }
   608  
   609  func (w *DefaultBaseWorker) initMessageHandlers(ctx context.Context) (retErr error) {
   610  	defer func() {
   611  		if retErr != nil {
   612  			if err := w.messageHandlerManager.Clean(context.Background()); err != nil {
   613  				w.Logger().Warn("Failed to clean up message handlers",
   614  					zap.String("master-id", w.masterID),
   615  					zap.String("worker-id", w.id))
   616  			}
   617  		}
   618  	}()
   619  	topic := frameModel.HeartbeatPongTopic(w.masterClient.MasterID(), w.id)
   620  	ok, err := w.messageHandlerManager.RegisterHandler(
   621  		ctx,
   622  		topic,
   623  		&frameModel.HeartbeatPongMessage{},
   624  		func(sender p2p.NodeID, value p2p.MessageValue) error {
   625  			msg := value.(*frameModel.HeartbeatPongMessage)
   626  			w.Logger().Info("heartbeat pong received",
   627  				zap.String("master-id", w.masterID),
   628  				zap.Any("msg", msg))
   629  			w.masterClient.HandleHeartbeat(sender, msg)
   630  			return nil
   631  		})
   632  	if err != nil {
   633  		return errors.Trace(err)
   634  	}
   635  	if !ok {
   636  		w.Logger().Panic("duplicate handler",
   637  			zap.String("topic", topic))
   638  	}
   639  
   640  	topic = frameModel.WorkerStatusChangeRequestTopic(w.masterID, w.id)
   641  	ok, err = w.messageHandlerManager.RegisterHandler(
   642  		ctx,
   643  		topic,
   644  		&frameModel.StatusChangeRequest{},
   645  		func(sender p2p.NodeID, value p2p.MessageValue) error {
   646  			msg, ok := value.(*frameModel.StatusChangeRequest)
   647  			if !ok {
   648  				return errors.ErrInvalidMasterMessage.GenWithStackByArgs(value)
   649  			}
   650  			w.messageRouter.AppendMessage(topic, msg)
   651  			return nil
   652  		})
   653  	if err != nil {
   654  		return errors.Trace(err)
   655  	}
   656  	if !ok {
   657  		w.Logger().Panic("duplicate handler", zap.String("topic", topic))
   658  	}
   659  
   660  	return nil
   661  }
   662  
   663  func (w *DefaultBaseWorker) onError(err error) {
   664  	w.errCenter.OnError(err)
   665  }