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