001/*
002 * Licensed to the Apache Software Foundation (ASF) under one
003 * or more contributor license agreements.  See the NOTICE file
004 * distributed with this work for additional information
005 * regarding copyright ownership.  The ASF licenses this file
006 * to you under the Apache License, Version 2.0 (the
007 * "License"); you may not use this file except in compliance
008 * with the License.  You may obtain a copy of the License at
009 *
010 *     http://www.apache.org/licenses/LICENSE-2.0
011 *
012 * Unless required by applicable law or agreed to in writing, software
013 * distributed under the License is distributed on an "AS IS" BASIS,
014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
015 * See the License for the specific language governing permissions and
016 * limitations under the License.
017 */
018package org.apache.hadoop.hbase.client;
019
020import static java.util.stream.Collectors.toList;
021import static org.apache.hadoop.hbase.HConstants.EMPTY_END_ROW;
022import static org.apache.hadoop.hbase.HConstants.EMPTY_START_ROW;
023import static org.apache.hadoop.hbase.util.FutureUtils.addListener;
024
025import java.io.IOException;
026import java.lang.reflect.UndeclaredThrowableException;
027import java.net.InetAddress;
028import java.net.UnknownHostException;
029import java.util.Arrays;
030import java.util.List;
031import java.util.Optional;
032import java.util.concurrent.CompletableFuture;
033import java.util.concurrent.ExecutorService;
034import java.util.concurrent.ThreadLocalRandom;
035import java.util.concurrent.TimeUnit;
036import java.util.concurrent.atomic.AtomicReference;
037import java.util.function.Function;
038import java.util.function.Predicate;
039import java.util.function.Supplier;
040import org.apache.hadoop.conf.Configuration;
041import org.apache.hadoop.hbase.Cell;
042import org.apache.hadoop.hbase.CellComparator;
043import org.apache.hadoop.hbase.HConstants;
044import org.apache.hadoop.hbase.PrivateCellUtil;
045import org.apache.hadoop.hbase.RegionLocations;
046import org.apache.hadoop.hbase.ServerName;
047import org.apache.hadoop.hbase.TableName;
048import org.apache.hadoop.hbase.client.metrics.ScanMetrics;
049import org.apache.hadoop.hbase.ipc.HBaseRpcController;
050import org.apache.hadoop.hbase.security.User;
051import org.apache.hadoop.hbase.security.UserProvider;
052import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
053import org.apache.hadoop.hbase.shaded.protobuf.ResponseConverter;
054import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService;
055import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos;
056import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ClientService;
057import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanResponse;
058import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MasterService;
059import org.apache.hadoop.hbase.util.Bytes;
060import org.apache.hadoop.hbase.util.ReflectionUtils;
061import org.apache.hadoop.ipc.RemoteException;
062import org.apache.hadoop.net.DNS;
063import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
064import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException;
065import org.apache.hbase.thirdparty.io.netty.util.Timer;
066import org.apache.yetus.audience.InterfaceAudience;
067import org.slf4j.Logger;
068import org.slf4j.LoggerFactory;
069
070/**
071 * Utility used by client connections.
072 */
073@InterfaceAudience.Private
074public final class ConnectionUtils {
075
076  private static final Logger LOG = LoggerFactory.getLogger(ConnectionUtils.class);
077
078  private ConnectionUtils() {
079  }
080
081  /**
082   * Calculate pause time. Built on {@link HConstants#RETRY_BACKOFF}.
083   * @param pause time to pause
084   * @param tries amount of tries
085   * @return How long to wait after <code>tries</code> retries
086   */
087  public static long getPauseTime(final long pause, final int tries) {
088    int ntries = tries;
089    if (ntries >= HConstants.RETRY_BACKOFF.length) {
090      ntries = HConstants.RETRY_BACKOFF.length - 1;
091    }
092    if (ntries < 0) {
093      ntries = 0;
094    }
095
096    long normalPause = pause * HConstants.RETRY_BACKOFF[ntries];
097    // 1% possible jitter
098    long jitter = (long) (normalPause * ThreadLocalRandom.current().nextFloat() * 0.01f);
099    return normalPause + jitter;
100  }
101
102  /**
103   * @param conn The connection for which to replace the generator.
104   * @param cnm Replaces the nonce generator used, for testing.
105   * @return old nonce generator.
106   */
107  public static NonceGenerator injectNonceGeneratorForTesting(ClusterConnection conn,
108      NonceGenerator cnm) {
109    return ConnectionImplementation.injectNonceGeneratorForTesting(conn, cnm);
110  }
111
112  /**
113   * Changes the configuration to set the number of retries needed when using Connection internally,
114   * e.g. for updating catalog tables, etc. Call this method before we create any Connections.
115   * @param c The Configuration instance to set the retries into.
116   * @param log Used to log what we set in here.
117   */
118  public static void setServerSideHConnectionRetriesConfig(final Configuration c, final String sn,
119      final Logger log) {
120    // TODO: Fix this. Not all connections from server side should have 10 times the retries.
121    int hcRetries = c.getInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER,
122      HConstants.DEFAULT_HBASE_CLIENT_RETRIES_NUMBER);
123    // Go big. Multiply by 10. If we can't get to meta after this many retries
124    // then something seriously wrong.
125    int serversideMultiplier = c.getInt(HConstants.HBASE_CLIENT_SERVERSIDE_RETRIES_MULTIPLIER,
126      HConstants.DEFAULT_HBASE_CLIENT_SERVERSIDE_RETRIES_MULTIPLIER);
127    int retries = hcRetries * serversideMultiplier;
128    c.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, retries);
129    log.info(sn + " server-side Connection retries=" + retries);
130  }
131
132  /**
133   * A ClusterConnection that will short-circuit RPC making direct invocations against the localhost
134   * if the invocation target is 'this' server; save on network and protobuf invocations.
135   */
136  // TODO This has to still do PB marshalling/unmarshalling stuff. Check how/whether we can avoid.
137  // Class is visible so can assert we are short-circuiting when expected.
138  public static class ShortCircuitingClusterConnection extends ConnectionImplementation {
139    private final ServerName serverName;
140    private final AdminService.BlockingInterface localHostAdmin;
141    private final ClientService.BlockingInterface localHostClient;
142
143    private ShortCircuitingClusterConnection(Configuration conf, ExecutorService pool, User user,
144        ServerName serverName, AdminService.BlockingInterface admin,
145        ClientService.BlockingInterface client) throws IOException {
146      super(conf, pool, user);
147      this.serverName = serverName;
148      this.localHostAdmin = admin;
149      this.localHostClient = client;
150    }
151
152    @Override
153    public AdminService.BlockingInterface getAdmin(ServerName sn) throws IOException {
154      return serverName.equals(sn) ? this.localHostAdmin : super.getAdmin(sn);
155    }
156
157    @Override
158    public ClientService.BlockingInterface getClient(ServerName sn) throws IOException {
159      return serverName.equals(sn) ? this.localHostClient : super.getClient(sn);
160    }
161
162    @Override
163    public MasterKeepAliveConnection getMaster() throws IOException {
164      if (this.localHostClient instanceof MasterService.BlockingInterface) {
165        return new ShortCircuitMasterConnection(
166          (MasterService.BlockingInterface) this.localHostClient);
167      }
168      return super.getMaster();
169    }
170  }
171
172  /**
173   * Creates a short-circuit connection that can bypass the RPC layer (serialization,
174   * deserialization, networking, etc..) when talking to a local server.
175   * @param conf the current configuration
176   * @param pool the thread pool to use for batch operations
177   * @param user the user the connection is for
178   * @param serverName the local server name
179   * @param admin the admin interface of the local server
180   * @param client the client interface of the local server
181   * @return an short-circuit connection.
182   * @throws IOException if IO failure occurred
183   */
184  public static ClusterConnection createShortCircuitConnection(final Configuration conf,
185      ExecutorService pool, User user, final ServerName serverName,
186      final AdminService.BlockingInterface admin, final ClientService.BlockingInterface client)
187      throws IOException {
188    if (user == null) {
189      user = UserProvider.instantiate(conf).getCurrent();
190    }
191    return new ShortCircuitingClusterConnection(conf, pool, user, serverName, admin, client);
192  }
193
194  /**
195   * Setup the connection class, so that it will not depend on master being online. Used for testing
196   * @param conf configuration to set
197   */
198  public static void setupMasterlessConnection(Configuration conf) {
199    conf.set(ClusterConnection.HBASE_CLIENT_CONNECTION_IMPL, MasterlessConnection.class.getName());
200  }
201
202  /**
203   * Some tests shut down the master. But table availability is a master RPC which is performed on
204   * region re-lookups.
205   */
206  static class MasterlessConnection extends ConnectionImplementation {
207    MasterlessConnection(Configuration conf, ExecutorService pool, User user) throws IOException {
208      super(conf, pool, user);
209    }
210
211    @Override
212    public boolean isTableDisabled(TableName tableName) throws IOException {
213      // treat all tables as enabled
214      return false;
215    }
216  }
217
218  /**
219   * Return retires + 1. The returned value will be in range [1, Integer.MAX_VALUE].
220   */
221  static int retries2Attempts(int retries) {
222    return Math.max(1, retries == Integer.MAX_VALUE ? Integer.MAX_VALUE : retries + 1);
223  }
224
225  /**
226   * Get a unique key for the rpc stub to the given server.
227   */
228  static String getStubKey(String serviceName, ServerName serverName, boolean hostnameCanChange) {
229    // Sometimes, servers go down and they come back up with the same hostname but a different
230    // IP address. Force a resolution of the rsHostname by trying to instantiate an
231    // InetSocketAddress, and this way we will rightfully get a new stubKey.
232    // Also, include the hostname in the key so as to take care of those cases where the
233    // DNS name is different but IP address remains the same.
234    String hostname = serverName.getHostname();
235    int port = serverName.getPort();
236    if (hostnameCanChange) {
237      try {
238        InetAddress ip = InetAddress.getByName(hostname);
239        return serviceName + "@" + hostname + "-" + ip.getHostAddress() + ":" + port;
240      } catch (UnknownHostException e) {
241        LOG.warn("Can not resolve " + hostname + ", please check your network", e);
242      }
243    }
244    return serviceName + "@" + hostname + ":" + port;
245  }
246
247  static void checkHasFamilies(Mutation mutation) {
248    Preconditions.checkArgument(mutation.numFamilies() > 0,
249      "Invalid arguments to %s, zero columns specified", mutation.toString());
250  }
251
252  /** Dummy nonce generator for disabled nonces. */
253  static final NonceGenerator NO_NONCE_GENERATOR = new NonceGenerator() {
254
255    @Override
256    public long newNonce() {
257      return HConstants.NO_NONCE;
258    }
259
260    @Override
261    public long getNonceGroup() {
262      return HConstants.NO_NONCE;
263    }
264  };
265
266  // A byte array in which all elements are the max byte, and it is used to
267  // construct closest front row
268  static final byte[] MAX_BYTE_ARRAY = Bytes.createMaxByteArray(9);
269
270  /**
271   * Create the closest row after the specified row
272   */
273  static byte[] createClosestRowAfter(byte[] row) {
274    return Arrays.copyOf(row, row.length + 1);
275  }
276
277  /**
278   * Create a row before the specified row and very close to the specified row.
279   */
280  static byte[] createCloseRowBefore(byte[] row) {
281    if (row.length == 0) {
282      return MAX_BYTE_ARRAY;
283    }
284    if (row[row.length - 1] == 0) {
285      return Arrays.copyOf(row, row.length - 1);
286    } else {
287      byte[] nextRow = new byte[row.length + MAX_BYTE_ARRAY.length];
288      System.arraycopy(row, 0, nextRow, 0, row.length - 1);
289      nextRow[row.length - 1] = (byte) ((row[row.length - 1] & 0xFF) - 1);
290      System.arraycopy(MAX_BYTE_ARRAY, 0, nextRow, row.length, MAX_BYTE_ARRAY.length);
291      return nextRow;
292    }
293  }
294
295  static boolean isEmptyStartRow(byte[] row) {
296    return Bytes.equals(row, EMPTY_START_ROW);
297  }
298
299  static boolean isEmptyStopRow(byte[] row) {
300    return Bytes.equals(row, EMPTY_END_ROW);
301  }
302
303  static void resetController(HBaseRpcController controller, long timeoutNs, int priority) {
304    controller.reset();
305    if (timeoutNs >= 0) {
306      controller.setCallTimeout(
307        (int) Math.min(Integer.MAX_VALUE, TimeUnit.NANOSECONDS.toMillis(timeoutNs)));
308    }
309    controller.setPriority(priority);
310  }
311
312  static Throwable translateException(Throwable t) {
313    if (t instanceof UndeclaredThrowableException && t.getCause() != null) {
314      t = t.getCause();
315    }
316    if (t instanceof RemoteException) {
317      t = ((RemoteException) t).unwrapRemoteException();
318    }
319    if (t instanceof ServiceException && t.getCause() != null) {
320      t = translateException(t.getCause());
321    }
322    return t;
323  }
324
325  static long calcEstimatedSize(Result rs) {
326    long estimatedHeapSizeOfResult = 0;
327    // We don't make Iterator here
328    for (Cell cell : rs.rawCells()) {
329      estimatedHeapSizeOfResult += cell.heapSize();
330    }
331    return estimatedHeapSizeOfResult;
332  }
333
334  static Result filterCells(Result result, Cell keepCellsAfter) {
335    if (keepCellsAfter == null) {
336      // do not need to filter
337      return result;
338    }
339    // not the same row
340    if (!PrivateCellUtil.matchingRows(keepCellsAfter, result.getRow(), 0, result.getRow().length)) {
341      return result;
342    }
343    Cell[] rawCells = result.rawCells();
344    int index = Arrays.binarySearch(rawCells, keepCellsAfter,
345      CellComparator.getInstance()::compareWithoutRow);
346    if (index < 0) {
347      index = -index - 1;
348    } else {
349      index++;
350    }
351    if (index == 0) {
352      return result;
353    }
354    if (index == rawCells.length) {
355      return null;
356    }
357    return Result.create(Arrays.copyOfRange(rawCells, index, rawCells.length), null,
358      result.isStale(), result.mayHaveMoreCellsInRow());
359  }
360
361  // Add a delta to avoid timeout immediately after a retry sleeping.
362  static final long SLEEP_DELTA_NS = TimeUnit.MILLISECONDS.toNanos(1);
363
364  static Get toCheckExistenceOnly(Get get) {
365    if (get.isCheckExistenceOnly()) {
366      return get;
367    }
368    return ReflectionUtils.newInstance(get.getClass(), get).setCheckExistenceOnly(true);
369  }
370
371  static List<Get> toCheckExistenceOnly(List<Get> gets) {
372    return gets.stream().map(ConnectionUtils::toCheckExistenceOnly).collect(toList());
373  }
374
375  static RegionLocateType getLocateType(Scan scan) {
376    if (scan.isReversed()) {
377      if (isEmptyStartRow(scan.getStartRow())) {
378        return RegionLocateType.BEFORE;
379      } else {
380        return scan.includeStartRow() ? RegionLocateType.CURRENT : RegionLocateType.BEFORE;
381      }
382    } else {
383      return scan.includeStartRow() ? RegionLocateType.CURRENT : RegionLocateType.AFTER;
384    }
385  }
386
387  static boolean noMoreResultsForScan(Scan scan, RegionInfo info) {
388    if (isEmptyStopRow(info.getEndKey())) {
389      return true;
390    }
391    if (isEmptyStopRow(scan.getStopRow())) {
392      return false;
393    }
394    int c = Bytes.compareTo(info.getEndKey(), scan.getStopRow());
395    // 1. if our stop row is less than the endKey of the region
396    // 2. if our stop row is equal to the endKey of the region and we do not include the stop row
397    // for scan.
398    return c > 0 || (c == 0 && !scan.includeStopRow());
399  }
400
401  static boolean noMoreResultsForReverseScan(Scan scan, RegionInfo info) {
402    if (isEmptyStartRow(info.getStartKey())) {
403      return true;
404    }
405    if (isEmptyStopRow(scan.getStopRow())) {
406      return false;
407    }
408    // no need to test the inclusive of the stop row as the start key of a region is included in
409    // the region.
410    return Bytes.compareTo(info.getStartKey(), scan.getStopRow()) <= 0;
411  }
412
413  static <T> CompletableFuture<List<T>> allOf(List<CompletableFuture<T>> futures) {
414    return CompletableFuture.allOf(futures.toArray(new CompletableFuture[0]))
415      .thenApply(v -> futures.stream().map(f -> f.getNow(null)).collect(toList()));
416  }
417
418  public static ScanResultCache createScanResultCache(Scan scan) {
419    if (scan.getAllowPartialResults()) {
420      return new AllowPartialScanResultCache();
421    } else if (scan.getBatch() > 0) {
422      return new BatchScanResultCache(scan.getBatch());
423    } else {
424      return new CompleteScanResultCache();
425    }
426  }
427
428  private static final String MY_ADDRESS = getMyAddress();
429
430  private static String getMyAddress() {
431    try {
432      return DNS.getDefaultHost("default", "default");
433    } catch (UnknownHostException uhe) {
434      LOG.error("cannot determine my address", uhe);
435      return null;
436    }
437  }
438
439  static boolean isRemote(String host) {
440    return !host.equalsIgnoreCase(MY_ADDRESS);
441  }
442
443  static void incRPCCallsMetrics(ScanMetrics scanMetrics, boolean isRegionServerRemote) {
444    if (scanMetrics == null) {
445      return;
446    }
447    scanMetrics.countOfRPCcalls.incrementAndGet();
448    if (isRegionServerRemote) {
449      scanMetrics.countOfRemoteRPCcalls.incrementAndGet();
450    }
451  }
452
453  static void incRPCRetriesMetrics(ScanMetrics scanMetrics, boolean isRegionServerRemote) {
454    if (scanMetrics == null) {
455      return;
456    }
457    scanMetrics.countOfRPCRetries.incrementAndGet();
458    if (isRegionServerRemote) {
459      scanMetrics.countOfRemoteRPCRetries.incrementAndGet();
460    }
461  }
462
463  static void updateResultsMetrics(ScanMetrics scanMetrics, Result[] rrs,
464      boolean isRegionServerRemote) {
465    if (scanMetrics == null || rrs == null || rrs.length == 0) {
466      return;
467    }
468    long resultSize = 0;
469    for (Result rr : rrs) {
470      for (Cell cell : rr.rawCells()) {
471        resultSize += PrivateCellUtil.estimatedSerializedSizeOf(cell);
472      }
473    }
474    scanMetrics.countOfBytesInResults.addAndGet(resultSize);
475    if (isRegionServerRemote) {
476      scanMetrics.countOfBytesInRemoteResults.addAndGet(resultSize);
477    }
478  }
479
480  /**
481   * Use the scan metrics returned by the server to add to the identically named counters in the
482   * client side metrics. If a counter does not exist with the same name as the server side metric,
483   * the attempt to increase the counter will fail.
484   */
485  static void updateServerSideMetrics(ScanMetrics scanMetrics, ScanResponse response) {
486    if (scanMetrics == null || response == null || !response.hasScanMetrics()) {
487      return;
488    }
489    ResponseConverter.getScanMetrics(response).forEach(scanMetrics::addToCounter);
490  }
491
492  static void incRegionCountMetrics(ScanMetrics scanMetrics) {
493    if (scanMetrics == null) {
494      return;
495    }
496    scanMetrics.countOfRegions.incrementAndGet();
497  }
498
499  /**
500   * Connect the two futures, if the src future is done, then mark the dst future as done. And if
501   * the dst future is done, then cancel the src future. This is used for timeline consistent read.
502   * <p/>
503   * Pass empty metrics if you want to link the primary future and the dst future so we will not
504   * increase the hedge read related metrics.
505   */
506  private static <T> void connect(CompletableFuture<T> srcFuture, CompletableFuture<T> dstFuture,
507      Optional<MetricsConnection> metrics) {
508    addListener(srcFuture, (r, e) -> {
509      if (e != null) {
510        dstFuture.completeExceptionally(e);
511      } else {
512        if (dstFuture.complete(r)) {
513          metrics.ifPresent(MetricsConnection::incrHedgedReadWin);
514        }
515      }
516    });
517    // The cancellation may be a dummy one as the dstFuture may be completed by this srcFuture.
518    // Notice that this is a bit tricky, as the execution chain maybe 'complete src -> complete dst
519    // -> cancel src', for now it seems to be fine, as the will use CAS to set the result first in
520    // CompletableFuture. If later this causes problems, we could use whenCompleteAsync to break the
521    // tie.
522    addListener(dstFuture, (r, e) -> srcFuture.cancel(false));
523  }
524
525  private static <T> void sendRequestsToSecondaryReplicas(
526      Function<Integer, CompletableFuture<T>> requestReplica, RegionLocations locs,
527      CompletableFuture<T> future, Optional<MetricsConnection> metrics) {
528    if (future.isDone()) {
529      // do not send requests to secondary replicas if the future is done, i.e, the primary request
530      // has already been finished.
531      return;
532    }
533    for (int replicaId = 1, n = locs.size(); replicaId < n; replicaId++) {
534      CompletableFuture<T> secondaryFuture = requestReplica.apply(replicaId);
535      metrics.ifPresent(MetricsConnection::incrHedgedReadOps);
536      connect(secondaryFuture, future, metrics);
537    }
538  }
539
540  static <T> CompletableFuture<T> timelineConsistentRead(AsyncRegionLocator locator,
541      TableName tableName, Query query, byte[] row, RegionLocateType locateType,
542      Function<Integer, CompletableFuture<T>> requestReplica, long rpcTimeoutNs,
543      long primaryCallTimeoutNs, Timer retryTimer, Optional<MetricsConnection> metrics) {
544    if (query.getConsistency() != Consistency.TIMELINE) {
545      return requestReplica.apply(RegionReplicaUtil.DEFAULT_REPLICA_ID);
546    }
547    // user specifies a replica id explicitly, just send request to the specific replica
548    if (query.getReplicaId() >= 0) {
549      return requestReplica.apply(query.getReplicaId());
550    }
551    // Timeline consistent read, where we may send requests to other region replicas
552    CompletableFuture<T> primaryFuture = requestReplica.apply(RegionReplicaUtil.DEFAULT_REPLICA_ID);
553    CompletableFuture<T> future = new CompletableFuture<>();
554    connect(primaryFuture, future, Optional.empty());
555    long startNs = System.nanoTime();
556    // after the getRegionLocations, all the locations for the replicas of this region should have
557    // been cached, so it is not big deal to locate them again when actually sending requests to
558    // these replicas.
559    addListener(locator.getRegionLocations(tableName, row, locateType, false, rpcTimeoutNs),
560      (locs, error) -> {
561        if (error != null) {
562          LOG.warn(
563            "Failed to locate all the replicas for table={}, row='{}', locateType={}" +
564              " give up timeline consistent read",
565            tableName, Bytes.toStringBinary(row), locateType, error);
566          return;
567        }
568        if (locs.size() <= 1) {
569          LOG.warn(
570            "There are no secondary replicas for region {}, give up timeline consistent read",
571            locs.getDefaultRegionLocation().getRegion());
572          return;
573        }
574        long delayNs = primaryCallTimeoutNs - (System.nanoTime() - startNs);
575        if (delayNs <= 0) {
576          sendRequestsToSecondaryReplicas(requestReplica, locs, future, metrics);
577        } else {
578          retryTimer.newTimeout(
579            timeout -> sendRequestsToSecondaryReplicas(requestReplica, locs, future, metrics),
580            delayNs, TimeUnit.NANOSECONDS);
581        }
582      });
583    return future;
584  }
585
586  // validate for well-formedness
587  static void validatePut(Put put, int maxKeyValueSize) throws IllegalArgumentException {
588    if (put.isEmpty()) {
589      throw new IllegalArgumentException("No columns to insert");
590    }
591    if (maxKeyValueSize > 0) {
592      for (List<Cell> list : put.getFamilyCellMap().values()) {
593        for (Cell cell : list) {
594          if (cell.getSerializedSize() > maxKeyValueSize) {
595            throw new IllegalArgumentException("KeyValue size too large");
596          }
597        }
598      }
599    }
600  }
601
602  /**
603   * Select the priority for the rpc call.
604   * <p/>
605   * The rules are:
606   * <ol>
607   * <li>If user set a priority explicitly, then just use it.</li>
608   * <li>For system table, use {@link HConstants#SYSTEMTABLE_QOS}.</li>
609   * <li>For other tables, use {@link HConstants#NORMAL_QOS}.</li>
610   * </ol>
611   * @param priority the priority set by user, can be {@link HConstants#PRIORITY_UNSET}.
612   * @param tableName the table we operate on
613   */
614  static int calcPriority(int priority, TableName tableName) {
615    if (priority != HConstants.PRIORITY_UNSET) {
616      return priority;
617    } else {
618      return getPriority(tableName);
619    }
620  }
621
622  static int getPriority(TableName tableName) {
623    if (tableName.isSystemTable()) {
624      return HConstants.SYSTEMTABLE_QOS;
625    } else {
626      return HConstants.NORMAL_QOS;
627    }
628  }
629
630  static <T> CompletableFuture<T> getOrFetch(AtomicReference<T> cacheRef,
631      AtomicReference<CompletableFuture<T>> futureRef, boolean reload,
632      Supplier<CompletableFuture<T>> fetch, Predicate<T> validator, String type) {
633    for (;;) {
634      if (!reload) {
635        T value = cacheRef.get();
636        if (value != null && validator.test(value)) {
637          return CompletableFuture.completedFuture(value);
638        }
639      }
640      LOG.trace("{} cache is null, try fetching from registry", type);
641      if (futureRef.compareAndSet(null, new CompletableFuture<>())) {
642        LOG.debug("Start fetching {} from registry", type);
643        CompletableFuture<T> future = futureRef.get();
644        addListener(fetch.get(), (value, error) -> {
645          if (error != null) {
646            LOG.debug("Failed to fetch {} from registry", type, error);
647            futureRef.getAndSet(null).completeExceptionally(error);
648            return;
649          }
650          LOG.debug("The fetched {} is {}", type, value);
651          // Here we update cache before reset future, so it is possible that someone can get a
652          // stale value. Consider this:
653          // 1. update cacheRef
654          // 2. someone clears the cache and relocates again
655          // 3. the futureRef is not null so the old future is used.
656          // 4. we clear futureRef and complete the future in it with the value being
657          // cleared in step 2.
658          // But we do not think it is a big deal as it rarely happens, and even if it happens, the
659          // caller will retry again later, no correctness problems.
660          cacheRef.set(value);
661          futureRef.set(null);
662          future.complete(value);
663        });
664        return future;
665      } else {
666        CompletableFuture<T> future = futureRef.get();
667        if (future != null) {
668          return future;
669        }
670      }
671    }
672  }
673
674  static void updateStats(Optional<ServerStatisticTracker> optStats,
675      Optional<MetricsConnection> optMetrics, ServerName serverName, MultiResponse resp) {
676    if (!optStats.isPresent() && !optMetrics.isPresent()) {
677      // ServerStatisticTracker and MetricsConnection are both not present, just return
678      return;
679    }
680    resp.getResults().forEach((regionName, regionResult) -> {
681      ClientProtos.RegionLoadStats stat = regionResult.getStat();
682      if (stat == null) {
683        LOG.error("No ClientProtos.RegionLoadStats found for server={}, region={}", serverName,
684          Bytes.toStringBinary(regionName));
685        return;
686      }
687      RegionLoadStats regionLoadStats = ProtobufUtil.createRegionLoadStats(stat);
688      optStats.ifPresent(
689        stats -> ResultStatsUtil.updateStats(stats, serverName, regionName, regionLoadStats));
690      optMetrics.ifPresent(
691        metrics -> ResultStatsUtil.updateStats(metrics, serverName, regionName, regionLoadStats));
692    });
693  }
694}