github.com/tsuna/gohbase@v0.0.0-20250731002811-4ffcadfba63e/region/client.go (about)

     1  // Copyright (C) 2015  The GoHBase Authors.  All rights reserved.
     2  // This file is part of GoHBase.
     3  // Use of this source code is governed by the Apache License 2.0
     4  // that can be found in the COPYING file.
     5  
     6  package region
     7  
     8  import (
     9  	"bufio"
    10  	"context"
    11  	"encoding/binary"
    12  	"encoding/json"
    13  	"errors"
    14  	"fmt"
    15  	"io"
    16  	"log/slog"
    17  	"net"
    18  	"strings"
    19  	"sync"
    20  	"sync/atomic"
    21  	"time"
    22  
    23  	"github.com/prometheus/client_golang/prometheus"
    24  
    25  	"github.com/tsuna/gohbase/hrpc"
    26  	"github.com/tsuna/gohbase/pb"
    27  	"google.golang.org/protobuf/encoding/protowire"
    28  	"google.golang.org/protobuf/proto"
    29  )
    30  
    31  // ClientType is a type alias to represent the type of this region client
    32  type ClientType string
    33  
    34  type canDeserializeCellBlocks interface {
    35  	// DeserializeCellBlocks populates passed protobuf message with results
    36  	// deserialized from the reader and returns number of bytes read or error.
    37  	DeserializeCellBlocks(proto.Message, []byte) (uint32, error)
    38  }
    39  
    40  type canSerializeCellBlocks interface {
    41  	// SerializeCellBlocks serializes RPC into protobuf for metadata
    42  	// as well as cellblocks for payload.
    43  	SerializeCellBlocks([][]byte) (proto.Message, [][]byte, uint32)
    44  	// CellBlocksEnabled returns true if cellblocks are enabled for this RPC
    45  	CellBlocksEnabled() bool
    46  }
    47  
    48  var (
    49  	// ErrMissingCallID is used when HBase sends us a response message for a
    50  	// request that we didn't send
    51  	ErrMissingCallID = ServerError{errors.New("got a response with a nonsensical call ID")}
    52  
    53  	// ErrClientClosed is returned to rpcs when Close() is called or when client
    54  	// died because of failed send or receive
    55  	ErrClientClosed = ServerError{errors.New("client is closed")}
    56  
    57  	// If a Java exception listed here is returned by HBase, the client should
    58  	// reestablish region and attempt to resend the RPC message, potentially via
    59  	// a different region client.
    60  	// The value of exception should be contained in the stack trace.
    61  	javaRegionExceptions = map[string]string{
    62  		"org.apache.hadoop.hbase.NotServingRegionException":       "",
    63  		"org.apache.hadoop.hbase.exceptions.RegionMovedException": "",
    64  		"java.io.IOException": "Cannot append; log is closed",
    65  	}
    66  
    67  	// If a Java exception listed here is returned by HBase, the client should
    68  	// backoff and resend the RPC message to the same region and region server.
    69  	// The value of exception should be contained in the stack trace.
    70  	javaRetryableExceptions = map[string]string{
    71  		"org.apache.hadoop.hbase.CallQueueTooBigException":          "",
    72  		"org.apache.hadoop.hbase.exceptions.RegionOpeningException": "",
    73  		"org.apache.hadoop.hbase.quotas.RpcThrottlingException":     "",
    74  		"org.apache.hadoop.hbase.RetryImmediatelyException":         "",
    75  		"org.apache.hadoop.hbase.RegionTooBusyException":            "",
    76  		"org.apache.hadoop.hbase.PleaseHoldException":               "",
    77  	}
    78  
    79  	// If a Java exception listed here is returned by HBase, the RegionClient
    80  	// will be closed and a new one should be established.
    81  	// The value of exception should be contained in the stack trace.
    82  	javaServerExceptions = map[string]string{
    83  		"org.apache.hadoop.hbase.regionserver.RegionServerAbortedException": "",
    84  		"org.apache.hadoop.hbase.regionserver.RegionServerStoppedException": "",
    85  		"org.apache.hadoop.hbase.exceptions.MasterStoppedException":         "",
    86  		"org.apache.hadoop.hbase.ipc.ServerNotRunningYetException":          "",
    87  	}
    88  )
    89  
    90  const (
    91  	//DefaultLookupTimeout is the default region lookup timeout
    92  	DefaultLookupTimeout = 30 * time.Second
    93  	//DefaultReadTimeout is the default region read timeout
    94  	DefaultReadTimeout = 30 * time.Second
    95  	// RegionClient is a ClientType that means this will be a normal client
    96  	RegionClient = ClientType("ClientService")
    97  
    98  	// MasterClient is a ClientType that means this client will talk to the
    99  	// master server
   100  	MasterClient = ClientType("MasterService")
   101  )
   102  
   103  var bufferPool sync.Pool
   104  
   105  func newBuffer(size int) []byte {
   106  	v := bufferPool.Get()
   107  	var b []byte
   108  	if v != nil {
   109  		b = v.([]byte)
   110  	}
   111  	return append(b[:0], make([]byte, size)...)
   112  }
   113  
   114  func freeBuffer(b []byte) {
   115  	bufferPool.Put(b[:0]) // nolint:staticcheck
   116  }
   117  
   118  // ServerError is an error that this region.Client can't recover from.
   119  // The connection to the RegionServer has to be closed and all queued and
   120  // outstanding RPCs will be failed / retried.
   121  type ServerError struct {
   122  	error
   123  }
   124  
   125  func formatErr(e interface{}, err error) string {
   126  	if err == nil {
   127  		return fmt.Sprintf("%T", e)
   128  	}
   129  	return fmt.Sprintf("%T: %s", e, err.Error())
   130  }
   131  
   132  func (e ServerError) Error() string {
   133  	return formatErr(e, e.error)
   134  }
   135  
   136  // RetryableError is an error that indicates the RPC should be retried after backoff
   137  // because the error is transient (e.g. a region being momentarily unavailable).
   138  type RetryableError struct {
   139  	error
   140  }
   141  
   142  func (e RetryableError) Error() string {
   143  	return formatErr(e, e.error)
   144  }
   145  
   146  // NotServingRegionError is an error that indicates the client should
   147  // reestablish the region and retry the RPC potentially via a different client
   148  type NotServingRegionError struct {
   149  	error
   150  }
   151  
   152  func (e NotServingRegionError) Error() string {
   153  	return formatErr(e, e.error)
   154  }
   155  
   156  // client manages a connection to a RegionServer.
   157  type client struct {
   158  	connM sync.Mutex
   159  	conn  net.Conn
   160  
   161  	// Address of the RegionServer.
   162  	addr  string
   163  	ctype ClientType
   164  
   165  	// dialOnce used for concurrent calls to Dial
   166  	dialOnce sync.Once
   167  	// failOnce used for concurrent calls to fail
   168  	failOnce sync.Once
   169  
   170  	rpcs chan []hrpc.Call
   171  	done chan struct{}
   172  
   173  	// sent contains the mapping of sent call IDs to RPC calls, so that when
   174  	// a response is received it can be tied to the correct RPC
   175  	sentM sync.Mutex // protects sent
   176  	sent  map[uint32]hrpc.Call
   177  
   178  	// inFlight is number of rpcs sent to regionserver awaiting response
   179  	inFlightM sync.Mutex // protects inFlight and SetReadDeadline
   180  	inFlight  uint32
   181  
   182  	id uint32
   183  
   184  	rpcQueueSize  int
   185  	flushInterval time.Duration
   186  	effectiveUser string
   187  
   188  	// readTimeout is the maximum amount of time to wait for regionserver reply
   189  	readTimeout time.Duration
   190  
   191  	// compressor for cellblocks. if nil, then no compression
   192  	compressor *compressor
   193  
   194  	// dialer is used to connect to region servers in non-standard ways
   195  	dialer func(ctx context.Context, network, addr string) (net.Conn, error)
   196  
   197  	logger *slog.Logger
   198  }
   199  
   200  // QueueRPC will add an rpc call to the queue for processing by the writer goroutine
   201  func (c *client) QueueRPC(rpc hrpc.Call) {
   202  	if c.rpcQueueSize > 1 && hrpc.CanBatch(rpc) {
   203  		c.QueueBatch(rpc.Context(), []hrpc.Call{rpc})
   204  	} else {
   205  		select {
   206  		case <-c.done:
   207  			// An unrecoverable error has occured,
   208  			// region client has been stopped,
   209  			// don't send rpcs
   210  			returnResult(rpc, nil, ErrClientClosed)
   211  		case <-rpc.Context().Done():
   212  			// If the deadline has been exceeded, don't bother sending the
   213  			// request. The function that placed the RPC in our queue should
   214  			// stop waiting for a result and return an error.
   215  		default:
   216  			if err := c.trySend(rpc); err != nil {
   217  				returnResult(rpc, nil, err)
   218  			}
   219  		}
   220  	}
   221  }
   222  
   223  // QueueBatch will add rpcs to the queue for processing by the writer
   224  // goroutine
   225  func (c *client) QueueBatch(ctx context.Context, rpcs []hrpc.Call) {
   226  	select {
   227  	case <-ctx.Done():
   228  	case <-c.done:
   229  		// return error results
   230  		res := hrpc.RPCResult{Error: ErrClientClosed}
   231  		for _, c := range rpcs {
   232  			c.ResultChan() <- res
   233  		}
   234  	case c.rpcs <- rpcs:
   235  	}
   236  }
   237  
   238  // Close asks this region.Client to close its connection to the RegionServer.
   239  // All queued and outstanding RPCs, if any, will be failed as if a connection
   240  // error had happened.
   241  func (c *client) Close() {
   242  	c.fail(ErrClientClosed)
   243  }
   244  
   245  // Addr returns address of the region server the client is connected to
   246  func (c *client) Addr() string {
   247  	return c.addr
   248  }
   249  
   250  // String returns a string represintation of the current region client
   251  func (c *client) String() string {
   252  	return fmt.Sprintf("RegionClient{Addr: %s}", c.addr)
   253  }
   254  
   255  func (c *client) inFlightUp() error {
   256  	c.inFlightM.Lock()
   257  	c.inFlight++
   258  	// we expect that at least the last request can be completed within readTimeout
   259  	if err := c.conn.SetReadDeadline(time.Now().Add(c.readTimeout)); err != nil {
   260  		c.inFlightM.Unlock()
   261  		return err
   262  	}
   263  	c.inFlightM.Unlock()
   264  	return nil
   265  }
   266  
   267  func (c *client) inFlightDown() error {
   268  	c.inFlightM.Lock()
   269  	c.inFlight--
   270  	// reset read timeout if we are not waiting for any responses
   271  	// in order to prevent from closing this client if there are no request
   272  	if c.inFlight == 0 {
   273  		if err := c.conn.SetReadDeadline(time.Time{}); err != nil {
   274  			c.inFlightM.Unlock()
   275  			return err
   276  		}
   277  	}
   278  	c.inFlightM.Unlock()
   279  	return nil
   280  }
   281  
   282  func (c *client) fail(err error) {
   283  	c.failOnce.Do(func() {
   284  		if err != ErrClientClosed {
   285  			c.logger.Error("error occured, closing region client", "client", c, "err", err)
   286  		}
   287  
   288  		// we don't close c.rpcs channel to make it block in select of QueueRPC
   289  		// and avoid dealing with synchronization of closing it while someone
   290  		// might be sending to it. Go's GC will take care of it.
   291  
   292  		// tell goroutines to stop
   293  		close(c.done)
   294  
   295  		// close connection to the regionserver
   296  		// to let it know that we can't receive anymore
   297  		// and fail all the rpcs being sent
   298  		if c.conn != nil {
   299  			c.conn.Close()
   300  		}
   301  
   302  		c.failSentRPCs()
   303  	})
   304  }
   305  
   306  func (c *client) failSentRPCs() {
   307  	// channel is closed, clean up awaiting rpcs
   308  	c.sentM.Lock()
   309  	sent := c.sent
   310  	c.sent = make(map[uint32]hrpc.Call)
   311  	c.sentM.Unlock()
   312  
   313  	c.logger.Debug("failing awaiting RPCs", "client", c, "count", len(sent))
   314  
   315  	// send error to awaiting rpcs
   316  	for _, rpc := range sent {
   317  		returnResult(rpc, nil, ErrClientClosed)
   318  	}
   319  }
   320  
   321  func (c *client) registerRPC(rpc hrpc.Call) uint32 {
   322  	currID := atomic.AddUint32(&c.id, 1)
   323  	c.sentM.Lock()
   324  	c.sent[currID] = rpc
   325  	c.sentM.Unlock()
   326  	return currID
   327  }
   328  
   329  func (c *client) unregisterRPC(id uint32) hrpc.Call {
   330  	c.sentM.Lock()
   331  	rpc := c.sent[id]
   332  	delete(c.sent, id)
   333  	c.sentM.Unlock()
   334  	return rpc
   335  }
   336  
   337  func (c *client) processRPCs() {
   338  	// TODO: flush when the size is too large
   339  	// TODO: if multi has only one call, send that call instead
   340  	m := newMulti(c.rpcQueueSize)
   341  	defer func() {
   342  		m.returnResults(nil, ErrClientClosed)
   343  	}()
   344  
   345  	flush := func(reason string) {
   346  		c.logger.Debug("flushing MultiRequest", "len", m.len(), "addr", c.Addr())
   347  
   348  		flushReasonCount.With(prometheus.Labels{
   349  			"reason": reason,
   350  		}).Inc()
   351  		flushSize.WithLabelValues(c.Addr()).Observe(float64(m.len()))
   352  
   353  		if err := c.trySend(m); err != nil {
   354  			m.returnResults(nil, err)
   355  		}
   356  
   357  		// Start preparing for the next batch
   358  		m = newMulti(c.rpcQueueSize)
   359  	}
   360  
   361  	for {
   362  		// first loop is to accomodate request heavy workload
   363  		// it will batch as long as conccurent writers are sending
   364  		// new rpcs or until multi is filled up
   365  		for {
   366  			select {
   367  			case <-c.done:
   368  				return
   369  			case rpcs := <-c.rpcs:
   370  				// have things queued up, batch them
   371  				if !m.add(rpcs) {
   372  					// can still put more rpcs into batch
   373  					continue
   374  				}
   375  			default:
   376  				// no more rpcs queued up
   377  			}
   378  			break
   379  		}
   380  
   381  		if l := m.len(); l == 0 {
   382  			// wait for the next batch
   383  			select {
   384  			case <-c.done:
   385  				return
   386  			case rpcs := <-c.rpcs:
   387  				m.add(rpcs)
   388  			}
   389  			continue
   390  		} else if l >= c.rpcQueueSize || c.flushInterval == 0 {
   391  			// batch is full, flush
   392  			flush("queue full")
   393  			continue
   394  		}
   395  
   396  		// second loop is to accomodate less frequent callers
   397  		// that would like to maximize their batches at the expense
   398  		// of waiting for flushInteval
   399  		timer := time.NewTimer(c.flushInterval)
   400  		reason := ""
   401  		for {
   402  			select {
   403  			case <-c.done:
   404  				return
   405  			case <-timer.C:
   406  				reason = "timeout"
   407  				// time to flush
   408  			case rpcs := <-c.rpcs:
   409  				if !m.add(rpcs) {
   410  					// can still put more rpcs into batch
   411  					continue
   412  				}
   413  				reason = "queue full"
   414  				// batch is full
   415  				if !timer.Stop() {
   416  					<-timer.C
   417  				}
   418  			}
   419  			break
   420  		}
   421  		flush(reason)
   422  	}
   423  }
   424  
   425  func returnResult(c hrpc.Call, msg proto.Message, err error) {
   426  	if m, ok := c.(*multi); ok {
   427  		m.returnResults(msg, err)
   428  	} else {
   429  		c.ResultChan() <- hrpc.RPCResult{Msg: msg, Error: err}
   430  	}
   431  }
   432  
   433  func (c *client) trySend(rpc hrpc.Call) (err error) {
   434  	if id, err := c.send(rpc); err != nil {
   435  		if _, ok := err.(ServerError); ok {
   436  			c.fail(err)
   437  		}
   438  		if r := c.unregisterRPC(id); r != nil {
   439  			// we are the ones to unregister the rpc,
   440  			// return err to notify client of it
   441  			return err
   442  		}
   443  	}
   444  	return nil
   445  }
   446  
   447  func (c *client) receiveRPCs() {
   448  	reader := bufio.NewReader(c.conn)
   449  	for {
   450  		select {
   451  		case <-c.done:
   452  			return
   453  		default:
   454  			if err := c.receive(reader); err != nil {
   455  				if _, ok := err.(ServerError); ok {
   456  					// fail the client and let the callers establish a new one
   457  					c.fail(err)
   458  					return
   459  				}
   460  				// in other cases we consider that the region client is healthy
   461  				// and return the error to caller to let them retry
   462  			}
   463  		}
   464  	}
   465  }
   466  
   467  func (c *client) receive(r io.Reader) (err error) {
   468  	var (
   469  		sz       [4]byte
   470  		header   pb.ResponseHeader
   471  		response proto.Message
   472  	)
   473  
   474  	_, err = io.ReadFull(r, sz[:])
   475  	if err != nil {
   476  		return ServerError{err}
   477  	}
   478  
   479  	size := binary.BigEndian.Uint32(sz[:])
   480  	b := make([]byte, size)
   481  
   482  	_, err = io.ReadFull(r, b)
   483  	if err != nil {
   484  		return ServerError{err}
   485  	}
   486  
   487  	// unmarshal header
   488  	headerBytes, headerLen := protowire.ConsumeBytes(b)
   489  	if headerLen < 0 {
   490  		return ServerError{fmt.Errorf("failed to decode the response header: %v",
   491  			protowire.ParseError(headerLen))}
   492  	}
   493  	if err = proto.Unmarshal(headerBytes, &header); err != nil {
   494  		return ServerError{fmt.Errorf("failed to decode the response header: %v", err)}
   495  	}
   496  
   497  	if header.CallId == nil {
   498  		return ErrMissingCallID
   499  	}
   500  
   501  	callID := *header.CallId
   502  	rpc := c.unregisterRPC(callID)
   503  	if rpc == nil {
   504  		return ServerError{fmt.Errorf("got a response with an unexpected call ID: %d", callID)}
   505  	}
   506  	if err := c.inFlightDown(); err != nil {
   507  		return ServerError{err}
   508  	}
   509  
   510  	select {
   511  	case <-rpc.Context().Done():
   512  		// context has expired, don't bother deserializing
   513  		return
   514  	default:
   515  	}
   516  
   517  	// Here we know for sure that we got a response for rpc we asked.
   518  	// It's our responsibility to deliver the response or error to the
   519  	// caller as we unregistered the rpc.
   520  	defer func() { returnResult(rpc, response, err) }()
   521  
   522  	if header.Exception != nil {
   523  		err = exceptionToError(*header.Exception.ExceptionClassName, *header.Exception.StackTrace)
   524  		return
   525  	}
   526  
   527  	response = rpc.NewResponse()
   528  
   529  	responseBytes, responseLen := protowire.ConsumeBytes(b[headerLen:])
   530  	if responseLen < 0 {
   531  		err = RetryableError{fmt.Errorf("failed to decode the response: %s",
   532  			protowire.ParseError(responseLen))}
   533  		return
   534  	}
   535  
   536  	if err = proto.Unmarshal(responseBytes, response); err != nil {
   537  		err = RetryableError{fmt.Errorf("failed to decode the response: %s", err)}
   538  		return
   539  	}
   540  
   541  	var cellsLen uint32
   542  	if header.CellBlockMeta != nil {
   543  		cellsLen = header.CellBlockMeta.GetLength()
   544  	}
   545  	if d, ok := rpc.(canDeserializeCellBlocks); cellsLen > 0 && ok {
   546  		b := b[size-cellsLen:]
   547  		if c.compressor != nil {
   548  			b, err = c.compressor.decompressCellblocks(b)
   549  			if err != nil {
   550  				err = RetryableError{fmt.Errorf("failed to decompress the response: %s", err)}
   551  				return
   552  			}
   553  		}
   554  		var nread uint32
   555  		nread, err = d.DeserializeCellBlocks(response, b)
   556  		if err != nil {
   557  			err = RetryableError{fmt.Errorf("failed to decode the response: %s", err)}
   558  			return
   559  		}
   560  
   561  		if int(nread) < len(b) {
   562  			err = RetryableError{
   563  				fmt.Errorf("short read: buffer length %d, read %d", len(b), nread)}
   564  			return
   565  		}
   566  	}
   567  	return
   568  }
   569  
   570  func exceptionToError(class, stack string) error {
   571  	err := fmt.Errorf("HBase Java exception %s:\n%s", class, stack)
   572  	if s, ok := javaRetryableExceptions[class]; ok && strings.Contains(stack, s) {
   573  		return RetryableError{err}
   574  	} else if s, ok := javaRegionExceptions[class]; ok && strings.Contains(stack, s) {
   575  		return NotServingRegionError{err}
   576  	} else if s, ok := javaServerExceptions[class]; ok && strings.Contains(stack, s) {
   577  		return ServerError{err}
   578  	}
   579  	return err
   580  }
   581  
   582  // write sends the given buffer to the RegionServer.
   583  func (c *client) write(buf []byte) error {
   584  	_, err := c.conn.Write(buf)
   585  	return err
   586  }
   587  
   588  // sendHello sends the "hello" message needed when opening a new connection.
   589  func (c *client) sendHello() error {
   590  	connHeader := &pb.ConnectionHeader{
   591  		UserInfo: &pb.UserInformation{
   592  			EffectiveUser: proto.String(c.effectiveUser),
   593  		},
   594  		ServiceName:         proto.String(string(c.ctype)),
   595  		CellBlockCodecClass: proto.String("org.apache.hadoop.hbase.codec.KeyValueCodec"),
   596  	}
   597  	if c.compressor != nil {
   598  		// if we have compression enabled, specify the compressor class
   599  		connHeader.CellBlockCompressorClass = proto.String(c.compressor.CellBlockCompressorClass())
   600  	}
   601  	data, err := proto.Marshal(connHeader)
   602  	if err != nil {
   603  		return fmt.Errorf("failed to marshal connection header: %s", err)
   604  	}
   605  
   606  	const header = "HBas\x00\x50" // \x50 = Simple Auth.
   607  	buf := make([]byte, 0, len(header)+4+len(data))
   608  	buf = append(buf, header...)
   609  	buf = buf[:len(header)+4]
   610  	binary.BigEndian.PutUint32(buf[6:], uint32(len(data)))
   611  	buf = append(buf, data...)
   612  	return c.write(buf)
   613  }
   614  
   615  // send sends an RPC out to the wire.
   616  // Returns the response (for now, as the call is synchronous).
   617  func (c *client) send(rpc hrpc.Call) (uint32, error) {
   618  	var request proto.Message
   619  	var cellblocks net.Buffers
   620  	var cellblocksLen uint32
   621  	if s, ok := rpc.(canSerializeCellBlocks); ok && s.CellBlocksEnabled() {
   622  		// request can be serialized to cellblocks
   623  		request, cellblocks, cellblocksLen = s.SerializeCellBlocks(nil)
   624  
   625  		if c.compressor != nil {
   626  			// we have compressor, encode the cellblocks
   627  			compressed := c.compressor.compressCellblocks(cellblocks, cellblocksLen)
   628  			defer freeBuffer(compressed)
   629  			cellblocks = net.Buffers{compressed}
   630  			cellblocksLen = uint32(len(compressed))
   631  		}
   632  	} else {
   633  		// plain protobuf request
   634  		request = rpc.ToProto()
   635  	}
   636  
   637  	// we have to register rpc after we marshal because
   638  	// registered rpc can fail before it was even sent
   639  	// in all the cases where c.fail() is called.
   640  	// If that happens, client can retry sending the rpc
   641  	// again potentially changing it's contents.
   642  	id := c.registerRPC(rpc)
   643  
   644  	b, err := marshalProto(rpc, id, request, cellblocksLen)
   645  	if err != nil {
   646  		return id, err
   647  	}
   648  
   649  	rpcSize.WithLabelValues(c.Addr()).Observe(float64(uint32(len(b)) + cellblocksLen))
   650  	if cellblocks != nil {
   651  		bfs := append(net.Buffers{b}, cellblocks...)
   652  		_, err = bfs.WriteTo(c.conn)
   653  	} else {
   654  		err = c.write(b)
   655  	}
   656  	if err != nil {
   657  		return id, ServerError{err}
   658  	}
   659  
   660  	if err := c.inFlightUp(); err != nil {
   661  		return id, ServerError{err}
   662  	}
   663  	return id, nil
   664  }
   665  
   666  var headerPool = sync.Pool{
   667  	New: func() any {
   668  		return &pb.RequestHeader{}
   669  	},
   670  }
   671  
   672  func getHeader() *pb.RequestHeader {
   673  	return headerPool.Get().(*pb.RequestHeader)
   674  }
   675  
   676  func returnHeader(header *pb.RequestHeader) {
   677  	header.Reset()
   678  	headerPool.Put(header)
   679  }
   680  
   681  var pbTrue = proto.Bool(true)
   682  
   683  func marshalProto(rpc hrpc.Call, callID uint32, request proto.Message,
   684  	cellblocksLen uint32) ([]byte, error) {
   685  	header := getHeader()
   686  	defer returnHeader(header)
   687  	header.MethodName = proto.String(rpc.Name())
   688  	header.RequestParam = pbTrue
   689  	header.CallId = &callID
   690  	if p := hrpc.GetPriority(rpc); p > 0 {
   691  		header.Priority = &p
   692  	}
   693  
   694  	if cellblocksLen > 0 {
   695  		header.CellBlockMeta = &pb.CellBlockMeta{
   696  			Length: &cellblocksLen,
   697  		}
   698  	}
   699  
   700  	// Wire protocol:
   701  	//
   702  	// 4 byte total length
   703  	//
   704  	// Protobuf data:
   705  	// varint length of header
   706  	// <header>
   707  	// varint length of request
   708  	// <request>
   709  	//
   710  	// Cellblocks:
   711  	// <cellblocks>
   712  	//
   713  	// As an optimization we use the net.Buffers.WriteTo API below to
   714  	// send a [][]byte in a single writev syscall rather than making a
   715  	// write syscall per []byte or concatenating the [][]byte into a
   716  	// single []byte. The cellblocks data is already organized into an
   717  	// [][]byte, so we just need to encode the total length and the
   718  	// protobuf data into a []byte.
   719  
   720  	if request == nil {
   721  		return nil, errors.New("failed to marshal request: proto: Marshal called with nil")
   722  	}
   723  
   724  	headerSize := proto.Size(header)
   725  	requestSize := proto.Size(request)
   726  
   727  	protobufLen := protowire.SizeVarint(uint64(headerSize)) + headerSize +
   728  		protowire.SizeVarint(uint64(requestSize)) + requestSize
   729  	totalLen := uint32(protobufLen) + cellblocksLen
   730  
   731  	b := make([]byte, 4, 4+protobufLen)
   732  
   733  	binary.BigEndian.PutUint32(b, totalLen)
   734  
   735  	b = protowire.AppendVarint(b, uint64(headerSize))
   736  	b, err := proto.MarshalOptions{UseCachedSize: true}.MarshalAppend(b, header)
   737  	if err != nil {
   738  		return nil, fmt.Errorf("failed to marshal request header: %s", err)
   739  	}
   740  
   741  	b = protowire.AppendVarint(b, uint64(requestSize))
   742  	b, err = proto.MarshalOptions{UseCachedSize: true}.MarshalAppend(b, request)
   743  	if err != nil {
   744  		return nil, fmt.Errorf("failed to marshal request: %s", err)
   745  	}
   746  	if len(b) != 4+protobufLen {
   747  		// This shouldn't ever happen, if it does there is a bug in
   748  		// the size calculation.
   749  		panic(fmt.Errorf("size of data sent doesn't match expected. expected: %d actual: %d"+
   750  			"headerSize: %d requestSize: %d\nheader: %s\nrequest: %s",
   751  			4+protobufLen, len(b), headerSize, requestSize, header, request))
   752  	}
   753  	return b, nil
   754  }
   755  
   756  func (c *client) MarshalJSON() ([]byte, error) {
   757  
   758  	type Address struct {
   759  		Network string
   760  		Address string
   761  	}
   762  
   763  	// the status of the Done channel as a string. Closed if its done, Not Closed o.w
   764  	var done_status string
   765  	if c.done != nil {
   766  		select {
   767  		case <-c.done:
   768  			done_status = "Closed"
   769  		default:
   770  			done_status = "Not Closed"
   771  		}
   772  	}
   773  
   774  	c.inFlightM.Lock()
   775  	inFlight := c.inFlight
   776  	c.inFlightM.Unlock()
   777  
   778  	// if conn is nil then we don't want to panic. So just get the addresses if conn is not nil
   779  	var localAddr, remoteAddr Address
   780  	c.connM.Lock()
   781  	conn := c.conn
   782  	c.connM.Unlock()
   783  	if conn != nil {
   784  		localAddress := conn.LocalAddr()
   785  		remoteAddress := conn.RemoteAddr()
   786  		localAddr = Address{localAddress.Network(), localAddress.String()}
   787  		remoteAddr = Address{remoteAddress.Network(), remoteAddress.String()}
   788  	}
   789  
   790  	state := struct {
   791  		ConnectionLocalAddress  Address
   792  		ConnectionRemoteAddress Address
   793  		RegionServerAddress     string
   794  		ClientType              ClientType
   795  		InFlight                uint32
   796  		Id                      uint32
   797  		Done_status             string
   798  	}{
   799  		ConnectionLocalAddress:  localAddr,
   800  		ConnectionRemoteAddress: remoteAddr,
   801  		RegionServerAddress:     c.addr,
   802  		ClientType:              c.ctype,
   803  		InFlight:                inFlight,
   804  		Id:                      atomic.LoadUint32(&c.id),
   805  		Done_status:             done_status,
   806  	}
   807  
   808  	return json.Marshal(state)
   809  }