github.com/pingcap/tiflow@v0.0.0-20240520035814-5bf52d54e205/engine/framework/internal/worker/master_client.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 worker
    15  
    16  import (
    17  	"context"
    18  	"sync"
    19  	"time"
    20  
    21  	"github.com/pingcap/log"
    22  	"github.com/pingcap/tiflow/engine/framework/config"
    23  	"github.com/pingcap/tiflow/engine/framework/metadata"
    24  	frameModel "github.com/pingcap/tiflow/engine/framework/model"
    25  	"github.com/pingcap/tiflow/engine/pkg/clock"
    26  	pkgOrm "github.com/pingcap/tiflow/engine/pkg/orm"
    27  	"github.com/pingcap/tiflow/engine/pkg/p2p"
    28  	"github.com/pingcap/tiflow/pkg/errors"
    29  	"go.uber.org/atomic"
    30  	"go.uber.org/zap"
    31  )
    32  
    33  const (
    34  	reloadMasterInfoTimeout        = 10 * time.Second
    35  	workerExitWaitForMasterTimeout = time.Second * 15
    36  )
    37  
    38  type masterClientCloseState = int32
    39  
    40  const (
    41  	masterClientNormal = masterClientCloseState(iota + 1)
    42  	masterClientClosing
    43  	masterClientClosed
    44  )
    45  
    46  // MasterClient is used by the BaseWorker to communicate with
    47  type MasterClient struct {
    48  	// infoLock protects master's information,
    49  	// as master can fail over and so the information can change.
    50  	infoLock    sync.RWMutex
    51  	masterNode  p2p.NodeID
    52  	masterEpoch frameModel.Epoch
    53  	workerEpoch frameModel.Epoch
    54  
    55  	lastMasterAckedPingTime atomic.Duration
    56  
    57  	closeState atomic.Int32
    58  	closeCh    chan struct{}
    59  
    60  	// Immutable fields
    61  	workerID        frameModel.WorkerID
    62  	masterID        frameModel.MasterID
    63  	timeoutConfig   config.TimeoutConfig
    64  	messageSender   p2p.MessageSender
    65  	frameMetaClient pkgOrm.Client
    66  
    67  	clk clock.Clock
    68  }
    69  
    70  // NewMasterClient creates a new MasterClient.
    71  func NewMasterClient(
    72  	masterID frameModel.MasterID,
    73  	workerID frameModel.WorkerID,
    74  	messageSender p2p.MessageSender,
    75  	metaCli pkgOrm.Client,
    76  	initTime clock.MonotonicTime,
    77  	clk clock.Clock,
    78  	workerEpoch frameModel.Epoch,
    79  ) *MasterClient {
    80  	return &MasterClient{
    81  		masterID:                masterID,
    82  		workerID:                workerID,
    83  		messageSender:           messageSender,
    84  		frameMetaClient:         metaCli,
    85  		lastMasterAckedPingTime: *atomic.NewDuration(time.Duration(initTime)),
    86  		closeState:              *atomic.NewInt32(masterClientNormal),
    87  		closeCh:                 make(chan struct{}),
    88  		timeoutConfig:           config.DefaultTimeoutConfig(),
    89  		clk:                     clk,
    90  		workerEpoch:             workerEpoch,
    91  	}
    92  }
    93  
    94  // InitMasterInfoFromMeta reads the meta store and tries to find where
    95  // the master is.
    96  func (m *MasterClient) InitMasterInfoFromMeta(ctx context.Context) error {
    97  	metaClient := metadata.NewMasterMetadataClient(m.masterID, m.frameMetaClient)
    98  	masterMeta, err := metaClient.Load(ctx)
    99  	if err != nil {
   100  		return err
   101  	}
   102  
   103  	m.putMasterInfo(masterMeta.NodeID, masterMeta.Epoch)
   104  	return nil
   105  }
   106  
   107  // SyncRefreshMasterInfo reloads the master's info. It is useful if the caller
   108  // anticipates a master failover to have happened.
   109  func (m *MasterClient) SyncRefreshMasterInfo(ctx context.Context) error {
   110  	errCh := m.asyncReloadMasterInfo(ctx)
   111  	select {
   112  	case <-ctx.Done():
   113  		return errors.Trace(ctx.Err())
   114  	case err := <-errCh:
   115  		if err != nil {
   116  			return err
   117  		}
   118  	}
   119  	return nil
   120  }
   121  
   122  func (m *MasterClient) asyncReloadMasterInfo(ctx context.Context) <-chan error {
   123  	errCh := make(chan error, 1)
   124  	go func() {
   125  		defer close(errCh)
   126  
   127  		timeoutCtx, cancel := context.WithTimeout(ctx, reloadMasterInfoTimeout)
   128  		defer cancel()
   129  
   130  		metaClient := metadata.NewMasterMetadataClient(m.masterID, m.frameMetaClient)
   131  		masterMeta, err := metaClient.Load(timeoutCtx)
   132  		if err != nil {
   133  			log.Warn("async reload master info failed", zap.Error(err))
   134  			errCh <- err
   135  			return
   136  		}
   137  
   138  		m.putMasterInfo(masterMeta.NodeID, masterMeta.Epoch)
   139  	}()
   140  	return errCh
   141  }
   142  
   143  func (m *MasterClient) putMasterInfo(nodeID p2p.NodeID, epoch frameModel.Epoch) {
   144  	m.infoLock.Lock()
   145  	defer m.infoLock.Unlock()
   146  
   147  	if epoch < m.masterEpoch {
   148  		return
   149  	}
   150  
   151  	if epoch >= m.masterEpoch {
   152  		m.masterEpoch = epoch
   153  		m.masterNode = nodeID
   154  	}
   155  }
   156  
   157  func (m *MasterClient) getMasterInfo() (p2p.NodeID, frameModel.Epoch) {
   158  	m.infoLock.RLock()
   159  	defer m.infoLock.RUnlock()
   160  
   161  	return m.masterNode, m.masterEpoch
   162  }
   163  
   164  // MasterID returns the masterID.
   165  func (m *MasterClient) MasterID() frameModel.MasterID {
   166  	// No need to lock here as masterID is immutable.
   167  	return m.masterID
   168  }
   169  
   170  // MasterNode returns the node ID of the executor
   171  // on which the master is being run.
   172  func (m *MasterClient) MasterNode() p2p.NodeID {
   173  	nodeID, _ := m.getMasterInfo()
   174  	return nodeID
   175  }
   176  
   177  // Epoch returns the master epoch.
   178  // Note that the epoch is increased when the master
   179  // restarts.
   180  func (m *MasterClient) Epoch() frameModel.Epoch {
   181  	_, epoch := m.getMasterInfo()
   182  	return epoch
   183  }
   184  
   185  // WorkerEpoch returns the worker epoch.
   186  // This value is a constant value for the master client of every single worker
   187  func (m *MasterClient) WorkerEpoch() frameModel.Epoch {
   188  	return m.workerEpoch
   189  }
   190  
   191  // HandleHeartbeat handles heartbeat messages received from the master.
   192  func (m *MasterClient) HandleHeartbeat(sender p2p.NodeID, msg *frameModel.HeartbeatPongMessage) {
   193  	if msg.ToWorkerID != m.workerID {
   194  		log.Warn("Received heartbeat for wrong workerID",
   195  			zap.Any("msg", msg), zap.String("actual-worker-id", m.workerID))
   196  		return
   197  	}
   198  
   199  	_, epoch := m.getMasterInfo()
   200  	if msg.Epoch < epoch {
   201  		log.Info("epoch does not match, ignore stale heartbeat",
   202  			zap.Any("msg", msg),
   203  			zap.Int64("master-epoch", epoch))
   204  		return
   205  	}
   206  
   207  	if msg.Epoch > epoch {
   208  		// We received a heartbeat from a restarted master, we need to record
   209  		// its information.
   210  		m.putMasterInfo(sender, msg.Epoch)
   211  	}
   212  
   213  	if msg.IsFinished {
   214  		oldSt := m.closeState.Swap(masterClientClosed)
   215  		if oldSt == masterClientNormal {
   216  			// Jumping from Normal to Closed in unexpected
   217  			log.Panic("unexpected master client close state",
   218  				zap.String("master-id", m.masterID),
   219  				zap.String("worker-id", m.workerID))
   220  		}
   221  		if oldSt == masterClientClosing {
   222  			close(m.closeCh)
   223  		}
   224  	}
   225  
   226  	// worker may receive stale heartbeat pong message from job master, stale
   227  	// message doesn't contribute to job master aliveness detection and even
   228  	// leads to false positive.
   229  	lastAckTime := m.lastMasterAckedPingTime.Load()
   230  	if lastAckTime > time.Duration(msg.SendTime) {
   231  		log.Info("received stale pong heartbeat",
   232  			zap.Any("msg", msg), zap.Int64("lastAckTime", int64(lastAckTime)))
   233  	} else {
   234  		m.lastMasterAckedPingTime.Store(time.Duration(msg.SendTime))
   235  	}
   236  }
   237  
   238  // CheckMasterTimeout checks whether the master has timed out, i.e. we have lost
   239  // contact with the master for a while.
   240  func (m *MasterClient) CheckMasterTimeout() (ok bool, err error) {
   241  	lastMasterAckedPingTime := clock.MonotonicTime(m.lastMasterAckedPingTime.Load())
   242  
   243  	sinceLastAcked := m.clk.Mono().Sub(lastMasterAckedPingTime)
   244  	if sinceLastAcked <= 2*m.timeoutConfig.WorkerHeartbeatInterval {
   245  		return true, nil
   246  	}
   247  
   248  	if sinceLastAcked > 2*m.timeoutConfig.WorkerHeartbeatInterval &&
   249  		sinceLastAcked < m.timeoutConfig.WorkerTimeoutDuration {
   250  
   251  		// We ignore the error here
   252  		_ = m.asyncReloadMasterInfo(context.Background())
   253  		return true, nil
   254  	}
   255  
   256  	return false, nil
   257  }
   258  
   259  // SendHeartBeat sends a heartbeat to the master.
   260  func (m *MasterClient) SendHeartBeat(ctx context.Context) error {
   261  	nodeID, epoch := m.getMasterInfo()
   262  	// We use the monotonic time because we would like to serialize a local timestamp.
   263  	// The timestamp will be returned in a PONG for time-out check, so we need
   264  	// the timestamp to be a local monotonic timestamp, which is not exposed by the
   265  	// standard library `time`.
   266  	sendTime := m.clk.Mono()
   267  	isFinished := m.closeState.Load() == masterClientClosing
   268  
   269  	heartbeatMsg := &frameModel.HeartbeatPingMessage{
   270  		SendTime:     sendTime,
   271  		FromWorkerID: m.workerID,
   272  		Epoch:        epoch,
   273  		WorkerEpoch:  m.WorkerEpoch(),
   274  		IsFinished:   isFinished,
   275  	}
   276  
   277  	log.Debug("sending heartbeat", zap.String("worker", m.workerID),
   278  		zap.String("master-id", m.masterID),
   279  		zap.Int64("epoch", epoch), zap.Int64("worker-epoch", heartbeatMsg.WorkerEpoch),
   280  		zap.Int64("sendTime", int64(sendTime)))
   281  	ok, err := m.messageSender.SendToNode(ctx, nodeID, frameModel.HeartbeatPingTopic(m.masterID), heartbeatMsg)
   282  	if err != nil {
   283  		return errors.Trace(err)
   284  	}
   285  	log.Info("sending heartbeat success", zap.String("worker", m.workerID),
   286  		zap.String("master-id", m.masterID),
   287  		zap.Int64("epoch", epoch), zap.Int64("worker-epoch", heartbeatMsg.WorkerEpoch),
   288  		zap.Int64("sendTime", int64(sendTime)))
   289  	if !ok {
   290  		// Reloads master info asynchronously.
   291  		// Not using `ctx` because the caller might cancel unexpectedly.
   292  		_ = m.asyncReloadMasterInfo(context.Background())
   293  	}
   294  	return nil
   295  }
   296  
   297  // IsMasterSideClosed returns whether the master has marked the worker as closed.
   298  // It is used when the worker initiates an exit with an error, but the network
   299  // is fine.
   300  func (m *MasterClient) IsMasterSideClosed() bool {
   301  	return m.closeState.Load() == masterClientClosed
   302  }
   303  
   304  // WaitClosed marks the current worker as exiting, and
   305  // blocks until the master has acknowledged the exit.
   306  // The caller should make sure that no concurrent calls to
   307  // WaitClosed happens.
   308  func (m *MasterClient) WaitClosed(ctx context.Context) error {
   309  	switch m.closeState.Load() {
   310  	case masterClientNormal:
   311  		if !m.closeState.CAS(masterClientNormal, masterClientClosing) {
   312  			log.Panic("Unexpected close state in master client, race?",
   313  				zap.String("master-id", m.masterID),
   314  				zap.String("worker-id", m.workerID))
   315  		}
   316  	case masterClientClosing:
   317  		break // breaks switch
   318  	case masterClientClosed:
   319  		return nil
   320  	}
   321  
   322  	timer := m.clk.Timer(workerExitWaitForMasterTimeout)
   323  	defer timer.Stop()
   324  
   325  	select {
   326  	case <-ctx.Done():
   327  		return errors.Trace(ctx.Err())
   328  	case <-timer.C:
   329  		return errors.Trace(context.DeadlineExceeded)
   330  	case <-m.closeCh:
   331  	}
   332  
   333  	if m.closeState.Load() != masterClientClosed {
   334  		log.Panic("Unexpected close state in master client, bug?",
   335  			zap.String("master-id", m.masterID),
   336  			zap.String("worker-id", m.workerID))
   337  	}
   338  
   339  	return nil
   340  }
   341  
   342  // used in unit test only
   343  func (m *MasterClient) getLastMasterAckedPingTime() clock.MonotonicTime {
   344  	return clock.MonotonicTime(m.lastMasterAckedPingTime.Load())
   345  }