github.com/pingcap/tiflow@v0.0.0-20240520035814-5bf52d54e205/cdc/kv/sharedconn/conn_and_client.go (about)

     1  // Copyright 2023 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 sharedconn
    15  
    16  import (
    17  	"context"
    18  	"io"
    19  	"sort"
    20  	"strings"
    21  	"sync"
    22  	"sync/atomic"
    23  	"time"
    24  
    25  	grpc_prometheus "github.com/grpc-ecosystem/go-grpc-prometheus"
    26  	"github.com/pingcap/kvproto/pkg/cdcpb"
    27  	"github.com/pingcap/tiflow/pkg/security"
    28  	"github.com/pingcap/tiflow/pkg/util"
    29  	"google.golang.org/grpc"
    30  	"google.golang.org/grpc/backoff"
    31  	grpccodes "google.golang.org/grpc/codes"
    32  	"google.golang.org/grpc/keepalive"
    33  	"google.golang.org/grpc/metadata"
    34  	grpcstatus "google.golang.org/grpc/status"
    35  )
    36  
    37  // StatusIsEOF checks whether status is caused by client send closing.
    38  func StatusIsEOF(status *grpcstatus.Status) bool {
    39  	return status == nil ||
    40  		status.Code() == grpccodes.Canceled ||
    41  		(status.Code() == grpccodes.Unknown && status.Message() == io.EOF.Error())
    42  }
    43  
    44  // ConnAndClientPool is a pool of ConnAndClient.
    45  type ConnAndClientPool struct {
    46  	credential        *security.Credential
    47  	grpcMetrics       *grpc_prometheus.ClientMetrics
    48  	maxStreamsPerConn int
    49  
    50  	sync.Mutex
    51  	stores map[string]*connArray
    52  }
    53  
    54  // ConnAndClient indicates a connection and a EventFeedV2 client.
    55  type ConnAndClient struct {
    56  	conn   *Conn
    57  	array  *connArray
    58  	client cdcpb.ChangeData_EventFeedV2Client
    59  	closed atomic.Bool
    60  }
    61  
    62  // Conn is a connection.
    63  type Conn struct {
    64  	*grpc.ClientConn
    65  	multiplexing bool
    66  	streams      int
    67  }
    68  
    69  type connArray struct {
    70  	pool         *ConnAndClientPool
    71  	addr         string
    72  	inConnecting atomic.Bool
    73  
    74  	sync.Mutex
    75  	conns []*Conn
    76  }
    77  
    78  // NewConnAndClientPool creates a new ConnAndClientPool.
    79  func NewConnAndClientPool(
    80  	credential *security.Credential,
    81  	grpcMetrics *grpc_prometheus.ClientMetrics,
    82  	maxStreamsPerConn ...int,
    83  ) *ConnAndClientPool {
    84  	return newConnAndClientPool(credential, grpcMetrics, 1000)
    85  }
    86  
    87  func newConnAndClientPool(
    88  	credential *security.Credential,
    89  	grpcMetrics *grpc_prometheus.ClientMetrics,
    90  	maxStreamsPerConn int,
    91  ) *ConnAndClientPool {
    92  	stores := make(map[string]*connArray, 64)
    93  	return &ConnAndClientPool{
    94  		credential:        credential,
    95  		grpcMetrics:       grpcMetrics,
    96  		maxStreamsPerConn: maxStreamsPerConn,
    97  		stores:            stores,
    98  	}
    99  }
   100  
   101  // Connect connects to addr.
   102  func (c *ConnAndClientPool) Connect(ctx context.Context, addr string) (cc *ConnAndClient, err error) {
   103  	var conns *connArray
   104  	c.Lock()
   105  	if conns = c.stores[addr]; conns == nil {
   106  		conns = &connArray{pool: c, addr: addr}
   107  		c.stores[addr] = conns
   108  	}
   109  	c.Unlock()
   110  
   111  	for {
   112  		conns.Lock()
   113  		if len(conns.conns) > 0 && conns.conns[0].streams < c.maxStreamsPerConn {
   114  			break
   115  		}
   116  
   117  		conns.Unlock()
   118  		var conn *Conn
   119  		if conn, err = conns.connect(ctx); err != nil {
   120  			return
   121  		}
   122  		if conn != nil {
   123  			conns.Lock()
   124  			conns.push(conn, true)
   125  			conns.sort(true)
   126  			break
   127  		}
   128  		if err = util.Hang(ctx, time.Second); err != nil {
   129  			return
   130  		}
   131  	}
   132  
   133  	cc = &ConnAndClient{conn: conns.conns[0], array: conns}
   134  	cc.conn.streams += 1
   135  	defer func() {
   136  		conns.Unlock()
   137  		if err != nil && cc != nil {
   138  			cc.Release()
   139  			cc = nil
   140  		}
   141  	}()
   142  
   143  	rpc := cdcpb.NewChangeDataClient(cc.conn.ClientConn)
   144  	if cc.conn.multiplexing {
   145  		ctx = getContextFromFeatures(ctx, []string{rpcMetaFeatureStreamMultiplexing})
   146  		cc.client, err = rpc.EventFeedV2(ctx)
   147  	} else {
   148  		cc.client, err = rpc.EventFeed(ctx)
   149  	}
   150  	return
   151  }
   152  
   153  // Client gets an EventFeedV2 client.
   154  func (c *ConnAndClient) Client() cdcpb.ChangeData_EventFeedV2Client {
   155  	return c.client
   156  }
   157  
   158  // Multiplexing indicates whether the client can be used for multiplexing or not.
   159  func (c *ConnAndClient) Multiplexing() bool {
   160  	return c.conn.multiplexing
   161  }
   162  
   163  // Release releases a ConnAndClient object.
   164  func (c *ConnAndClient) Release() {
   165  	if c.client != nil && !c.closed.Load() {
   166  		_ = c.client.CloseSend()
   167  		c.closed.Store(true)
   168  	}
   169  	if c.conn != nil && c.array != nil {
   170  		c.array.release(c.conn, false)
   171  		c.conn = nil
   172  		c.array = nil
   173  	}
   174  }
   175  
   176  func (c *connArray) connect(ctx context.Context) (conn *Conn, err error) {
   177  	if c.inConnecting.CompareAndSwap(false, true) {
   178  		defer c.inConnecting.Store(false)
   179  		var clientConn *grpc.ClientConn
   180  		if clientConn, err = c.pool.connect(ctx, c.addr); err != nil {
   181  			return
   182  		}
   183  
   184  		rpc := cdcpb.NewChangeDataClient(clientConn)
   185  		ctx = getContextFromFeatures(ctx, []string{rpcMetaFeatureStreamMultiplexing})
   186  		var client cdcpb.ChangeData_EventFeedV2Client
   187  		if client, err = rpc.EventFeedV2(ctx); err == nil {
   188  			_ = client.CloseSend()
   189  			_, err = client.Recv()
   190  		}
   191  
   192  		status := grpcstatus.Convert(err)
   193  		if StatusIsEOF(status) {
   194  			conn = new(Conn)
   195  			conn.ClientConn = clientConn
   196  			conn.multiplexing = true
   197  			err = nil
   198  		} else if status.Code() == grpccodes.Unimplemented {
   199  			conn = new(Conn)
   200  			conn.ClientConn = clientConn
   201  			conn.multiplexing = false
   202  			err = nil
   203  		} else {
   204  			_ = clientConn.Close()
   205  		}
   206  	}
   207  	return
   208  }
   209  
   210  func (c *connArray) push(conn *Conn, locked bool) {
   211  	if !locked {
   212  		c.Lock()
   213  		defer c.Unlock()
   214  	}
   215  	c.conns = append(c.conns, conn)
   216  	c.sort(true)
   217  }
   218  
   219  func (c *connArray) release(conn *Conn, locked bool) {
   220  	if !locked {
   221  		c.Lock()
   222  		defer c.Unlock()
   223  	}
   224  	conn.streams -= 1
   225  	if conn.streams == 0 {
   226  		for i := range c.conns {
   227  			if c.conns[i] == conn {
   228  				c.conns[i] = c.conns[len(c.conns)-1]
   229  				c.conns = c.conns[:len(c.conns)-1]
   230  				break
   231  			}
   232  		}
   233  		if len(c.conns) == 0 {
   234  			c.pool.Lock()
   235  			delete(c.pool.stores, c.addr)
   236  			c.pool.Unlock()
   237  		}
   238  		_ = conn.ClientConn.Close()
   239  	}
   240  	c.sort(true)
   241  }
   242  
   243  func (c *connArray) sort(locked bool) {
   244  	if !locked {
   245  		c.Lock()
   246  		defer c.Unlock()
   247  	}
   248  	sort.Slice(c.conns, func(i, j int) bool {
   249  		return (c.conns[i].multiplexing && !c.conns[j].multiplexing) ||
   250  			(c.conns[i].multiplexing == c.conns[j].multiplexing && c.conns[i].streams < c.conns[j].streams)
   251  	})
   252  }
   253  
   254  func (c *ConnAndClientPool) connect(ctx context.Context, target string) (*grpc.ClientConn, error) {
   255  	grpcTLSOption, err := c.credential.ToGRPCDialOption()
   256  	if err != nil {
   257  		return nil, err
   258  	}
   259  
   260  	dialOptions := []grpc.DialOption{
   261  		grpcTLSOption,
   262  		grpc.WithInitialWindowSize(grpcInitialWindowSize),
   263  		grpc.WithInitialConnWindowSize(grpcInitialConnWindowSize),
   264  		grpc.WithDefaultCallOptions(grpc.MaxCallRecvMsgSize(grpcMaxCallRecvMsgSize)),
   265  		grpc.WithConnectParams(grpc.ConnectParams{
   266  			Backoff: backoff.Config{
   267  				BaseDelay:  time.Second,
   268  				Multiplier: 1.1,
   269  				Jitter:     0.1,
   270  				MaxDelay:   3 * time.Second,
   271  			},
   272  			MinConnectTimeout: 3 * time.Second,
   273  		}),
   274  		grpc.WithKeepaliveParams(keepalive.ClientParameters{
   275  			Time:                10 * time.Second,
   276  			Timeout:             3 * time.Second,
   277  			PermitWithoutStream: true,
   278  		}),
   279  	}
   280  
   281  	if c.grpcMetrics != nil {
   282  		dialOptions = append(dialOptions, grpc.WithUnaryInterceptor(c.grpcMetrics.UnaryClientInterceptor()))
   283  		dialOptions = append(dialOptions, grpc.WithStreamInterceptor(c.grpcMetrics.StreamClientInterceptor()))
   284  	}
   285  
   286  	return grpc.DialContext(ctx, target, dialOptions...)
   287  }
   288  
   289  const (
   290  	grpcInitialWindowSize     = (1 << 16) - 1
   291  	grpcInitialConnWindowSize = 1 << 23
   292  	grpcMaxCallRecvMsgSize    = 1 << 28
   293  
   294  	rpcMetaFeaturesKey string = "features"
   295  	rpcMetaFeaturesSep string = ","
   296  
   297  	// this feature supports these interactions with TiKV sides:
   298  	// 1. in one GRPC stream, TiKV will merge resolved timestamps into several buckets based on
   299  	//    `RequestId`s. For example, region 100 and 101 have been subscribed twice with `RequestId`
   300  	//    1 and 2, TiKV will sends a ResolvedTs message
   301  	//    [{"RequestId": 1, "regions": [100, 101]}, {"RequestId": 2, "regions": [100, 101]}]
   302  	//    to the TiCDC client.
   303  	// 2. TiCDC can deregister all regions with a same request ID by specifying the `RequestId`.
   304  	rpcMetaFeatureStreamMultiplexing string = "stream-multiplexing"
   305  )
   306  
   307  func getContextFromFeatures(ctx context.Context, features []string) context.Context {
   308  	return metadata.NewOutgoingContext(
   309  		ctx,
   310  		metadata.New(map[string]string{
   311  			rpcMetaFeaturesKey: strings.Join(features, rpcMetaFeaturesSep),
   312  		}),
   313  	)
   314  }