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}