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

     1  // Copyright (C) 2016  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 gohbase
     7  
     8  import (
     9  	"bytes"
    10  	"context"
    11  	"errors"
    12  	"fmt"
    13  	"io"
    14  	"math"
    15  	"strconv"
    16  	"time"
    17  
    18  	"github.com/tsuna/gohbase/hrpc"
    19  	"github.com/tsuna/gohbase/internal/observability"
    20  	"github.com/tsuna/gohbase/pb"
    21  	"github.com/tsuna/gohbase/region"
    22  	"github.com/tsuna/gohbase/zk"
    23  	"go.opentelemetry.io/otel/attribute"
    24  	"go.opentelemetry.io/otel/codes"
    25  	"go.opentelemetry.io/otel/trace"
    26  	"google.golang.org/protobuf/proto"
    27  )
    28  
    29  type regionInfoAndAddr struct {
    30  	regionInfo hrpc.RegionInfo
    31  	addr       string
    32  }
    33  
    34  // Constants
    35  var (
    36  	// Name of the meta region.
    37  	metaTableName = []byte("hbase:meta")
    38  
    39  	infoFamily = map[string][]string{
    40  		"info": nil,
    41  	}
    42  
    43  	// TableNotFound is returned when attempting to access a table that
    44  	// doesn't exist on this cluster.
    45  	TableNotFound = errors.New("table not found")
    46  
    47  	// ErrCannotFindRegion is returned when it took too many tries to find a
    48  	// region for the request. It's likely that hbase:meta has overlaps or some other
    49  	// inconsistency.
    50  	ErrCannotFindRegion = errors.New("cannot find region for the rpc")
    51  
    52  	// ErrClientClosed is returned when the gohbase client has been closed
    53  	ErrClientClosed = errors.New("client is closed")
    54  )
    55  
    56  const (
    57  	// maxFindRegionTries is the maximum number of times to try to send an RPC
    58  	maxFindRegionTries = 10
    59  
    60  	backoffStart = 16 * time.Millisecond
    61  )
    62  
    63  func (c *client) getRegionForRpc(ctx context.Context, rpc hrpc.Call) (hrpc.RegionInfo, error) {
    64  	for i := 0; i < maxFindRegionTries; i++ {
    65  		// Check the cache for a region that can handle this request
    66  		if reg := c.getRegionFromCache(rpc.Table(), rpc.Key()); reg != nil {
    67  			return reg, nil
    68  		}
    69  
    70  		if reg, err := c.findRegion(ctx, rpc.Table(), rpc.Key()); reg != nil {
    71  			return reg, nil
    72  		} else if err != nil {
    73  			return nil, err
    74  		}
    75  	}
    76  	return nil, ErrCannotFindRegion
    77  }
    78  
    79  func (c *client) SendRPC(rpc hrpc.Call) (msg proto.Message, err error) {
    80  	start := time.Now()
    81  	description := rpc.Description()
    82  	ctx, sp := observability.StartSpan(rpc.Context(), description,
    83  		trace.WithAttributes(
    84  			attribute.String("table", strconv.Quote(string(rpc.Table()))),
    85  			attribute.String("key", strconv.Quote(string(rpc.Key()))),
    86  		))
    87  	defer func() {
    88  		result := "ok"
    89  		if err != nil {
    90  			result = "error"
    91  			sp.SetStatus(codes.Error, err.Error())
    92  		}
    93  
    94  		o := operationDurationSeconds.WithLabelValues(description, result)
    95  
    96  		observability.ObserveWithTrace(ctx, o, time.Since(start).Seconds())
    97  		sp.End()
    98  	}()
    99  
   100  	backoff := backoffStart
   101  	serverErrorCount := 0
   102  	for {
   103  		var rc hrpc.RegionClient
   104  		rc, err = c.getRegionAndClientForRPC(ctx, rpc)
   105  		if err != nil {
   106  			return nil, err
   107  		}
   108  		rpcStart := time.Now()
   109  		msg, err = c.sendRPCToRegionClient(ctx, rpc, rc)
   110  		switch err.(type) {
   111  		case region.RetryableError:
   112  			if scan, ok := rpc.(*hrpc.Scan); ok {
   113  				c.scanRpcScanStats(scan, msg, err, true, rpcStart, time.Now())
   114  			}
   115  			sp.AddEvent("retrySleep")
   116  			backoff, err = sleepAndIncreaseBackoff(ctx, backoff)
   117  			if err != nil {
   118  				return msg, err
   119  			}
   120  			continue // retry
   121  		case region.ServerError:
   122  			if scan, ok := rpc.(*hrpc.Scan); ok {
   123  				c.scanRpcScanStats(scan, msg, err, true, rpcStart, time.Now())
   124  			}
   125  			// Retry ServerError immediately, as we want failover fast to
   126  			// another server. But if HBase keep sending us ServerError, we
   127  			// should start to backoff. We don't want to overwhelm HBase.
   128  			if serverErrorCount > 1 {
   129  				sp.AddEvent("retrySleep")
   130  				backoff, err = sleepAndIncreaseBackoff(ctx, backoff)
   131  				if err != nil {
   132  					return msg, err
   133  				}
   134  			}
   135  			serverErrorCount++
   136  			continue // retry
   137  		case region.NotServingRegionError:
   138  			if scan, ok := rpc.(*hrpc.Scan); ok {
   139  				c.scanRpcScanStats(scan, msg, err, true, rpcStart, time.Now())
   140  			}
   141  			continue // retry
   142  		}
   143  		if scan, ok := rpc.(*hrpc.Scan); ok {
   144  			c.scanRpcScanStats(scan, msg, err, false, rpcStart, time.Now())
   145  		}
   146  		return msg, err
   147  	}
   148  }
   149  
   150  func (c *client) scanRpcScanStats(scan *hrpc.Scan, resp proto.Message, err error,
   151  	retry bool, start, end time.Time) {
   152  	if scan.ScanStatsHandler() != nil {
   153  		stats := &hrpc.ScanStats{}
   154  		// Update the ScanMetrics if they are being tracked. For ScanStats, these ScanMetrics
   155  		// are collected per call to SendRPC and therefore may not be reflective of the entire
   156  		// result of the Scan request if the results are split across multiple calls to
   157  		// Scanner.Next().
   158  		if scan.TrackScanMetrics() && resp != nil {
   159  			scanres, ok := resp.(*pb.ScanResponse)
   160  			if !ok {
   161  				c.logger.Debug("got non ScanResponse for ScanRequest, no ScanMetrics to add")
   162  			} else {
   163  				if scanres.ScanMetrics != nil {
   164  					stats.ScanMetrics = make(map[string]int64)
   165  					for _, m := range scanres.ScanMetrics.GetMetrics() {
   166  						stats.ScanMetrics[m.GetName()] = m.GetValue()
   167  					}
   168  				}
   169  			}
   170  		}
   171  
   172  		stats.Table = scan.Table()
   173  		stats.StartRow = scan.StartRow()
   174  		stats.EndRow = scan.StopRow()
   175  		if reg := scan.Region(); reg != nil {
   176  			stats.RegionID = reg.ID()
   177  			if cl := reg.Client(); cl != nil {
   178  				stats.RegionServer = cl.Addr()
   179  			}
   180  		}
   181  		stats.ScannerID = scan.ScannerId()
   182  		stats.ScanStatsID = scan.ScanStatsID()
   183  		stats.Start = start
   184  		stats.End = end
   185  		if err != nil {
   186  			stats.Error = true
   187  		}
   188  		stats.Retryable = retry
   189  		stats.ResponseSize = scan.ResponseSize
   190  		scan.ScanStatsHandler()(stats)
   191  	}
   192  }
   193  
   194  func (c *client) getRegionAndClientForRPC(ctx context.Context, rpc hrpc.Call) (
   195  	hrpc.RegionClient, error) {
   196  	for {
   197  		reg, err := c.getRegionForRpc(ctx, rpc)
   198  		if err != nil {
   199  			return nil, err
   200  		}
   201  		if ch := reg.AvailabilityChan(); ch != nil { // region is currently unavailable
   202  			select {
   203  			case <-ctx.Done():
   204  				return nil, ctx.Err()
   205  			case <-c.done:
   206  				return nil, ErrClientClosed
   207  			case <-ch:
   208  			}
   209  		}
   210  
   211  		client := reg.Client()
   212  		if client == nil {
   213  			// There was an error getting the region client. Mark the
   214  			// region as unavailable.
   215  			if reg.MarkUnavailable() {
   216  				// If this was the first goroutine to mark the region as
   217  				// unavailable, start a goroutine to reestablish a connection
   218  				go c.reestablishRegion(reg)
   219  			}
   220  			if ch := reg.AvailabilityChan(); ch != nil {
   221  				select {
   222  				case <-ctx.Done():
   223  					return nil, ctx.Err()
   224  				case <-c.done:
   225  					return nil, ErrClientClosed
   226  				case <-ch:
   227  				}
   228  			}
   229  			if reg.Context().Err() != nil {
   230  				// region is dead because it was split or merged,
   231  				// retry lookup
   232  				continue
   233  			}
   234  			client = reg.Client()
   235  			if client == nil {
   236  				continue
   237  			}
   238  		}
   239  		rpc.SetRegion(reg)
   240  		return client, nil
   241  	}
   242  }
   243  
   244  var (
   245  	// NotExecutedError is returned when an RPC in a batch is not
   246  	// executed due to encountering a different error in the batch.
   247  	NotExecutedError = errors.New(
   248  		"RPC in batch not executed due to another error")
   249  )
   250  
   251  // SendBatch will execute all the Calls in batch. Every Call must have
   252  // the same table and must be Batchable.
   253  //
   254  // SendBatch will discover the correct region and region server for
   255  // each Call and dispatch the Calls accordingly. SendBatch is not an
   256  // atomic operation. Some calls may fail and others succeed. Calls
   257  // sharing a region will execute in the order passed into SendBatch.
   258  //
   259  // SendBatch returns a slice of [hrpc.RPCResult] each containing a
   260  // response and an error. The results will be returned in the same
   261  // order as the Calls in the batch, in other words the i'th result
   262  // will be for the i'th call. A nil error means the Call executed
   263  // successfully. allOK is true if all calls completed successfully,
   264  // and false if any calls failed and the errors in the results need to
   265  // be checked.
   266  //
   267  // SendBatch will continue retrying each RPC in batch until it
   268  // succeeds, fails with a non-retryable error, or the context is
   269  // canceled.
   270  func (c *client) SendBatch(ctx context.Context, batch []hrpc.Call) (
   271  	res []hrpc.RPCResult, allOK bool) {
   272  	if len(batch) == 0 {
   273  		return nil, true
   274  	}
   275  
   276  	allOK = true
   277  
   278  	start := time.Now()
   279  	description := "SendBatch"
   280  	ctx, sp := observability.StartSpan(ctx, description)
   281  	defer func() {
   282  		result := "ok"
   283  		if !allOK {
   284  			result = "error"
   285  			sp.SetStatus(codes.Error, "batch error")
   286  		}
   287  
   288  		o := operationDurationSeconds.WithLabelValues(description, result)
   289  
   290  		observability.ObserveWithTrace(ctx, o, time.Since(start).Seconds())
   291  		sp.End()
   292  	}()
   293  
   294  	table := batch[0].Table()
   295  	res = make([]hrpc.RPCResult, len(batch))
   296  	rpcToRes := make(map[hrpc.Call]int, len(batch))
   297  	for i, rpc := range batch {
   298  		// map Call to index in res so that we can set the correct
   299  		// result as Calls complete
   300  		if j, dup := rpcToRes[rpc]; dup {
   301  			res[i].Error = fmt.Errorf("duplicate call in batch at index %d", j)
   302  			allOK = false
   303  			continue
   304  		}
   305  		rpcToRes[rpc] = i
   306  
   307  		// Initialize res with NotExecutedError. As RPCs are executed this
   308  		// will be replaced by a more specific error or nil if no error
   309  		// occurs.
   310  		res[i].Error = NotExecutedError
   311  
   312  		if !bytes.Equal(rpc.Table(), table) {
   313  			res[i].Error = fmt.Errorf("multiple tables in batch request: %q and %q",
   314  				string(table), string(rpc.Table()))
   315  			allOK = false
   316  		} else if !hrpc.CanBatch(rpc) {
   317  			res[i].Error = errors.New("non-batchable call passed to SendBatch")
   318  			allOK = false
   319  		}
   320  	}
   321  	if !allOK {
   322  		return res, allOK
   323  	}
   324  
   325  	// Send and wait for responses loop. This loop will partition the
   326  	// batch per-regionserver batches, send those batches to the
   327  	// region server and wait for results. Any RPCs that hit retryable
   328  	// errors will be made into a new batch and passed through this
   329  	// loop again.
   330  
   331  	// unretryableErrorSeen set to true when any RPC in the batch hits
   332  	// an error that is not retryable. This is used to remember to
   333  	// return allOK=false even after we retry RPCs that hit retryable
   334  	// errors and those all succeed.
   335  	var unretryableErrorSeen bool
   336  	var retries []hrpc.Call
   337  	backoff := backoffStart
   338  
   339  	for {
   340  		rpcByClient, ok := c.findClients(ctx, batch, res)
   341  		if !ok {
   342  			return res, false
   343  		}
   344  		sendBatchSplitCount.Observe(float64(len(rpcByClient)))
   345  
   346  		// Send each group of RPCs to region client to be executed.
   347  		type clientAndRPCs struct {
   348  			client hrpc.RegionClient
   349  			rpcs   []hrpc.Call
   350  		}
   351  		// keep track of the order requests are queued so that we can wait
   352  		// for their responses in the same order.
   353  		cAndRs := make([]clientAndRPCs, 0, len(rpcByClient))
   354  		for client, rpcs := range rpcByClient {
   355  			client.QueueBatch(ctx, rpcs)
   356  			cAndRs = append(cAndRs, clientAndRPCs{client, rpcs})
   357  		}
   358  
   359  		// batch wil be used to hold any RPCs that need to be retried
   360  		batch = batch[:0]
   361  		var needBackoff bool
   362  
   363  		func() { // func used to scope the span
   364  			ctx, sp := observability.StartSpan(ctx, "waitForResult")
   365  			defer sp.End()
   366  			for _, cAndR := range cAndRs {
   367  				shouldRetry, shouldBackoff, unretryableError, ok := c.waitForCompletion(
   368  					ctx, cAndR.client, cAndR.rpcs, res, rpcToRes)
   369  				if !ok {
   370  					allOK = false
   371  					retries = append(retries, shouldRetry...)
   372  					needBackoff = needBackoff || shouldBackoff
   373  					unretryableErrorSeen = unretryableErrorSeen || unretryableError
   374  				}
   375  			}
   376  		}()
   377  
   378  		// Exit retry loop if no RPCs are retryable because they all
   379  		// succeeded or hit unretryable errors (this is true if
   380  		// retries is empty), or the context is done.
   381  		if len(retries) == 0 || ctx.Err() != nil {
   382  			break
   383  		}
   384  		if needBackoff {
   385  			sp.AddEvent("retrySleep")
   386  			var err error
   387  			backoff, err = sleepAndIncreaseBackoff(ctx, backoff)
   388  			if err != nil {
   389  				break
   390  			}
   391  		} else {
   392  			sp.AddEvent("retry")
   393  		}
   394  		// Set state for next loop iteration
   395  		batch = retries
   396  		retries = retries[:0]
   397  		allOK = !unretryableErrorSeen
   398  	}
   399  
   400  	return res, allOK
   401  }
   402  
   403  // findClients takes a batch of rpcs and discovers the region and
   404  // region client associated with each. A map is returned with rpcs
   405  // grouped by their region client. If any error is encountered, the
   406  // corresponding slot in res will be updated with that error and a
   407  // BatchError is returned.
   408  //
   409  // findClients will not return on the first errror encountered. It
   410  // will iterate through all the RPCs to ensure that all unknown
   411  // regions encountered in the batch will start being initialized.
   412  func (c *client) findClients(ctx context.Context, batch []hrpc.Call, res []hrpc.RPCResult) (
   413  	map[hrpc.RegionClient][]hrpc.Call, bool) {
   414  
   415  	rpcByClient := make(map[hrpc.RegionClient][]hrpc.Call)
   416  	ok := true
   417  	for i, rpc := range batch {
   418  		rc, err := c.getRegionAndClientForRPC(ctx, rpc)
   419  		if err != nil {
   420  			res[i].Error = err
   421  			ok = false
   422  			continue // see if any more RPCs are missing regions
   423  		}
   424  		rpcByClient[rc] = append(rpcByClient[rc], rpc)
   425  	}
   426  	return rpcByClient, ok
   427  }
   428  
   429  // waitForCompletion waits for the completion of all rpcs, updating
   430  // the appropriate index in results with the help of rpcToRes. If all
   431  // rpcs succeed then ok will return true, otherwise:
   432  //   - ok will be false
   433  //   - retryables will contain RPCs that can be retried
   434  //   - shouldBackoff will be true if any retryable RPCs need a backoff before retrying
   435  //   - unretryableError will be true if there were errors seen on RPCs
   436  //     that were not retryable. It communicates that retryables does
   437  //     not contain all the RPCs that failed, so even though those
   438  //     retryable RPCs may eventually succeed we need to return !ok to
   439  //     the caller of SendBatch.
   440  func (c *client) waitForCompletion(ctx context.Context, rc hrpc.RegionClient,
   441  	rpcs []hrpc.Call, results []hrpc.RPCResult, rpcToRes map[hrpc.Call]int) (
   442  	retryables []hrpc.Call, shouldBackoff, unretryableError, ok bool) {
   443  
   444  	ok = true
   445  	canceledIndex := len(rpcs)
   446  
   447  loop:
   448  	for i, rpc := range rpcs {
   449  		select {
   450  		case res := <-rpc.ResultChan():
   451  			results[rpcToRes[rpc]] = res
   452  			if res.Error != nil {
   453  				c.handleResultError(res.Error, rpc.Region(), rc)
   454  				ok = false
   455  				switch res.Error.(type) {
   456  				case region.RetryableError:
   457  					shouldBackoff = true
   458  					retryables = append(retryables, rpc)
   459  				case region.ServerError, region.NotServingRegionError:
   460  					retryables = append(retryables, rpc)
   461  				default:
   462  					unretryableError = true
   463  				}
   464  			}
   465  
   466  		case <-ctx.Done():
   467  			canceledIndex = i
   468  			ok = false
   469  			break loop
   470  		}
   471  	}
   472  
   473  	// If the context was canceled we may have exited the loop above
   474  	// without checking for every result. Do a non-blocking read of
   475  	// the ResultChan for the remaining RPCs. If not ready the result
   476  	// will be the context error.
   477  	for _, rpc := range rpcs[canceledIndex:] {
   478  		select {
   479  		case res := <-rpc.ResultChan():
   480  			results[rpcToRes[rpc]] = res
   481  			if res.Error != nil {
   482  				c.handleResultError(res.Error, rpc.Region(), rc)
   483  			}
   484  		default:
   485  			results[rpcToRes[rpc]].Error = ctx.Err()
   486  		}
   487  	}
   488  
   489  	return retryables, shouldBackoff, unretryableError, ok
   490  }
   491  
   492  // handleResultErrorOverride can be used to override behaviour of handleResultError in testing
   493  var handleResultErrorOverride func(err error, reg hrpc.RegionInfo, rc hrpc.RegionClient)
   494  
   495  func (c *client) handleResultError(err error, reg hrpc.RegionInfo, rc hrpc.RegionClient) {
   496  	if handleResultErrorOverride != nil {
   497  		handleResultErrorOverride(err, reg, rc)
   498  		return
   499  	}
   500  
   501  	// Check for errors
   502  	switch err.(type) {
   503  	case region.NotServingRegionError:
   504  		// There's an error specific to this region, but
   505  		// our region client is fine. Mark this region as
   506  		// unavailable (as opposed to all regions sharing
   507  		// the client), and start a goroutine to reestablish
   508  		// it.
   509  		if reg.MarkUnavailable() {
   510  			go c.reestablishRegion(reg)
   511  		}
   512  	case region.ServerError:
   513  		// If it was an unrecoverable error, the region client is
   514  		// considered dead.
   515  		if reg == c.adminRegionInfo {
   516  			// If this is the admin client, mark the region
   517  			// as unavailable and start up a goroutine to
   518  			// reconnect if it wasn't already marked as such.
   519  			if reg.MarkUnavailable() {
   520  				go c.reestablishRegion(reg)
   521  			}
   522  		} else {
   523  			c.clientDown(rc, reg)
   524  		}
   525  	}
   526  }
   527  
   528  func sendBlocking(ctx context.Context, rc hrpc.RegionClient, rpc hrpc.Call) (
   529  	hrpc.RPCResult, error) {
   530  	rc.QueueRPC(rpc)
   531  
   532  	ctx, sp := observability.StartSpan(ctx, "waitForResult")
   533  	defer sp.End()
   534  	var res hrpc.RPCResult
   535  	// Wait for the response
   536  	select {
   537  	case res = <-rpc.ResultChan():
   538  		return res, nil
   539  	case <-ctx.Done():
   540  		return res, rpc.Context().Err()
   541  	}
   542  }
   543  
   544  func (c *client) sendRPCToRegionClient(ctx context.Context, rpc hrpc.Call, rc hrpc.RegionClient) (
   545  	proto.Message, error) {
   546  	res, err := sendBlocking(ctx, rc, rpc)
   547  	if err != nil {
   548  		return nil, err
   549  	}
   550  	if res.Error != nil {
   551  		c.handleResultError(res.Error, rpc.Region(), rc)
   552  	}
   553  	return res.Msg, res.Error
   554  }
   555  
   556  // clientDown removes client from cache and marks all the regions
   557  // sharing this region's client as unavailable, and start a goroutine
   558  // to reconnect for each of them.
   559  //
   560  // Due to races filling in the clients cache it may not be completely
   561  // accurate. reg is the region we were trying to access when we saw an
   562  // issue with the region client, so make sure it is marked unavailable
   563  // even if it doesn't appear in the clients cache.
   564  func (c *client) clientDown(client hrpc.RegionClient, reg hrpc.RegionInfo) {
   565  	downregions := c.clients.clientDown(client)
   566  	if reg.MarkUnavailable() {
   567  		reg.SetClient(nil)
   568  		go c.reestablishRegion(reg)
   569  	}
   570  	for downreg := range downregions {
   571  		if downreg == reg {
   572  			continue
   573  		}
   574  		if downreg.MarkUnavailable() {
   575  			downreg.SetClient(nil)
   576  			go c.reestablishRegion(downreg)
   577  		}
   578  	}
   579  }
   580  
   581  func (c *client) lookupRegion(ctx context.Context,
   582  	table, key []byte) (hrpc.RegionInfo, string, error) {
   583  	var reg hrpc.RegionInfo
   584  	var addr string
   585  	var err error
   586  	backoff := backoffStart
   587  	for {
   588  		// If it takes longer than regionLookupTimeout, fail so that we can sleep
   589  		lookupCtx, cancel := context.WithTimeout(ctx, c.regionLookupTimeout)
   590  		if c.clientType == region.MasterClient {
   591  			c.logger.Debug("looking up master", "resource", zk.Master)
   592  
   593  			addr, err = c.zkLookup(lookupCtx, zk.Master)
   594  			cancel()
   595  			reg = c.adminRegionInfo
   596  		} else if bytes.Equal(table, metaTableName) {
   597  			c.logger.Debug("looking up region server of hbase:meta", "resource", zk.Meta)
   598  
   599  			addr, err = c.zkLookup(lookupCtx, zk.Meta)
   600  			cancel()
   601  			reg = c.metaRegionInfo
   602  		} else {
   603  			c.logger.Debug("looking up region",
   604  				"table", strconv.Quote(string(table)), "key", strconv.Quote(string(key)))
   605  
   606  			reg, addr, err = c.metaLookup(lookupCtx, table, key)
   607  			cancel()
   608  			if err == TableNotFound {
   609  				c.logger.Debug("hbase:meta does not know about this table/key",
   610  					"table", strconv.Quote(string(table)),
   611  					"key", strconv.Quote(string(key)), "err", err)
   612  
   613  				return nil, "", err
   614  			} else if err == ErrClientClosed {
   615  				return nil, "", err
   616  			}
   617  		}
   618  		if err == nil {
   619  			c.logger.Debug("looked up a region", "table", strconv.Quote(string(table)),
   620  				"key", strconv.Quote(string(key)), "region", reg, "addr", addr)
   621  
   622  			return reg, addr, nil
   623  		}
   624  
   625  		c.logger.Error("failed looking up region", "table", strconv.Quote(string(table)),
   626  			"key", strconv.Quote(string(key)), "backoff", backoff, "err", err)
   627  
   628  		// This will be hit if there was an error locating the region
   629  		backoff, err = sleepAndIncreaseBackoff(ctx, backoff)
   630  		if err != nil {
   631  			return nil, "", err
   632  		}
   633  	}
   634  }
   635  
   636  func (c *client) lookupAllRegions(ctx context.Context,
   637  	table []byte) ([]regionInfoAndAddr, error) {
   638  	var regs []regionInfoAndAddr
   639  	var err error
   640  	backoff := backoffStart
   641  	for {
   642  		// If it takes longer than regionLookupTimeout, fail so that we can sleep
   643  		lookupCtx, cancel := context.WithTimeout(ctx, c.regionLookupTimeout)
   644  		c.logger.Debug("looking up regions", "table", strconv.Quote(string(table)))
   645  
   646  		regs, err = c.metaLookupForTable(lookupCtx, table)
   647  		cancel()
   648  		if err == TableNotFound {
   649  			c.logger.Debug("hbase:meta does not know about this table",
   650  				"table", strconv.Quote(string(table)), "err", err)
   651  
   652  			return nil, err
   653  		} else if err == ErrClientClosed {
   654  			return nil, err
   655  		}
   656  
   657  		if err == nil {
   658  			c.logger.Debug("looked up all regions",
   659  				"table", strconv.Quote(string(table)), "regionsAndAddr", regs)
   660  
   661  			return regs, nil
   662  		}
   663  
   664  		c.logger.Error("failed looking up regions", "table", strconv.Quote(string(table)),
   665  			"backoff", backoff, "err", err)
   666  
   667  		// This will be hit if there was an error locating the region
   668  		backoff, err = sleepAndIncreaseBackoff(ctx, backoff)
   669  		if err != nil {
   670  			return nil, err
   671  		}
   672  	}
   673  }
   674  
   675  func (c *client) findAllRegions(ctx context.Context, table []byte) ([]regionInfoAndAddr, error) {
   676  	regs, err := c.lookupAllRegions(ctx, table)
   677  	if err != nil {
   678  		return nil, err
   679  	}
   680  	for _, regaddr := range regs {
   681  		reg, addr := regaddr.regionInfo, regaddr.addr
   682  		reg.MarkUnavailable()
   683  
   684  		if reg != c.metaRegionInfo && reg != c.adminRegionInfo {
   685  			// Check that the region wasn't added to
   686  			// the cache while we were looking it up.
   687  			overlaps, replaced := c.regions.put(reg)
   688  			if !replaced {
   689  				// the same or younger regions are already in cache
   690  				continue
   691  			}
   692  
   693  			// otherwise, new region in cache, delete overlaps from client's cache
   694  			for _, r := range overlaps {
   695  				c.clients.del(r)
   696  			}
   697  		}
   698  
   699  		// Start a goroutine to connect to the region
   700  		go c.establishRegion(reg, addr)
   701  	}
   702  
   703  	return regs, nil
   704  }
   705  
   706  func (c *client) findRegion(ctx context.Context, table, key []byte) (hrpc.RegionInfo, error) {
   707  	// The region was not in the cache, it
   708  	// must be looked up in the meta table
   709  	reg, addr, err := c.lookupRegion(ctx, table, key)
   710  	if err != nil {
   711  		return nil, err
   712  	}
   713  
   714  	// We are the ones that looked up the region, so we need to
   715  	// mark it unavailable and find a client for it.
   716  	reg.MarkUnavailable()
   717  
   718  	if reg != c.metaRegionInfo && reg != c.adminRegionInfo {
   719  		// Check that the region wasn't added to
   720  		// the cache while we were looking it up.
   721  		overlaps, replaced := c.regions.put(reg)
   722  		if !replaced {
   723  			// the same or younger regions are already in cache, retry looking up in cache
   724  			return nil, nil
   725  		}
   726  
   727  		// otherwise, new region in cache, delete overlaps from client's cache
   728  		for _, r := range overlaps {
   729  			c.clients.del(r)
   730  		}
   731  	}
   732  
   733  	// Start a goroutine to connect to the region
   734  	go c.establishRegion(reg, addr)
   735  
   736  	// Wait for the new region to become
   737  	// available, and then send the RPC
   738  	return reg, nil
   739  }
   740  
   741  // Searches in the regions cache for the region hosting the given row.
   742  func (c *client) getRegionFromCache(table, key []byte) hrpc.RegionInfo {
   743  	if c.clientType == region.MasterClient {
   744  		return c.adminRegionInfo
   745  	} else if bytes.Equal(table, metaTableName) {
   746  		return c.metaRegionInfo
   747  	}
   748  	regionName := createRegionSearchKey(table, key)
   749  	_, region := c.regions.get(regionName)
   750  	if region == nil {
   751  		return nil
   752  	}
   753  
   754  	// make sure the returned region is for the same table
   755  	if !bytes.Equal(fullyQualifiedTable(region), table) {
   756  		// not the same table, can happen if we got the last region
   757  		return nil
   758  	}
   759  
   760  	if len(region.StopKey()) != 0 &&
   761  		// If the stop key is an empty byte array, it means this region is the
   762  		// last region for this table and this key ought to be in that region.
   763  		bytes.Compare(key, region.StopKey()) >= 0 {
   764  		return nil
   765  	}
   766  
   767  	return region
   768  }
   769  
   770  // Creates the META key to search for in order to locate the given key.
   771  func createRegionSearchKey(table, key []byte) []byte {
   772  	// Shorten the key such that the generated meta key is <= MAX_ROW_LENGTH (MaxInt16), otherwise
   773  	// HBase will throw an exception.
   774  	keylen := math.MaxInt16 - len(table) - 3
   775  	if len(key) < keylen {
   776  		keylen = len(key)
   777  	}
   778  
   779  	metaKey := make([]byte, 0, len(table)+keylen+3)
   780  	metaKey = append(metaKey, table...)
   781  	metaKey = append(metaKey, ',')
   782  	metaKey = append(metaKey, key[:keylen]...)
   783  	metaKey = append(metaKey, ',')
   784  	// ':' is the first byte greater than '9'.  We always want to find the
   785  	// entry with the greatest timestamp, so by looking right before ':'
   786  	// we'll find it.
   787  	metaKey = append(metaKey, ':')
   788  	return metaKey
   789  }
   790  
   791  // metaLookup checks meta table for the region in which the given row key for the given table is.
   792  func (c *client) metaLookup(ctx context.Context,
   793  	table, key []byte) (hrpc.RegionInfo, string, error) {
   794  	metaKey := createRegionSearchKey(table, key)
   795  	rpc, err := hrpc.NewScanRange(ctx, metaTableName, metaKey, table,
   796  		hrpc.Families(infoFamily),
   797  		hrpc.Reversed(),
   798  		hrpc.CloseScanner(),
   799  		hrpc.NumberOfRows(1))
   800  	if err != nil {
   801  		return nil, "", err
   802  	}
   803  
   804  	scanner := c.Scan(rpc)
   805  	resp, err := scanner.Next()
   806  	if err == io.EOF {
   807  		return nil, "", TableNotFound
   808  	}
   809  	if err != nil {
   810  		return nil, "", err
   811  	}
   812  
   813  	reg, addr, err := region.ParseRegionInfo(resp)
   814  	if err != nil {
   815  		return nil, "", err
   816  	}
   817  	if !bytes.Equal(table, fullyQualifiedTable(reg)) {
   818  		// This would indicate a bug in HBase.
   819  		return nil, "", fmt.Errorf("meta returned an entry for the wrong table!"+
   820  			"  Looked up table=%q key=%q got region=%s", table, key, reg)
   821  	} else if len(reg.StopKey()) != 0 &&
   822  		bytes.Compare(key, reg.StopKey()) >= 0 {
   823  		// This would indicate a hole in the meta table.
   824  		return nil, "", fmt.Errorf("meta returned an entry for the wrong region!"+
   825  			"  Looked up table=%q key=%q got region=%s", table, key, reg)
   826  	}
   827  	return reg, addr, nil
   828  }
   829  
   830  // Creates the META key to search for all regions
   831  func createAllRegionSearchKey(table []byte) []byte {
   832  	metaKey := make([]byte, 0, len(table)+1)
   833  	metaKey = append(metaKey, table...)
   834  	// '.' is the first byte greater than ','.  Meta table entry has
   835  	// the format table,key,timestamp. By adding '.' to the stop row
   836  	// we scan all keys for table
   837  	metaKey = append(metaKey, '.')
   838  	return metaKey
   839  }
   840  
   841  // metaLookupForTable checks meta table for all the open table regions.
   842  func (c *client) metaLookupForTable(ctx context.Context,
   843  	table []byte) ([]regionInfoAndAddr, error) {
   844  	metaKey := createAllRegionSearchKey(table)
   845  	rpc, err := hrpc.NewScanRange(ctx, metaTableName, table, metaKey,
   846  		hrpc.Families(infoFamily))
   847  	if err != nil {
   848  		return nil, err
   849  	}
   850  
   851  	var regions []regionInfoAndAddr
   852  	scanner := c.Scan(rpc)
   853  	for {
   854  		resp, err := scanner.Next()
   855  		if err == io.EOF {
   856  			break
   857  		}
   858  		if err != nil {
   859  			return nil, err
   860  		}
   861  
   862  		reg, addr, err := region.ParseRegionInfo(resp)
   863  		if err != nil {
   864  			// Ignore error, but log if it's anything else than OfflineRegionError. This really
   865  			// shouldn't happen unless HBase meta table is corrupted/changed format.
   866  			if _, ok := err.(region.OfflineRegionError); !ok {
   867  				c.logger.Debug("failed to parse region", "err", err)
   868  			}
   869  			continue
   870  		}
   871  
   872  		regions = append(regions, regionInfoAndAddr{regionInfo: reg, addr: addr})
   873  	}
   874  
   875  	if len(regions) == 0 {
   876  		return nil, TableNotFound
   877  	}
   878  	return regions, nil
   879  }
   880  
   881  func fullyQualifiedTable(reg hrpc.RegionInfo) []byte {
   882  	namespace := reg.Namespace()
   883  	table := reg.Table()
   884  	if namespace == nil {
   885  		return table
   886  	}
   887  	// non-default namespace table
   888  	fqTable := make([]byte, 0, len(namespace)+1+len(table))
   889  	fqTable = append(fqTable, namespace...)
   890  	fqTable = append(fqTable, byte(':'))
   891  	fqTable = append(fqTable, table...)
   892  	return fqTable
   893  }
   894  
   895  func (c *client) reestablishRegion(reg hrpc.RegionInfo) {
   896  	select {
   897  	case <-c.done:
   898  		return
   899  	default:
   900  	}
   901  
   902  	c.logger.Debug("reestablishing region", "region", reg)
   903  	c.establishRegion(reg, "")
   904  }
   905  
   906  // probeKey returns a key in region that is unlikely to have data at it
   907  // in order to test if the region is online. This prevents the Get request
   908  // to actually fetch the data from the storage which consumes resources
   909  // of the region server
   910  func probeKey(reg hrpc.RegionInfo) []byte {
   911  	// now we create a probe key: reg.StartKey() + 17 zeros
   912  	probe := make([]byte, len(reg.StartKey())+17)
   913  	copy(probe, reg.StartKey())
   914  	return probe
   915  }
   916  
   917  // isRegionEstablished checks whether regionserver accepts rpcs for the region.
   918  // Returns the cause if not established.
   919  func isRegionEstablished(rc hrpc.RegionClient, reg hrpc.RegionInfo) error {
   920  	probe, err := hrpc.NewGet(context.Background(), fullyQualifiedTable(reg), probeKey(reg),
   921  		hrpc.SkipBatch())
   922  	if err != nil {
   923  		panic(fmt.Sprintf("should not happen: %s", err))
   924  	}
   925  	probe.ExistsOnly()
   926  
   927  	probe.SetRegion(reg)
   928  	res, err := sendBlocking(probe.Context(), rc, probe)
   929  	if err != nil {
   930  		panic(fmt.Sprintf("should not happen: %s", err))
   931  	}
   932  
   933  	switch res.Error.(type) {
   934  	case region.ServerError, region.NotServingRegionError, region.RetryableError:
   935  		return res.Error
   936  	default:
   937  		return nil
   938  	}
   939  }
   940  
   941  // establishRegionOverride can be set by tests to override the
   942  // behavior of establishRegion
   943  var establishRegionOverride func(reg hrpc.RegionInfo, addr string)
   944  
   945  func (c *client) establishRegion(reg hrpc.RegionInfo, addr string) {
   946  	if establishRegionOverride != nil {
   947  		establishRegionOverride(reg, addr)
   948  		return
   949  	}
   950  
   951  	var backoff time.Duration
   952  	var err error
   953  	for {
   954  		backoff, err = sleepAndIncreaseBackoff(reg.Context(), backoff)
   955  		if err != nil {
   956  			// region is dead
   957  			reg.MarkAvailable()
   958  			return
   959  		}
   960  		if addr == "" {
   961  			// need to look up region and address of the regionserver
   962  			originalReg := reg
   963  			// lookup region forever until we get it or we learn that it doesn't exist
   964  			reg, addr, err = c.lookupRegion(originalReg.Context(),
   965  				fullyQualifiedTable(originalReg), originalReg.StartKey())
   966  
   967  			if err == TableNotFound {
   968  				// region doesn't exist, delete it from caches
   969  				c.regions.del(originalReg)
   970  				c.clients.del(originalReg)
   971  				originalReg.MarkAvailable()
   972  
   973  				c.logger.Info("region does not exist anymore",
   974  					"region", originalReg.String(), "err", err, "backoff", backoff)
   975  
   976  				return
   977  			} else if originalReg.Context().Err() != nil {
   978  				// region is dead
   979  				originalReg.MarkAvailable()
   980  
   981  				c.logger.Info("region became dead while establishing client for it",
   982  					"region", originalReg.String(), "err", err, "backoff", backoff)
   983  
   984  				return
   985  			} else if err == ErrClientClosed {
   986  				// client has been closed
   987  				return
   988  			} else if err != nil {
   989  				panic(fmt.Errorf("unknown error occurred when looking up region %q, "+
   990  					"backoff=%s: %s", originalReg.String(), backoff, err))
   991  			}
   992  			if !bytes.Equal(reg.Name(), originalReg.Name()) {
   993  				// put new region and remove overlapping ones.
   994  				// Should remove the original region as well.
   995  				reg.MarkUnavailable()
   996  				overlaps, replaced := c.regions.put(reg)
   997  				if !replaced {
   998  					// a region that is the same or younger is already in cache
   999  					reg.MarkAvailable()
  1000  					originalReg.MarkAvailable()
  1001  					return
  1002  				}
  1003  				// otherwise delete the overlapped regions in cache
  1004  				for _, r := range overlaps {
  1005  					c.clients.del(r)
  1006  				}
  1007  				// let rpcs know that they can retry and either get the newly
  1008  				// added region from cache or lookup the one they need
  1009  				originalReg.MarkAvailable()
  1010  			} else {
  1011  				// same region, discard the looked up one
  1012  				reg = originalReg
  1013  			}
  1014  		}
  1015  
  1016  		var client hrpc.RegionClient
  1017  		if reg == c.adminRegionInfo {
  1018  			// admin region is used for talking to master, so it only has one connection to
  1019  			// master that we don't add to the cache
  1020  			// TODO: consider combining this case with the regular regionserver path
  1021  			client = c.newRegionClientFn(addr, c.clientType, c.rpcQueueSize, c.flushInterval,
  1022  				c.effectiveUser, c.regionReadTimeout, nil, c.regionDialer, c.logger)
  1023  		} else {
  1024  			client = c.clients.put(addr, reg, func() hrpc.RegionClient {
  1025  				return c.newRegionClientFn(addr, c.clientType, c.rpcQueueSize, c.flushInterval,
  1026  					c.effectiveUser, c.regionReadTimeout, c.compressionCodec,
  1027  					c.regionDialer, c.logger)
  1028  			})
  1029  		}
  1030  
  1031  		// connect to the region's regionserver.
  1032  		// only the first caller to Dial gets to actually connect, other concurrent calls
  1033  		// will block until connected or an error.
  1034  		dialCtx, cancel := context.WithTimeout(reg.Context(), c.regionLookupTimeout)
  1035  		err = client.Dial(dialCtx)
  1036  		cancel()
  1037  
  1038  		if err == nil {
  1039  			if reg == c.adminRegionInfo {
  1040  				reg.SetClient(client)
  1041  				reg.MarkAvailable()
  1042  				return
  1043  			}
  1044  
  1045  			if err = isRegionEstablished(client, reg); err == nil {
  1046  				// set region client so that as soon as we mark it available,
  1047  				// concurrent readers are able to find the client
  1048  				reg.SetClient(client)
  1049  				reg.MarkAvailable()
  1050  				return
  1051  			} else if _, ok := err.(region.ServerError); ok {
  1052  				// the client we got died
  1053  				c.clientDown(client, reg)
  1054  			}
  1055  		} else if err == context.Canceled {
  1056  			// region is dead
  1057  			reg.MarkAvailable()
  1058  			return
  1059  		} else {
  1060  			// otherwise Dial failed, purge the client and retry.
  1061  			// note that it's safer to reestablish all regions for this client as well
  1062  			// because they could have ended up setteling for the same client.
  1063  			c.clientDown(client, reg)
  1064  		}
  1065  
  1066  		c.logger.Debug("region was not established, retrying",
  1067  			"region", reg, "backoff", backoff, "err", err)
  1068  		// reset address because we weren't able to connect to it
  1069  		// or regionserver says it's still offline, should look up again
  1070  		addr = ""
  1071  	}
  1072  }
  1073  
  1074  // sleepAndIncreaseBackoffOverride can be set by tests to override the
  1075  // behavior of sleepAndIncreaseBackoff
  1076  var sleepAndIncreaseBackoffOverride func(
  1077  	ctx context.Context, backoff time.Duration) (time.Duration, error)
  1078  
  1079  func sleepAndIncreaseBackoff(ctx context.Context, backoff time.Duration) (time.Duration, error) {
  1080  	if sleepAndIncreaseBackoffOverride != nil {
  1081  		return sleepAndIncreaseBackoffOverride(ctx, backoff)
  1082  	}
  1083  	if backoff == 0 {
  1084  		return backoffStart, nil
  1085  	}
  1086  
  1087  	select {
  1088  	case <-time.After(backoff):
  1089  	case <-ctx.Done():
  1090  		return 0, ctx.Err()
  1091  	}
  1092  
  1093  	// Keep track of the amount of time spend sleeping in retry backoff. Ignore if context was
  1094  	// canceled.
  1095  	retryBackoffDuration.Observe(backoff.Seconds())
  1096  
  1097  	// When changing this formula, update the buckets of the retryBackoffDuration metric too.
  1098  	if backoff < 5*time.Second {
  1099  		return backoff * 2, nil
  1100  	} else if backoff < 30*time.Second {
  1101  		return backoff + 5*time.Second, nil
  1102  	}
  1103  	return backoff, nil
  1104  }
  1105  
  1106  // zkResult contains the result of a ZooKeeper lookup (when we're looking for
  1107  // the meta region or the HMaster).
  1108  type zkResult struct {
  1109  	addr string
  1110  	err  error
  1111  }
  1112  
  1113  // zkLookup asynchronously looks up the meta region or HMaster in ZooKeeper.
  1114  func (c *client) zkLookup(ctx context.Context, resource zk.ResourceName) (string, error) {
  1115  	// We make this a buffered channel so that if we stop waiting due to a
  1116  	// timeout, we won't block the zkLookupSync() that we start in a
  1117  	// separate goroutine.
  1118  	reschan := make(chan zkResult, 1)
  1119  	go func() {
  1120  		addr, err := c.zkClient.LocateResource(resource.Prepend(c.zkRoot))
  1121  		// This is guaranteed to never block as the channel is always buffered.
  1122  		reschan <- zkResult{addr, err}
  1123  	}()
  1124  	select {
  1125  	case res := <-reschan:
  1126  		return res.addr, res.err
  1127  	case <-ctx.Done():
  1128  		return "", ctx.Err()
  1129  	}
  1130  }