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}