github.com/pingcap/tiflow@v0.0.0-20240520035814-5bf52d54e205/pkg/p2p/message_router.go (about)

     1  // Copyright 2021 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 p2p
    15  
    16  import (
    17  	"context"
    18  	"sync"
    19  
    20  	"github.com/pingcap/errors"
    21  	"github.com/pingcap/log"
    22  	"github.com/pingcap/tiflow/pkg/security"
    23  	"go.uber.org/atomic"
    24  	"go.uber.org/zap"
    25  )
    26  
    27  // MessageRouter is used to maintain clients to all the peers in the cluster
    28  // that the local node needs to communicate with.
    29  type MessageRouter interface {
    30  	// AddPeer should be invoked when a new peer is discovered.
    31  	AddPeer(id NodeID, addr string)
    32  	// RemovePeer should be invoked when a peer is determined to
    33  	// be permanently unavailable.
    34  	RemovePeer(id NodeID)
    35  	// GetClient returns a MessageClient for `target`. It returns
    36  	// nil if the target peer does not exist. The returned client
    37  	// is canceled if RemovePeer is called on `target`.
    38  	GetClient(target NodeID) MessageClient
    39  	// GetLocalChannel returns a channel that can be used for intra-node communication.
    40  	GetLocalChannel() <-chan RawMessageEntry
    41  	// Close cancels all clients maintained internally and waits for all clients to exit.
    42  	Close()
    43  	// Err returns a channel to receive errors from.
    44  	Err() <-chan error
    45  }
    46  
    47  type messageRouterImpl struct {
    48  	mu         sync.RWMutex
    49  	addressMap map[NodeID]string
    50  	clients    map[NodeID]clientWrapper
    51  
    52  	wg       sync.WaitGroup
    53  	isClosed atomic.Bool
    54  	errCh    chan error
    55  
    56  	// read only field
    57  	credentials  *security.Credential
    58  	selfID       NodeID
    59  	clientConfig *MessageClientConfig
    60  
    61  	enableLocalClient bool
    62  }
    63  
    64  // NewMessageRouterWithLocalClient creates a new MessageRouter with a local client.
    65  func NewMessageRouterWithLocalClient(selfID NodeID, credentials *security.Credential, clientConfig *MessageClientConfig) *messageRouterImpl {
    66  	return newMessageRouterWithLocalClient(selfID, credentials, clientConfig, true)
    67  }
    68  
    69  // NewMessageRouter creates a new MessageRouter
    70  func NewMessageRouter(selfID NodeID, credentials *security.Credential, clientConfig *MessageClientConfig) *messageRouterImpl {
    71  	return newMessageRouterWithLocalClient(selfID, credentials, clientConfig, false)
    72  }
    73  
    74  func newMessageRouterWithLocalClient(
    75  	selfID NodeID,
    76  	credentials *security.Credential,
    77  	clientConfig *MessageClientConfig,
    78  	enableLocalClient bool,
    79  ) *messageRouterImpl {
    80  	return &messageRouterImpl{
    81  		addressMap:        make(map[NodeID]string),
    82  		clients:           make(map[NodeID]clientWrapper),
    83  		errCh:             make(chan error, 1), // one error at most
    84  		credentials:       credentials,
    85  		selfID:            selfID,
    86  		clientConfig:      clientConfig,
    87  		enableLocalClient: enableLocalClient,
    88  	}
    89  }
    90  
    91  type clientWrapper struct {
    92  	MessageClient
    93  	cancelFn context.CancelFunc
    94  }
    95  
    96  // AddPeer implements MessageRouter.
    97  func (m *messageRouterImpl) AddPeer(id NodeID, addr string) {
    98  	m.mu.Lock()
    99  	defer m.mu.Unlock()
   100  
   101  	m.addressMap[id] = addr
   102  }
   103  
   104  // RemovePeer implements MessageRouter.
   105  func (m *messageRouterImpl) RemovePeer(id NodeID) {
   106  	m.mu.Lock()
   107  	defer m.mu.Unlock()
   108  
   109  	delete(m.addressMap, id)
   110  	// The client is removed from m.clients only after it is successfully
   111  	// canceled, to prevent duplicate clients to the same target.
   112  	if clientWrapper, ok := m.clients[id]; ok {
   113  		clientWrapper.cancelFn()
   114  	}
   115  }
   116  
   117  func (m *messageRouterImpl) GetLocalChannel() <-chan RawMessageEntry {
   118  	if !m.enableLocalClient {
   119  		return nil
   120  	}
   121  	localClient := m.GetClient(m.selfID)
   122  	c, ok := localClient.(*localMessageClient)
   123  	if !ok {
   124  		log.Panic("local client is not a localMessageClient")
   125  	}
   126  	return c.localCh
   127  }
   128  
   129  // GetClient implements MessageRouter. The client will be created lazily.
   130  // It returns nil if the target peer does not exist.
   131  func (m *messageRouterImpl) GetClient(target NodeID) MessageClient {
   132  	m.mu.RLock()
   133  	// fast path
   134  	if cliWrapper, ok := m.clients[target]; ok {
   135  		m.mu.RUnlock()
   136  		return cliWrapper.MessageClient
   137  	}
   138  
   139  	// There is no ready-to-use client for target
   140  	m.mu.RUnlock()
   141  	// escalate the lock
   142  	m.mu.Lock()
   143  	defer m.mu.Unlock()
   144  
   145  	// repeats the logic in fast path after escalating the lock, since
   146  	// the lock was briefly released.
   147  	if cliWrapper, ok := m.clients[target]; ok {
   148  		return cliWrapper.MessageClient
   149  	}
   150  
   151  	var cliWrapper clientWrapper
   152  	if m.enableLocalClient && target == m.selfID {
   153  		ctx, cancel := context.WithCancel(context.Background())
   154  		cliWrapper = clientWrapper{
   155  			MessageClient: newLocalMessageClient(ctx, m.clientConfig),
   156  			cancelFn:      cancel,
   157  		}
   158  	} else {
   159  		addr, ok := m.addressMap[target]
   160  		if !ok {
   161  			log.Warn("failed to create client, no peer",
   162  				zap.String("target", target),
   163  				zap.StackSkip("stack", 1))
   164  			// There is no address for this target. We are not able to create a client.
   165  			// The client is expected to retry if the target peer is added later.
   166  			return nil
   167  		}
   168  		ctx, cancel := context.WithCancel(context.Background())
   169  		client := NewGrpcMessageClient(m.selfID, m.clientConfig)
   170  		cliWrapper = clientWrapper{
   171  			MessageClient: client,
   172  			cancelFn:      cancel,
   173  		}
   174  		m.wg.Add(1)
   175  		go func() {
   176  			defer m.wg.Done()
   177  			defer cancel()
   178  			err := client.Run(ctx, "tcp", addr, target, m.credentials)
   179  			if err != nil {
   180  				log.Warn("p2p client exited with error",
   181  					zap.String("addr", addr),
   182  					zap.String("targetCapture", target),
   183  					zap.Error(err))
   184  			} else {
   185  				log.Info("peer message client exited",
   186  					zap.String("addr", addr),
   187  					zap.String("targetCapture", target))
   188  			}
   189  			if errors.Cause(err) != context.Canceled {
   190  				// Send the error to the error channel.
   191  				select {
   192  				case m.errCh <- err:
   193  				default:
   194  					// We allow an error to be lost in case the channel is full.
   195  				}
   196  			}
   197  
   198  			m.mu.Lock()
   199  			defer m.mu.Unlock()
   200  			delete(m.clients, target)
   201  		}()
   202  	}
   203  
   204  	m.clients[target] = cliWrapper
   205  	return cliWrapper.MessageClient
   206  }
   207  
   208  func (m *messageRouterImpl) Close() {
   209  	if m.isClosed.Swap(true) {
   210  		// the messageRouter is already closed
   211  		return
   212  	}
   213  
   214  	m.mu.Lock()
   215  	for _, cliWrapper := range m.clients {
   216  		cliWrapper.cancelFn()
   217  	}
   218  	m.mu.Unlock()
   219  
   220  	m.wg.Wait()
   221  }
   222  
   223  func (m *messageRouterImpl) Err() <-chan error {
   224  	return m.errCh
   225  }