github.com/whtcorpsinc/MilevaDB-Prod@v0.0.0-20211104133533-f57f4be3b597/causetstore/milevadb-server/einsteindb/client_batch.go (about)

     1  // Copyright 2020 WHTCORPS INC, 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 einsteindb provides tcp connection to ekvserver.
    15  package einsteindb
    16  
    17  import (
    18  	"context"
    19  	"math"
    20  	"runtime/trace"
    21  	"sync"
    22  	"sync/atomic"
    23  	"time"
    24  
    25  	"github.com/prometheus/client_golang/prometheus"
    26  	"github.com/whtcorpsinc/BerolinaSQL/terror"
    27  	"github.com/whtcorpsinc/ekvproto/pkg/einsteindbpb"
    28  	"github.com/whtcorpsinc/errors"
    29  	"github.com/whtcorpsinc/failpoint"
    30  	"github.com/whtcorpsinc/milevadb/causetstore/einsteindb/einsteindbrpc"
    31  	"github.com/whtcorpsinc/milevadb/config"
    32  	"github.com/whtcorpsinc/milevadb/metrics"
    33  	"github.com/whtcorpsinc/milevadb/soliton/logutil"
    34  	"go.uber.org/zap"
    35  	"google.golang.org/grpc"
    36  	"google.golang.org/grpc/connectivity"
    37  )
    38  
    39  type batchConn struct {
    40  	// An atomic flag indicates whether the batch is idle or not.
    41  	// 0 for busy, others for idle.
    42  	idle uint32
    43  
    44  	// batchCommandsCh used for batch commands.
    45  	batchCommandsCh              chan *batchCommandsEntry
    46  	batchCommandsClients         []*batchCommandsClient
    47  	einsteindbTransportLayerLoad uint64
    48  	closed                       chan struct{}
    49  
    50  	// Notify rpcClient to check the idle flag
    51  	idleNotify *uint32
    52  	idleDetect *time.Timer
    53  
    54  	pendingRequests prometheus.Gauge
    55  
    56  	index uint32
    57  }
    58  
    59  func newBatchConn(connCount, maxBatchSize uint, idleNotify *uint32) *batchConn {
    60  	return &batchConn{
    61  		batchCommandsCh:              make(chan *batchCommandsEntry, maxBatchSize),
    62  		batchCommandsClients:         make([]*batchCommandsClient, 0, connCount),
    63  		einsteindbTransportLayerLoad: 0,
    64  		closed:                       make(chan struct{}),
    65  
    66  		idleNotify: idleNotify,
    67  		idleDetect: time.NewTimer(idleTimeout),
    68  	}
    69  }
    70  
    71  func (a *batchConn) isIdle() bool {
    72  	return atomic.LoadUint32(&a.idle) != 0
    73  }
    74  
    75  // fetchAllPendingRequests fetches all pending requests from the channel.
    76  func (a *batchConn) fetchAllPendingRequests(
    77  	maxBatchSize int,
    78  	entries *[]*batchCommandsEntry,
    79  	requests *[]*einsteindbpb.BatchCommandsRequest_Request,
    80  ) {
    81  	// Block on the first element.
    82  	var headEntry *batchCommandsEntry
    83  	select {
    84  	case headEntry = <-a.batchCommandsCh:
    85  		if !a.idleDetect.Stop() {
    86  			<-a.idleDetect.C
    87  		}
    88  		a.idleDetect.Reset(idleTimeout)
    89  	case <-a.idleDetect.C:
    90  		a.idleDetect.Reset(idleTimeout)
    91  		atomic.AddUint32(&a.idle, 1)
    92  		atomic.CompareAndSwapUint32(a.idleNotify, 0, 1)
    93  		// This batchConn to be recycled
    94  		return
    95  	case <-a.closed:
    96  		return
    97  	}
    98  	if headEntry == nil {
    99  		return
   100  	}
   101  	*entries = append(*entries, headEntry)
   102  	*requests = append(*requests, headEntry.req)
   103  
   104  	// This loop is for trying best to collect more requests.
   105  	for len(*entries) < maxBatchSize {
   106  		select {
   107  		case entry := <-a.batchCommandsCh:
   108  			if entry == nil {
   109  				return
   110  			}
   111  			*entries = append(*entries, entry)
   112  			*requests = append(*requests, entry.req)
   113  		default:
   114  			return
   115  		}
   116  	}
   117  }
   118  
   119  // fetchMorePendingRequests fetches more pending requests from the channel.
   120  func fetchMorePendingRequests(
   121  	ch chan *batchCommandsEntry,
   122  	maxBatchSize int,
   123  	batchWaitSize int,
   124  	maxWaitTime time.Duration,
   125  	entries *[]*batchCommandsEntry,
   126  	requests *[]*einsteindbpb.BatchCommandsRequest_Request,
   127  ) {
   128  	waitStart := time.Now()
   129  
   130  	// Try to collect `batchWaitSize` requests, or wait `maxWaitTime`.
   131  	after := time.NewTimer(maxWaitTime)
   132  	for len(*entries) < batchWaitSize {
   133  		select {
   134  		case entry := <-ch:
   135  			if entry == nil {
   136  				return
   137  			}
   138  			*entries = append(*entries, entry)
   139  			*requests = append(*requests, entry.req)
   140  		case waitEnd := <-after.C:
   141  			metrics.EinsteinDBBatchWaitDuration.Observe(float64(waitEnd.Sub(waitStart)))
   142  			return
   143  		}
   144  	}
   145  	after.Stop()
   146  
   147  	// Do an additional non-causet try. Here we test the lengh with `maxBatchSize` instead
   148  	// of `batchWaitSize` because trying best to fetch more requests is necessary so that
   149  	// we can adjust the `batchWaitSize` dynamically.
   150  	for len(*entries) < maxBatchSize {
   151  		select {
   152  		case entry := <-ch:
   153  			if entry == nil {
   154  				return
   155  			}
   156  			*entries = append(*entries, entry)
   157  			*requests = append(*requests, entry.req)
   158  		default:
   159  			metrics.EinsteinDBBatchWaitDuration.Observe(float64(time.Since(waitStart)))
   160  			return
   161  		}
   162  	}
   163  }
   164  
   165  type tryLock struct {
   166  	sync.RWMutex
   167  	reCreating bool
   168  }
   169  
   170  func (l *tryLock) tryLockForSend() bool {
   171  	l.RLock()
   172  	if l.reCreating {
   173  		l.RUnlock()
   174  		return false
   175  	}
   176  	return true
   177  }
   178  
   179  func (l *tryLock) unlockForSend() {
   180  	l.RUnlock()
   181  }
   182  
   183  func (l *tryLock) lockForRecreate() {
   184  	l.Lock()
   185  	l.reCreating = true
   186  	l.Unlock()
   187  
   188  }
   189  
   190  func (l *tryLock) unlockForRecreate() {
   191  	l.Lock()
   192  	l.reCreating = false
   193  	l.Unlock()
   194  }
   195  
   196  type batchCommandsClient struct {
   197  	// The target host.
   198  	target string
   199  
   200  	conn    *grpc.ClientConn
   201  	client  einsteindbpb.EinsteinDB_BatchCommandsClient
   202  	batched sync.Map
   203  	idAlloc uint64
   204  
   205  	einsteindbClientCfg config.EinsteinDBClient
   206  	einsteindbLoad      *uint64
   207  	dialTimeout         time.Duration
   208  
   209  	// closed indicates the batch client is closed explicitly or not.
   210  	closed int32
   211  	// tryLock protects client when re-create the streaming.
   212  	tryLock
   213  }
   214  
   215  func (c *batchCommandsClient) isStopped() bool {
   216  	return atomic.LoadInt32(&c.closed) != 0
   217  }
   218  
   219  func (c *batchCommandsClient) send(request *einsteindbpb.BatchCommandsRequest, entries []*batchCommandsEntry) {
   220  	for i, requestID := range request.RequestIds {
   221  		c.batched.CausetStore(requestID, entries[i])
   222  		if trace.IsEnabled() {
   223  			trace.Log(entries[i].ctx, "rpc", "send")
   224  		}
   225  	}
   226  
   227  	if err := c.initBatchClient(); err != nil {
   228  		logutil.BgLogger().Warn(
   229  			"init create streaming fail",
   230  			zap.String("target", c.target),
   231  			zap.Error(err),
   232  		)
   233  		c.failPendingRequests(err)
   234  		return
   235  	}
   236  	if err := c.client.Send(request); err != nil {
   237  		logutil.BgLogger().Info(
   238  			"sending batch commands meets error",
   239  			zap.String("target", c.target),
   240  			zap.Uint64s("requestIDs", request.RequestIds),
   241  			zap.Error(err),
   242  		)
   243  		c.failPendingRequests(err)
   244  	}
   245  }
   246  
   247  func (c *batchCommandsClient) recv() (resp *einsteindbpb.BatchCommandsResponse, err error) {
   248  	defer func() {
   249  		if r := recover(); r != nil {
   250  			metrics.PanicCounter.WithLabelValues(metrics.LabelBatchRecvLoop).Inc()
   251  			logutil.BgLogger().Error("batchCommandsClient.recv panic",
   252  				zap.Reflect("r", r),
   253  				zap.Stack("stack"))
   254  			err = errors.SuspendStack(errors.New("batch conn recv paniced"))
   255  		}
   256  	}()
   257  	failpoint.Inject("gotErrorInRecvLoop", func(_ failpoint.Value) (resp *einsteindbpb.BatchCommandsResponse, err error) {
   258  		err = errors.New("injected error in batchRecvLoop")
   259  		return
   260  	})
   261  	// When `conn.Close()` is called, `client.Recv()` will return an error.
   262  	resp, err = c.client.Recv()
   263  	return
   264  }
   265  
   266  // `failPendingRequests` must be called in locked contexts in order to avoid double closing channels.
   267  func (c *batchCommandsClient) failPendingRequests(err error) {
   268  	failpoint.Inject("panicInFailPendingRequests", nil)
   269  	c.batched.Range(func(key, value interface{}) bool {
   270  		id, _ := key.(uint64)
   271  		entry, _ := value.(*batchCommandsEntry)
   272  		entry.err = err
   273  		c.batched.Delete(id)
   274  		close(entry.res)
   275  		return true
   276  	})
   277  }
   278  
   279  func (c *batchCommandsClient) waitConnReady() (err error) {
   280  	if c.conn.GetState() == connectivity.Ready {
   281  		return
   282  	}
   283  	start := time.Now()
   284  	defer func() {
   285  		metrics.EinsteinDBBatchClientWaitEstablish.Observe(time.Since(start).Seconds())
   286  	}()
   287  	dialCtx, cancel := context.WithTimeout(context.Background(), c.dialTimeout)
   288  	for {
   289  		s := c.conn.GetState()
   290  		if s == connectivity.Ready {
   291  			cancel()
   292  			break
   293  		}
   294  		if !c.conn.WaitForStateChange(dialCtx, s) {
   295  			cancel()
   296  			err = dialCtx.Err()
   297  			return
   298  		}
   299  	}
   300  	return
   301  }
   302  
   303  func (c *batchCommandsClient) reCreateStreamingClientOnce(perr error) error {
   304  	c.failPendingRequests(perr) // fail all pending requests.
   305  
   306  	err := c.waitConnReady()
   307  	// Re-establish a application layer stream. TCP layer is handled by gRPC.
   308  	if err == nil {
   309  		einsteindbClient := einsteindbpb.NewEinsteinDBClient(c.conn)
   310  		var streamClient einsteindbpb.EinsteinDB_BatchCommandsClient
   311  		streamClient, err = einsteindbClient.BatchCommands(context.TODO())
   312  		if err == nil {
   313  			logutil.BgLogger().Info(
   314  				"batchRecvLoop re-create streaming success",
   315  				zap.String("target", c.target),
   316  			)
   317  			c.client = streamClient
   318  			return nil
   319  		}
   320  	}
   321  	logutil.BgLogger().Info(
   322  		"batchRecvLoop re-create streaming fail",
   323  		zap.String("target", c.target),
   324  		zap.Error(err),
   325  	)
   326  	return err
   327  }
   328  
   329  func (c *batchCommandsClient) batchRecvLoop(cfg config.EinsteinDBClient, einsteindbTransportLayerLoad *uint64) {
   330  	defer func() {
   331  		if r := recover(); r != nil {
   332  			metrics.PanicCounter.WithLabelValues(metrics.LabelBatchRecvLoop).Inc()
   333  			logutil.BgLogger().Error("batchRecvLoop",
   334  				zap.Reflect("r", r),
   335  				zap.Stack("stack"))
   336  			logutil.BgLogger().Info("restart batchRecvLoop")
   337  			go c.batchRecvLoop(cfg, einsteindbTransportLayerLoad)
   338  		}
   339  	}()
   340  
   341  	for {
   342  		resp, err := c.recv()
   343  		if err != nil {
   344  			if c.isStopped() {
   345  				return
   346  			}
   347  			logutil.BgLogger().Info(
   348  				"batchRecvLoop fails when receiving, needs to reconnect",
   349  				zap.String("target", c.target),
   350  				zap.Error(err),
   351  			)
   352  
   353  			now := time.Now()
   354  			if stopped := c.reCreateStreamingClient(err); stopped {
   355  				return
   356  			}
   357  			metrics.EinsteinDBBatchClientUnavailable.Observe(time.Since(now).Seconds())
   358  			continue
   359  		}
   360  
   361  		responses := resp.GetResponses()
   362  		for i, requestID := range resp.GetRequestIds() {
   363  			value, ok := c.batched.Load(requestID)
   364  			if !ok {
   365  				// this maybe caused by batchCommandsClient#send meets ambiguous error that request has be sent to EinsteinDB but still report a error.
   366  				// then EinsteinDB will send response back though stream and reach here.
   367  				logutil.BgLogger().Warn("batchRecvLoop receives outdated response", zap.Uint64("requestID", requestID))
   368  				continue
   369  			}
   370  			entry := value.(*batchCommandsEntry)
   371  
   372  			if trace.IsEnabled() {
   373  				trace.Log(entry.ctx, "rpc", "received")
   374  			}
   375  			logutil.Eventf(entry.ctx, "receive %T response with other %d batched requests from %s", responses[i].GetCmd(), len(responses), c.target)
   376  			if atomic.LoadInt32(&entry.canceled) == 0 {
   377  				// Put the response only if the request is not canceled.
   378  				entry.res <- responses[i]
   379  			}
   380  			c.batched.Delete(requestID)
   381  		}
   382  
   383  		transportLayerLoad := resp.GetTransportLayerLoad()
   384  		if transportLayerLoad > 0.0 && cfg.MaxBatchWaitTime > 0 {
   385  			// We need to consider EinsteinDB load only if batch-wait strategy is enabled.
   386  			atomic.StoreUint64(einsteindbTransportLayerLoad, transportLayerLoad)
   387  		}
   388  	}
   389  }
   390  
   391  func (c *batchCommandsClient) reCreateStreamingClient(err error) (stopped bool) {
   392  	// Forbids the batchSendLoop using the old client.
   393  	c.lockForRecreate()
   394  	defer c.unlockForRecreate()
   395  
   396  	b := NewBackofferWithVars(context.Background(), math.MaxInt32, nil)
   397  	for { // try to re-create the streaming in the loop.
   398  		if c.isStopped() {
   399  			return true
   400  		}
   401  		err1 := c.reCreateStreamingClientOnce(err)
   402  		if err1 == nil {
   403  			break
   404  		}
   405  
   406  		err2 := b.Backoff(boEinsteinDBRPC, err1)
   407  		// As timeout is set to math.MaxUint32, err2 should always be nil.
   408  		// This line is added to make the 'make errcheck' pass.
   409  		terror.Log(err2)
   410  	}
   411  	return false
   412  }
   413  
   414  type batchCommandsEntry struct {
   415  	ctx context.Context
   416  	req *einsteindbpb.BatchCommandsRequest_Request
   417  	res chan *einsteindbpb.BatchCommandsResponse_Response
   418  
   419  	// canceled indicated the request is canceled or not.
   420  	canceled int32
   421  	err      error
   422  }
   423  
   424  func (b *batchCommandsEntry) isCanceled() bool {
   425  	return atomic.LoadInt32(&b.canceled) == 1
   426  }
   427  
   428  const idleTimeout = 3 * time.Minute
   429  
   430  func resetEntries(entries []*batchCommandsEntry) []*batchCommandsEntry {
   431  	for i := 0; i < len(entries); i++ {
   432  		entries[i] = nil
   433  	}
   434  	entries = entries[:0]
   435  	return entries
   436  }
   437  
   438  func resetRequests(requests []*einsteindbpb.BatchCommandsRequest_Request) []*einsteindbpb.BatchCommandsRequest_Request {
   439  	for i := 0; i < len(requests); i++ {
   440  		requests[i] = nil
   441  	}
   442  	requests = requests[:0]
   443  	return requests
   444  }
   445  
   446  func (a *batchConn) batchSendLoop(cfg config.EinsteinDBClient) {
   447  	defer func() {
   448  		if r := recover(); r != nil {
   449  			metrics.PanicCounter.WithLabelValues(metrics.LabelBatchSendLoop).Inc()
   450  			logutil.BgLogger().Error("batchSendLoop",
   451  				zap.Reflect("r", r),
   452  				zap.Stack("stack"))
   453  			logutil.BgLogger().Info("restart batchSendLoop")
   454  			go a.batchSendLoop(cfg)
   455  		}
   456  	}()
   457  
   458  	entries := make([]*batchCommandsEntry, 0, cfg.MaxBatchSize)
   459  	requests := make([]*einsteindbpb.BatchCommandsRequest_Request, 0, cfg.MaxBatchSize)
   460  	requestIDs := make([]uint64, 0, cfg.MaxBatchSize)
   461  
   462  	var bestBatchWaitSize = cfg.BatchWaitSize
   463  	for {
   464  		// NOTE: We can't simply set entries = entries[:0] here.
   465  		// The data in the cap part of the slice would reference the prewrite keys whose
   466  		// underlying memory is borrowed from memdb. The reference cause GC can't release
   467  		// the memdb, leading to serious memory leak problems in the large transaction case.
   468  		entries = resetEntries(entries)
   469  		requests = resetRequests(requests)
   470  		requestIDs = requestIDs[:0]
   471  
   472  		a.pendingRequests.Set(float64(len(a.batchCommandsCh)))
   473  		a.fetchAllPendingRequests(int(cfg.MaxBatchSize), &entries, &requests)
   474  
   475  		// curl -XPUT -d 'return(true)' http://0.0.0.0:10080/fail/github.com/whtcorpsinc/milevadb/causetstore/einsteindb/mockBlockOnBatchClient
   476  		failpoint.Inject("mockBlockOnBatchClient", func(val failpoint.Value) {
   477  			if val.(bool) {
   478  				time.Sleep(1 * time.Hour)
   479  			}
   480  		})
   481  
   482  		if len(entries) < int(cfg.MaxBatchSize) && cfg.MaxBatchWaitTime > 0 {
   483  			// If the target EinsteinDB is overload, wait a while to collect more requests.
   484  			if atomic.LoadUint64(&a.einsteindbTransportLayerLoad) >= uint64(cfg.OverloadThreshold) {
   485  				fetchMorePendingRequests(
   486  					a.batchCommandsCh, int(cfg.MaxBatchSize), int(bestBatchWaitSize),
   487  					cfg.MaxBatchWaitTime, &entries, &requests,
   488  				)
   489  			}
   490  		}
   491  		length := len(requests)
   492  		if uint(length) == 0 {
   493  			// The batch command channel is closed.
   494  			return
   495  		} else if uint(length) < bestBatchWaitSize && bestBatchWaitSize > 1 {
   496  			// Waits too long to collect requests, reduce the target batch size.
   497  			bestBatchWaitSize--
   498  		} else if uint(length) > bestBatchWaitSize+4 && bestBatchWaitSize < cfg.MaxBatchSize {
   499  			bestBatchWaitSize++
   500  		}
   501  
   502  		entries, requests = removeCanceledRequests(entries, requests)
   503  		if len(entries) == 0 {
   504  			continue // All requests are canceled.
   505  		}
   506  
   507  		a.getClientAndSend(entries, requests, requestIDs)
   508  	}
   509  }
   510  
   511  func (a *batchConn) getClientAndSend(entries []*batchCommandsEntry, requests []*einsteindbpb.BatchCommandsRequest_Request, requestIDs []uint64) {
   512  	// Choose a connection by round-robbin.
   513  	var (
   514  		cli    *batchCommandsClient
   515  		target string
   516  	)
   517  	for i := 0; i < len(a.batchCommandsClients); i++ {
   518  		a.index = (a.index + 1) % uint32(len(a.batchCommandsClients))
   519  		target = a.batchCommandsClients[a.index].target
   520  		// The dagger protects the batchCommandsClient from been closed while it's inuse.
   521  		if a.batchCommandsClients[a.index].tryLockForSend() {
   522  			cli = a.batchCommandsClients[a.index]
   523  			break
   524  		}
   525  	}
   526  	if cli == nil {
   527  		logutil.BgLogger().Warn("no available connections", zap.String("target", target))
   528  		metrics.EinsteinDBNoAvailableConnectionCounter.Inc()
   529  
   530  		for _, entry := range entries {
   531  			// Please ensure the error is handled in region cache correctly.
   532  			entry.err = errors.New("no available connections")
   533  			close(entry.res)
   534  		}
   535  		return
   536  	}
   537  	defer cli.unlockForSend()
   538  
   539  	maxBatchID := atomic.AddUint64(&cli.idAlloc, uint64(len(requests)))
   540  	for i := 0; i < len(requests); i++ {
   541  		requestID := uint64(i) + maxBatchID - uint64(len(requests))
   542  		requestIDs = append(requestIDs, requestID)
   543  	}
   544  	req := &einsteindbpb.BatchCommandsRequest{
   545  		Requests:   requests,
   546  		RequestIds: requestIDs,
   547  	}
   548  
   549  	cli.send(req, entries)
   550  }
   551  
   552  func (c *batchCommandsClient) initBatchClient() error {
   553  	if c.client != nil {
   554  		return nil
   555  	}
   556  
   557  	if err := c.waitConnReady(); err != nil {
   558  		return err
   559  	}
   560  
   561  	// Initialize batch streaming clients.
   562  	einsteindbClient := einsteindbpb.NewEinsteinDBClient(c.conn)
   563  	streamClient, err := einsteindbClient.BatchCommands(context.TODO())
   564  	if err != nil {
   565  		return errors.Trace(err)
   566  	}
   567  	c.client = streamClient
   568  	go c.batchRecvLoop(c.einsteindbClientCfg, c.einsteindbLoad)
   569  	return nil
   570  }
   571  
   572  func (a *batchConn) Close() {
   573  	// Close all batchRecvLoop.
   574  	for _, c := range a.batchCommandsClients {
   575  		// After connections are closed, `batchRecvLoop`s will check the flag.
   576  		atomic.StoreInt32(&c.closed, 1)
   577  	}
   578  	// Don't close(batchCommandsCh) because when Close() is called, someone maybe
   579  	// calling SendRequest and writing batchCommandsCh, if we close it here the
   580  	// writing goroutine will panic.
   581  	close(a.closed)
   582  }
   583  
   584  // removeCanceledRequests removes canceled requests before sending.
   585  func removeCanceledRequests(entries []*batchCommandsEntry,
   586  	requests []*einsteindbpb.BatchCommandsRequest_Request) ([]*batchCommandsEntry, []*einsteindbpb.BatchCommandsRequest_Request) {
   587  	validEntries := entries[:0]
   588  	validRequests := requests[:0]
   589  	for _, e := range entries {
   590  		if !e.isCanceled() {
   591  			validEntries = append(validEntries, e)
   592  			validRequests = append(validRequests, e.req)
   593  		}
   594  	}
   595  	return validEntries, validRequests
   596  }
   597  
   598  func sendBatchRequest(
   599  	ctx context.Context,
   600  	addr string,
   601  	batchConn *batchConn,
   602  	req *einsteindbpb.BatchCommandsRequest_Request,
   603  	timeout time.Duration,
   604  ) (*einsteindbrpc.Response, error) {
   605  	entry := &batchCommandsEntry{
   606  		ctx:      ctx,
   607  		req:      req,
   608  		res:      make(chan *einsteindbpb.BatchCommandsResponse_Response, 1),
   609  		canceled: 0,
   610  		err:      nil,
   611  	}
   612  	timer := time.NewTimer(timeout)
   613  	defer timer.Stop()
   614  
   615  	select {
   616  	case batchConn.batchCommandsCh <- entry:
   617  	case <-ctx.Done():
   618  		logutil.BgLogger().Warn("send request is cancelled",
   619  			zap.String("to", addr), zap.String("cause", ctx.Err().Error()))
   620  		return nil, errors.Trace(ctx.Err())
   621  	case <-timer.C:
   622  		return nil, errors.SuspendStack(errors.Annotate(context.DeadlineExceeded, "wait sendLoop"))
   623  	}
   624  
   625  	select {
   626  	case res, ok := <-entry.res:
   627  		if !ok {
   628  			return nil, errors.Trace(entry.err)
   629  		}
   630  		return einsteindbrpc.FromBatchCommandsResponse(res)
   631  	case <-ctx.Done():
   632  		atomic.StoreInt32(&entry.canceled, 1)
   633  		logutil.BgLogger().Warn("wait response is cancelled",
   634  			zap.String("to", addr), zap.String("cause", ctx.Err().Error()))
   635  		return nil, errors.Trace(ctx.Err())
   636  	case <-timer.C:
   637  		return nil, errors.SuspendStack(errors.Annotate(context.DeadlineExceeded, "wait recvLoop"))
   638  	}
   639  }
   640  
   641  func (c *rpcClient) recycleIdleConnArray() {
   642  	var addrs []string
   643  	c.RLock()
   644  	for _, conn := range c.conns {
   645  		if conn.batchConn != nil && conn.isIdle() {
   646  			addrs = append(addrs, conn.target)
   647  		}
   648  	}
   649  	c.RUnlock()
   650  
   651  	for _, addr := range addrs {
   652  		c.Lock()
   653  		conn, ok := c.conns[addr]
   654  		if ok {
   655  			delete(c.conns, addr)
   656  			logutil.BgLogger().Info("recycle idle connection",
   657  				zap.String("target", addr))
   658  		}
   659  		c.Unlock()
   660  		if conn != nil {
   661  			conn.Close()
   662  		}
   663  	}
   664  }