github.com/pingcap/tiflow@v0.0.0-20240520035814-5bf52d54e205/engine/pkg/client/executor_group.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 client
    15  
    16  import (
    17  	"container/list"
    18  	"context"
    19  	"sync"
    20  	"time"
    21  
    22  	"github.com/pingcap/tiflow/engine/model"
    23  	"github.com/pingcap/tiflow/pkg/errors"
    24  	"github.com/pingcap/tiflow/pkg/retry"
    25  	"github.com/pingcap/tiflow/pkg/security"
    26  	"go.uber.org/zap"
    27  )
    28  
    29  const (
    30  	// getExecutorBlockTimeout indicates the maximum wait time
    31  	// when we would like to retrieve the client for a given executor ID.
    32  	//
    33  	// The scenarios where we may need to wait is that
    34  	// 1) Due to an inconsistency of the servermaster's and the executor's states,
    35  	//    the executor has not received the information of another peer yet,
    36  	// 2) The target executor has gone offline.
    37  	//
    38  	// Here we do not want to set getExecutorBlockTimeout to avoid a large
    39  	// tail latency for scheduling a worker.
    40  	getExecutorBlockTimeout = 1 * time.Second
    41  
    42  	// tombstoneKeepTime indicates how long we should keep tombstone records.
    43  	tombstoneKeepTime = 5 * time.Minute
    44  )
    45  
    46  // ExecutorGroup holds a group of ExecutorClients.
    47  // It is used by any component that would like to invoke RPC
    48  // methods on the executors.
    49  type ExecutorGroup interface {
    50  	// GetExecutorClient tries to get the ExecutorClient for the given executor.
    51  	// It will return (nil, false) immediately if no such executor is found.
    52  	GetExecutorClient(id model.ExecutorID) (ExecutorClient, bool)
    53  
    54  	// GetExecutorClientB tries to get the ExecutorClient for the given executor.
    55  	// It blocks until either the context has been canceled or the executor ID becomes valid.
    56  	GetExecutorClientB(ctx context.Context, id model.ExecutorID) (ExecutorClient, error)
    57  }
    58  
    59  // A tombstoneEntry records when a given executor has been
    60  // removed from an ExecutorGroup.
    61  type tombstoneEntry struct {
    62  	id         model.ExecutorID
    63  	removeTime time.Time
    64  }
    65  
    66  // DefaultExecutorGroup is the default implementation for ExecutorGroup.
    67  type DefaultExecutorGroup struct {
    68  	mu      sync.RWMutex
    69  	clients map[model.ExecutorID]ExecutorClient
    70  	// tombstoneList stores a list of offline executors to reduce
    71  	// the tail latency of trying to get client for a non-existent executor.
    72  	tombstoneList *list.List // stores tombstoneEntry
    73  
    74  	logger            *zap.Logger
    75  	clientFactory     executorClientFactory
    76  	tombstoneKeepTime time.Duration
    77  }
    78  
    79  // NewExecutorGroup creates a new ExecutorGroup.
    80  func NewExecutorGroup(
    81  	credentials *security.Credential,
    82  	logger *zap.Logger,
    83  ) *DefaultExecutorGroup {
    84  	return newExecutorGroupWithClientFactory(
    85  		logger,
    86  		newExecutorClientFactory(credentials, logger))
    87  }
    88  
    89  func newExecutorGroupWithClientFactory(
    90  	logger *zap.Logger,
    91  	factory executorClientFactory,
    92  ) *DefaultExecutorGroup {
    93  	if logger == nil {
    94  		logger = zap.L()
    95  	}
    96  	return &DefaultExecutorGroup{
    97  		clients:       make(map[model.ExecutorID]ExecutorClient),
    98  		clientFactory: factory,
    99  		tombstoneList: list.New(),
   100  
   101  		logger:            logger,
   102  		tombstoneKeepTime: tombstoneKeepTime,
   103  	}
   104  }
   105  
   106  // GetExecutorClient tries to get the ExecutorClient for the given executor.
   107  func (g *DefaultExecutorGroup) GetExecutorClient(id model.ExecutorID) (ExecutorClient, bool) {
   108  	g.mu.RLock()
   109  	defer g.mu.RUnlock()
   110  
   111  	if client, ok := g.clients[id]; ok {
   112  		return client, true
   113  	}
   114  	return nil, false
   115  }
   116  
   117  // GetExecutorClientB tries to get the ExecutorClient for the given executor.
   118  // It blocks until either the context has been canceled or the executor ID becomes valid.
   119  //
   120  // When an Executor goes offline, the ID is added to a tombstone list so that
   121  // we can fail fast. The assumption here is that executor IDs will not be reused.
   122  func (g *DefaultExecutorGroup) GetExecutorClientB(ctx context.Context, id model.ExecutorID) (ExecutorClient, error) {
   123  	ctx, cancel := context.WithTimeout(ctx, getExecutorBlockTimeout)
   124  	defer cancel()
   125  
   126  	g.cleanTombstones()
   127  
   128  	var ret ExecutorClient
   129  	err := retry.Do(ctx, func() error {
   130  		g.mu.RLock()
   131  		defer g.mu.RUnlock()
   132  
   133  		if client, ok := g.clients[id]; ok {
   134  			ret = client
   135  			return nil
   136  		}
   137  		for item := g.tombstoneList.Front(); item != nil; item = item.Next() {
   138  			tombstone := item.Value.(tombstoneEntry)
   139  			if tombstone.id == id {
   140  				return errors.ErrTombstoneExecutor.GenWithStackByArgs(id)
   141  			}
   142  		}
   143  		return errors.ErrExecutorNotFound.GenWithStackByArgs(id)
   144  	},
   145  		retry.WithBackoffMaxDelay(10),
   146  		retry.WithIsRetryableErr(func(err error) bool {
   147  			return errors.Is(err, errors.ErrExecutorNotFound)
   148  		}))
   149  	if err != nil {
   150  		if errors.IsContextCanceledError(err) || errors.IsContextDeadlineExceededError(err) {
   151  			return nil, errors.ErrExecutorNotFound.GenWithStackByArgs(id)
   152  		}
   153  		return nil, err
   154  	}
   155  	return ret, nil
   156  }
   157  
   158  // UpdateExecutorList updates the stored clients using a map from executor IDs
   159  // to their addresses.
   160  // Note: This method will not wait for the clients to be fully connected.
   161  // In the rare case where grpc.Dial does fail, an error will be returned to the caller
   162  // of this method, and the caller should retry appropriately.
   163  func (g *DefaultExecutorGroup) UpdateExecutorList(executors map[model.ExecutorID]string) error {
   164  	g.mu.Lock()
   165  	defer g.mu.Unlock()
   166  
   167  	// Remove outdated clients.
   168  	for executorID, client := range g.clients {
   169  		if _, exists := executors[executorID]; exists {
   170  			continue
   171  		}
   172  		// The executor does not exist in the new list.
   173  		client.Close()
   174  
   175  		delete(g.clients, executorID)
   176  		g.logger.Info("executor client removed",
   177  			zap.String("executor-id", string(executorID)))
   178  	}
   179  
   180  	for executorID, addr := range executors {
   181  		if _, exists := g.clients[executorID]; exists {
   182  			// The executor already exists.
   183  			continue
   184  		}
   185  
   186  		// NewExecutorClient should be non-blocking.
   187  		client, err := g.clientFactory.NewExecutorClient(addr)
   188  		if err != nil {
   189  			g.logger.Warn("failed to create new client",
   190  				zap.String("executor-id", string(executorID)),
   191  				zap.String("address", addr),
   192  				zap.Error(err))
   193  			return err
   194  		}
   195  
   196  		g.clients[executorID] = client
   197  		g.logger.Info("executor client added",
   198  			zap.String("executor-id", string(executorID)),
   199  			zap.String("address", addr))
   200  	}
   201  	return nil
   202  }
   203  
   204  // AddExecutor adds an executor to the executor group. A new ExecutorClient will
   205  // be created by this method.
   206  // Note that since we are using asynchronous Dial, this method usually does not fail
   207  // even if a bad address is provided.
   208  func (g *DefaultExecutorGroup) AddExecutor(executorID model.ExecutorID, addr string) error {
   209  	g.mu.Lock()
   210  	defer g.mu.Unlock()
   211  
   212  	if _, exists := g.clients[executorID]; exists {
   213  		return errors.ErrExecutorAlreadyExists.GenWithStackByArgs(executorID)
   214  	}
   215  
   216  	// NewExecutorClient should be non-blocking.
   217  	client, err := g.clientFactory.NewExecutorClient(addr)
   218  	if err != nil {
   219  		g.logger.Warn("failed to create new client",
   220  			zap.String("executor-id", string(executorID)),
   221  			zap.String("address", addr),
   222  			zap.Error(err))
   223  		return err
   224  	}
   225  
   226  	g.clients[executorID] = client
   227  	g.logger.Info("executor client added",
   228  		zap.String("executor-id", string(executorID)),
   229  		zap.String("address", addr))
   230  	return nil
   231  }
   232  
   233  // RemoveExecutor removes an executor from the group.
   234  // Note that the ExecutorClient maintained will be closed.
   235  func (g *DefaultExecutorGroup) RemoveExecutor(executorID model.ExecutorID) error {
   236  	g.mu.Lock()
   237  	defer g.mu.Unlock()
   238  
   239  	client, exists := g.clients[executorID]
   240  	if !exists {
   241  		g.logger.Info("trying to remove non-existent executor",
   242  			zap.String("executor-id", string(executorID)))
   243  		return errors.ErrExecutorNotFound.GenWithStackByArgs(executorID)
   244  	}
   245  
   246  	client.Close()
   247  
   248  	delete(g.clients, executorID)
   249  	g.tombstoneList.PushFront(tombstoneEntry{
   250  		id:         executorID,
   251  		removeTime: time.Now(),
   252  	})
   253  	g.logger.Info("executor client removed",
   254  		zap.String("executor-id", string(executorID)))
   255  
   256  	return nil
   257  }
   258  
   259  func (g *DefaultExecutorGroup) cleanTombstones() {
   260  	g.mu.Lock()
   261  	defer g.mu.Unlock()
   262  
   263  	for g.tombstoneList.Len() > 0 {
   264  		elem := g.tombstoneList.Back()
   265  		tombstone := elem.Value.(tombstoneEntry)
   266  		if time.Since(tombstone.removeTime) > g.tombstoneKeepTime {
   267  			g.tombstoneList.Remove(elem)
   268  			continue
   269  		}
   270  		break
   271  	}
   272  }