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 */ 018 019package org.apache.hadoop.hbase.ipc; 020 021import static org.apache.hadoop.hbase.ipc.IPCUtil.toIOE; 022import static org.apache.hadoop.hbase.ipc.IPCUtil.wrapException; 023 024import java.io.IOException; 025import java.net.InetSocketAddress; 026import java.net.SocketAddress; 027import java.net.UnknownHostException; 028import java.util.Collection; 029import java.util.concurrent.Executors; 030import java.util.concurrent.ScheduledExecutorService; 031import java.util.concurrent.ScheduledFuture; 032import java.util.concurrent.TimeUnit; 033import java.util.concurrent.atomic.AtomicInteger; 034import org.apache.hadoop.conf.Configuration; 035import org.apache.hadoop.hbase.HConstants; 036import org.apache.hadoop.hbase.ServerName; 037import org.apache.hadoop.hbase.client.MetricsConnection; 038import org.apache.hadoop.hbase.codec.Codec; 039import org.apache.hadoop.hbase.codec.KeyValueCodec; 040import org.apache.hadoop.hbase.security.User; 041import org.apache.hadoop.hbase.security.UserProvider; 042import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; 043import org.apache.hadoop.hbase.util.PoolMap; 044import org.apache.hadoop.hbase.util.Threads; 045import org.apache.hadoop.io.compress.CompressionCodec; 046import org.apache.hadoop.ipc.RemoteException; 047import org.apache.yetus.audience.InterfaceAudience; 048import org.slf4j.Logger; 049import org.slf4j.LoggerFactory; 050 051import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; 052import org.apache.hbase.thirdparty.com.google.common.base.Preconditions; 053import org.apache.hbase.thirdparty.com.google.common.cache.CacheBuilder; 054import org.apache.hbase.thirdparty.com.google.common.cache.CacheLoader; 055import org.apache.hbase.thirdparty.com.google.common.cache.LoadingCache; 056import org.apache.hbase.thirdparty.com.google.protobuf.BlockingRpcChannel; 057import org.apache.hbase.thirdparty.com.google.protobuf.Descriptors; 058import org.apache.hbase.thirdparty.com.google.protobuf.Message; 059import org.apache.hbase.thirdparty.com.google.protobuf.RpcCallback; 060import org.apache.hbase.thirdparty.com.google.protobuf.RpcChannel; 061import org.apache.hbase.thirdparty.com.google.protobuf.RpcController; 062import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException; 063import org.apache.hbase.thirdparty.io.netty.util.HashedWheelTimer; 064 065import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos; 066 067/** 068 * Provides the basics for a RpcClient implementation like configuration and Logging. 069 * <p> 070 * Locking schema of the current IPC implementation 071 * <ul> 072 * <li>There is a lock in {@link AbstractRpcClient} to protect the fetching or creating 073 * connection.</li> 074 * <li>There is a lock in {@link Call} to make sure that we can only finish the call once.</li> 075 * <li>The same for {@link HBaseRpcController} as {@link Call}. And see the comment of 076 * {@link HBaseRpcController#notifyOnCancel(RpcCallback, HBaseRpcController.CancellationCallback)} 077 * of how to deal with cancel.</li> 078 * <li>For connection implementation, the construction of a connection should be as fast as possible 079 * because the creation is protected under a lock. Connect to remote side when needed. There is no 080 * forced locking schema for a connection implementation.</li> 081 * <li>For the locking order, the {@link Call} and {@link HBaseRpcController}'s lock should be held 082 * at last. So the callbacks in {@link Call} and {@link HBaseRpcController} should be execute 083 * outside the lock in {@link Call} and {@link HBaseRpcController} which means the implementations 084 * of the callbacks are free to hold any lock.</li> 085 * </ul> 086 * @since 2.0.0 087 */ 088@InterfaceAudience.Private 089public abstract class AbstractRpcClient<T extends RpcConnection> implements RpcClient { 090 // Log level is being changed in tests 091 public static final Logger LOG = LoggerFactory.getLogger(AbstractRpcClient.class); 092 093 protected static final HashedWheelTimer WHEEL_TIMER = new HashedWheelTimer( 094 Threads.newDaemonThreadFactory("RpcClient-timer"), 10, TimeUnit.MILLISECONDS); 095 096 private static final ScheduledExecutorService IDLE_CONN_SWEEPER = Executors 097 .newScheduledThreadPool(1, Threads.newDaemonThreadFactory("Idle-Rpc-Conn-Sweeper")); 098 099 protected boolean running = true; // if client runs 100 101 protected final Configuration conf; 102 protected final String clusterId; 103 protected final SocketAddress localAddr; 104 protected final MetricsConnection metrics; 105 106 protected final UserProvider userProvider; 107 protected final CellBlockBuilder cellBlockBuilder; 108 109 protected final int minIdleTimeBeforeClose; // if the connection is idle for more than this 110 // time (in ms), it will be closed at any moment. 111 protected final int maxRetries; // the max. no. of retries for socket connections 112 protected final long failureSleep; // Time to sleep before retry on failure. 113 protected final boolean tcpNoDelay; // if T then disable Nagle's Algorithm 114 protected final boolean tcpKeepAlive; // if T then use keepalives 115 protected final Codec codec; 116 protected final CompressionCodec compressor; 117 protected final boolean fallbackAllowed; 118 119 protected final FailedServers failedServers; 120 121 protected final int connectTO; 122 protected final int readTO; 123 protected final int writeTO; 124 125 protected final PoolMap<ConnectionId, T> connections; 126 127 private final AtomicInteger callIdCnt = new AtomicInteger(0); 128 129 private final ScheduledFuture<?> cleanupIdleConnectionTask; 130 131 private int maxConcurrentCallsPerServer; 132 133 private static final LoadingCache<InetSocketAddress, AtomicInteger> concurrentCounterCache = 134 CacheBuilder.newBuilder().expireAfterAccess(1, TimeUnit.HOURS). 135 build(new CacheLoader<InetSocketAddress, AtomicInteger>() { 136 @Override public AtomicInteger load(InetSocketAddress key) throws Exception { 137 return new AtomicInteger(0); 138 } 139 }); 140 141 /** 142 * Construct an IPC client for the cluster <code>clusterId</code> 143 * @param conf configuration 144 * @param clusterId the cluster id 145 * @param localAddr client socket bind address. 146 * @param metrics the connection metrics 147 */ 148 public AbstractRpcClient(Configuration conf, String clusterId, SocketAddress localAddr, 149 MetricsConnection metrics) { 150 this.userProvider = UserProvider.instantiate(conf); 151 this.localAddr = localAddr; 152 this.tcpKeepAlive = conf.getBoolean("hbase.ipc.client.tcpkeepalive", true); 153 this.clusterId = clusterId != null ? clusterId : HConstants.CLUSTER_ID_DEFAULT; 154 this.failureSleep = conf.getLong(HConstants.HBASE_CLIENT_PAUSE, 155 HConstants.DEFAULT_HBASE_CLIENT_PAUSE); 156 this.maxRetries = conf.getInt("hbase.ipc.client.connect.max.retries", 0); 157 this.tcpNoDelay = conf.getBoolean("hbase.ipc.client.tcpnodelay", true); 158 this.cellBlockBuilder = new CellBlockBuilder(conf); 159 160 this.minIdleTimeBeforeClose = conf.getInt(IDLE_TIME, 120000); // 2 minutes 161 this.conf = conf; 162 this.codec = getCodec(); 163 this.compressor = getCompressor(conf); 164 this.fallbackAllowed = conf.getBoolean(IPC_CLIENT_FALLBACK_TO_SIMPLE_AUTH_ALLOWED_KEY, 165 IPC_CLIENT_FALLBACK_TO_SIMPLE_AUTH_ALLOWED_DEFAULT); 166 this.failedServers = new FailedServers(conf); 167 this.connectTO = conf.getInt(SOCKET_TIMEOUT_CONNECT, DEFAULT_SOCKET_TIMEOUT_CONNECT); 168 this.readTO = conf.getInt(SOCKET_TIMEOUT_READ, DEFAULT_SOCKET_TIMEOUT_READ); 169 this.writeTO = conf.getInt(SOCKET_TIMEOUT_WRITE, DEFAULT_SOCKET_TIMEOUT_WRITE); 170 this.metrics = metrics; 171 this.maxConcurrentCallsPerServer = conf.getInt( 172 HConstants.HBASE_CLIENT_PERSERVER_REQUESTS_THRESHOLD, 173 HConstants.DEFAULT_HBASE_CLIENT_PERSERVER_REQUESTS_THRESHOLD); 174 175 this.connections = new PoolMap<>(getPoolType(conf), getPoolSize(conf)); 176 177 this.cleanupIdleConnectionTask = IDLE_CONN_SWEEPER.scheduleAtFixedRate(new Runnable() { 178 179 @Override 180 public void run() { 181 cleanupIdleConnections(); 182 } 183 }, minIdleTimeBeforeClose, minIdleTimeBeforeClose, TimeUnit.MILLISECONDS); 184 185 if (LOG.isDebugEnabled()) { 186 LOG.debug("Codec=" + this.codec + ", compressor=" + this.compressor + ", tcpKeepAlive=" 187 + this.tcpKeepAlive + ", tcpNoDelay=" + this.tcpNoDelay + ", connectTO=" + this.connectTO 188 + ", readTO=" + this.readTO + ", writeTO=" + this.writeTO + ", minIdleTimeBeforeClose=" 189 + this.minIdleTimeBeforeClose + ", maxRetries=" + this.maxRetries + ", fallbackAllowed=" 190 + this.fallbackAllowed + ", bind address=" 191 + (this.localAddr != null ? this.localAddr : "null")); 192 } 193 } 194 195 private void cleanupIdleConnections() { 196 long closeBeforeTime = EnvironmentEdgeManager.currentTime() - minIdleTimeBeforeClose; 197 synchronized (connections) { 198 for (T conn : connections.values()) { 199 // Remove connection if it has not been chosen by anyone for more than maxIdleTime, and the 200 // connection itself has already shutdown. The latter check is because we may still 201 // have some pending calls on connection so we should not shutdown the connection outside. 202 // The connection itself will disconnect if there is no pending call for maxIdleTime. 203 if (conn.getLastTouched() < closeBeforeTime && !conn.isActive()) { 204 if (LOG.isTraceEnabled()) { 205 LOG.trace("Cleanup idle connection to {}", conn.remoteId().address); 206 } 207 connections.removeValue(conn.remoteId(), conn); 208 conn.cleanupConnection(); 209 } 210 } 211 } 212 } 213 214 @VisibleForTesting 215 public static String getDefaultCodec(final Configuration c) { 216 // If "hbase.client.default.rpc.codec" is empty string -- you can't set it to null because 217 // Configuration will complain -- then no default codec (and we'll pb everything). Else 218 // default is KeyValueCodec 219 return c.get(DEFAULT_CODEC_CLASS, KeyValueCodec.class.getCanonicalName()); 220 } 221 222 /** 223 * Encapsulate the ugly casting and RuntimeException conversion in private method. 224 * @return Codec to use on this client. 225 */ 226 Codec getCodec() { 227 // For NO CODEC, "hbase.client.rpc.codec" must be configured with empty string AND 228 // "hbase.client.default.rpc.codec" also -- because default is to do cell block encoding. 229 String className = conf.get(HConstants.RPC_CODEC_CONF_KEY, getDefaultCodec(this.conf)); 230 if (className == null || className.length() == 0) { 231 return null; 232 } 233 try { 234 return (Codec) Class.forName(className).getDeclaredConstructor().newInstance(); 235 } catch (Exception e) { 236 throw new RuntimeException("Failed getting codec " + className, e); 237 } 238 } 239 240 @Override 241 public boolean hasCellBlockSupport() { 242 return this.codec != null; 243 } 244 245 // for writing tests that want to throw exception when connecting. 246 @VisibleForTesting 247 boolean isTcpNoDelay() { 248 return tcpNoDelay; 249 } 250 251 /** 252 * Encapsulate the ugly casting and RuntimeException conversion in private method. 253 * @param conf configuration 254 * @return The compressor to use on this client. 255 */ 256 private static CompressionCodec getCompressor(final Configuration conf) { 257 String className = conf.get("hbase.client.rpc.compressor", null); 258 if (className == null || className.isEmpty()) { 259 return null; 260 } 261 try { 262 return (CompressionCodec) Class.forName(className).getDeclaredConstructor().newInstance(); 263 } catch (Exception e) { 264 throw new RuntimeException("Failed getting compressor " + className, e); 265 } 266 } 267 268 /** 269 * Return the pool type specified in the configuration, which must be set to either 270 * {@link org.apache.hadoop.hbase.util.PoolMap.PoolType#RoundRobin} or 271 * {@link org.apache.hadoop.hbase.util.PoolMap.PoolType#ThreadLocal}, otherwise default to the 272 * former. For applications with many user threads, use a small round-robin pool. For applications 273 * with few user threads, you may want to try using a thread-local pool. In any case, the number 274 * of {@link org.apache.hadoop.hbase.ipc.RpcClient} instances should not exceed the operating 275 * system's hard limit on the number of connections. 276 * @param config configuration 277 * @return either a {@link org.apache.hadoop.hbase.util.PoolMap.PoolType#RoundRobin} or 278 * {@link org.apache.hadoop.hbase.util.PoolMap.PoolType#ThreadLocal} 279 */ 280 private static PoolMap.PoolType getPoolType(Configuration config) { 281 return PoolMap.PoolType.valueOf(config.get(HConstants.HBASE_CLIENT_IPC_POOL_TYPE), 282 PoolMap.PoolType.RoundRobin, PoolMap.PoolType.ThreadLocal); 283 } 284 285 /** 286 * Return the pool size specified in the configuration, which is applicable only if the pool type 287 * is {@link org.apache.hadoop.hbase.util.PoolMap.PoolType#RoundRobin}. 288 * @param config configuration 289 * @return the maximum pool size 290 */ 291 private static int getPoolSize(Configuration config) { 292 return config.getInt(HConstants.HBASE_CLIENT_IPC_POOL_SIZE, 1); 293 } 294 295 private int nextCallId() { 296 int id, next; 297 do { 298 id = callIdCnt.get(); 299 next = id < Integer.MAX_VALUE ? id + 1 : 0; 300 } while (!callIdCnt.compareAndSet(id, next)); 301 return id; 302 } 303 304 /** 305 * Make a blocking call. Throws exceptions if there are network problems or if the remote code 306 * threw an exception. 307 * @param ticket Be careful which ticket you pass. A new user will mean a new Connection. 308 * {@link UserProvider#getCurrent()} makes a new instance of User each time so will be a 309 * new Connection each time. 310 * @return A pair with the Message response and the Cell data (if any). 311 */ 312 private Message callBlockingMethod(Descriptors.MethodDescriptor md, HBaseRpcController hrc, 313 Message param, Message returnType, final User ticket, final InetSocketAddress isa) 314 throws ServiceException { 315 BlockingRpcCallback<Message> done = new BlockingRpcCallback<>(); 316 callMethod(md, hrc, param, returnType, ticket, isa, done); 317 Message val; 318 try { 319 val = done.get(); 320 } catch (IOException e) { 321 throw new ServiceException(e); 322 } 323 if (hrc.failed()) { 324 throw new ServiceException(hrc.getFailed()); 325 } else { 326 return val; 327 } 328 } 329 330 /** 331 * Get a connection from the pool, or create a new one and add it to the pool. Connections to a 332 * given host/port are reused. 333 */ 334 private T getConnection(ConnectionId remoteId) throws IOException { 335 if (failedServers.isFailedServer(remoteId.getAddress())) { 336 if (LOG.isDebugEnabled()) { 337 LOG.debug("Not trying to connect to " + remoteId.address 338 + " this server is in the failed servers list"); 339 } 340 throw new FailedServerException( 341 "This server is in the failed servers list: " + remoteId.address); 342 } 343 T conn; 344 synchronized (connections) { 345 if (!running) { 346 throw new StoppedRpcClientException(); 347 } 348 conn = connections.get(remoteId); 349 if (conn == null) { 350 conn = createConnection(remoteId); 351 connections.put(remoteId, conn); 352 } 353 conn.setLastTouched(EnvironmentEdgeManager.currentTime()); 354 } 355 return conn; 356 } 357 358 /** 359 * Not connected. 360 */ 361 protected abstract T createConnection(ConnectionId remoteId) throws IOException; 362 363 private void onCallFinished(Call call, HBaseRpcController hrc, InetSocketAddress addr, 364 RpcCallback<Message> callback) { 365 call.callStats.setCallTimeMs(EnvironmentEdgeManager.currentTime() - call.getStartTime()); 366 if (metrics != null) { 367 metrics.updateRpc(call.md, call.param, call.callStats); 368 } 369 if (LOG.isTraceEnabled()) { 370 LOG.trace( 371 "Call: " + call.md.getName() + ", callTime: " + call.callStats.getCallTimeMs() + "ms"); 372 } 373 if (call.error != null) { 374 if (call.error instanceof RemoteException) { 375 call.error.fillInStackTrace(); 376 hrc.setFailed(call.error); 377 } else { 378 hrc.setFailed(wrapException(addr, call.error)); 379 } 380 callback.run(null); 381 } else { 382 hrc.setDone(call.cells); 383 callback.run(call.response); 384 } 385 } 386 387 Call callMethod(final Descriptors.MethodDescriptor md, final HBaseRpcController hrc, 388 final Message param, Message returnType, final User ticket, final InetSocketAddress addr, 389 final RpcCallback<Message> callback) { 390 final MetricsConnection.CallStats cs = MetricsConnection.newCallStats(); 391 cs.setStartTime(EnvironmentEdgeManager.currentTime()); 392 393 if (param instanceof ClientProtos.MultiRequest) { 394 ClientProtos.MultiRequest req = (ClientProtos.MultiRequest) param; 395 int numActions = 0; 396 for (ClientProtos.RegionAction regionAction : req.getRegionActionList()) { 397 numActions += regionAction.getActionCount(); 398 } 399 400 cs.setNumActionsPerServer(numActions); 401 } 402 403 final AtomicInteger counter = concurrentCounterCache.getUnchecked(addr); 404 Call call = new Call(nextCallId(), md, param, hrc.cellScanner(), returnType, 405 hrc.getCallTimeout(), hrc.getPriority(), new RpcCallback<Call>() { 406 @Override 407 public void run(Call call) { 408 counter.decrementAndGet(); 409 onCallFinished(call, hrc, addr, callback); 410 } 411 }, cs); 412 ConnectionId remoteId = new ConnectionId(ticket, md.getService().getName(), addr); 413 int count = counter.incrementAndGet(); 414 try { 415 if (count > maxConcurrentCallsPerServer) { 416 throw new ServerTooBusyException(addr, count); 417 } 418 cs.setConcurrentCallsPerServer(count); 419 T connection = getConnection(remoteId); 420 connection.sendRequest(call, hrc); 421 } catch (Exception e) { 422 call.setException(toIOE(e)); 423 } 424 return call; 425 } 426 427 InetSocketAddress createAddr(ServerName sn) throws UnknownHostException { 428 InetSocketAddress addr = new InetSocketAddress(sn.getHostname(), sn.getPort()); 429 if (addr.isUnresolved()) { 430 throw new UnknownHostException("can not resolve " + sn.getServerName()); 431 } 432 return addr; 433 } 434 435 /** 436 * Interrupt the connections to the given ip:port server. This should be called if the server is 437 * known as actually dead. This will not prevent current operation to be retried, and, depending 438 * on their own behavior, they may retry on the same server. This can be a feature, for example at 439 * startup. In any case, they're likely to get connection refused (if the process died) or no 440 * route to host: i.e. their next retries should be faster and with a safe exception. 441 */ 442 @Override 443 public void cancelConnections(ServerName sn) { 444 synchronized (connections) { 445 for (T connection : connections.values()) { 446 ConnectionId remoteId = connection.remoteId(); 447 if (remoteId.address.getPort() == sn.getPort() 448 && remoteId.address.getHostName().equals(sn.getHostname())) { 449 LOG.info("The server on " + sn.toString() + " is dead - stopping the connection " 450 + connection.remoteId); 451 connections.removeValue(remoteId, connection); 452 connection.shutdown(); 453 connection.cleanupConnection(); 454 } 455 } 456 } 457 } 458 /** 459 * Configure an hbase rpccontroller 460 * @param controller to configure 461 * @param channelOperationTimeout timeout for operation 462 * @return configured controller 463 */ 464 static HBaseRpcController configureHBaseRpcController( 465 RpcController controller, int channelOperationTimeout) { 466 HBaseRpcController hrc; 467 if (controller != null && controller instanceof HBaseRpcController) { 468 hrc = (HBaseRpcController) controller; 469 if (!hrc.hasCallTimeout()) { 470 hrc.setCallTimeout(channelOperationTimeout); 471 } 472 } else { 473 hrc = new HBaseRpcControllerImpl(); 474 hrc.setCallTimeout(channelOperationTimeout); 475 } 476 return hrc; 477 } 478 479 protected abstract void closeInternal(); 480 481 @Override 482 public void close() { 483 if (LOG.isDebugEnabled()) { 484 LOG.debug("Stopping rpc client"); 485 } 486 Collection<T> connToClose; 487 synchronized (connections) { 488 if (!running) { 489 return; 490 } 491 running = false; 492 connToClose = connections.values(); 493 connections.clear(); 494 } 495 cleanupIdleConnectionTask.cancel(true); 496 for (T conn : connToClose) { 497 conn.shutdown(); 498 } 499 closeInternal(); 500 for (T conn : connToClose) { 501 conn.cleanupConnection(); 502 } 503 } 504 505 @Override 506 public BlockingRpcChannel createBlockingRpcChannel(final ServerName sn, final User ticket, 507 int rpcTimeout) throws UnknownHostException { 508 return new BlockingRpcChannelImplementation(this, createAddr(sn), ticket, rpcTimeout); 509 } 510 511 @Override 512 public RpcChannel createRpcChannel(ServerName sn, User user, int rpcTimeout) 513 throws UnknownHostException { 514 return new RpcChannelImplementation(this, createAddr(sn), user, rpcTimeout); 515 } 516 517 private static class AbstractRpcChannel { 518 519 protected final InetSocketAddress addr; 520 521 protected final AbstractRpcClient<?> rpcClient; 522 523 protected final User ticket; 524 525 protected final int rpcTimeout; 526 527 protected AbstractRpcChannel(AbstractRpcClient<?> rpcClient, InetSocketAddress addr, 528 User ticket, int rpcTimeout) { 529 this.addr = addr; 530 this.rpcClient = rpcClient; 531 this.ticket = ticket; 532 this.rpcTimeout = rpcTimeout; 533 } 534 535 /** 536 * Configure an rpc controller 537 * @param controller to configure 538 * @return configured rpc controller 539 */ 540 protected HBaseRpcController configureRpcController(RpcController controller) { 541 HBaseRpcController hrc; 542 // TODO: Ideally we should not use an RpcController other than HBaseRpcController at client 543 // side. And now we may use ServerRpcController. 544 if (controller != null && controller instanceof HBaseRpcController) { 545 hrc = (HBaseRpcController) controller; 546 if (!hrc.hasCallTimeout()) { 547 hrc.setCallTimeout(rpcTimeout); 548 } 549 } else { 550 hrc = new HBaseRpcControllerImpl(); 551 hrc.setCallTimeout(rpcTimeout); 552 } 553 return hrc; 554 } 555 } 556 557 /** 558 * Blocking rpc channel that goes via hbase rpc. 559 */ 560 @VisibleForTesting 561 public static class BlockingRpcChannelImplementation extends AbstractRpcChannel 562 implements BlockingRpcChannel { 563 564 protected BlockingRpcChannelImplementation(AbstractRpcClient<?> rpcClient, 565 InetSocketAddress addr, User ticket, int rpcTimeout) { 566 super(rpcClient, addr, ticket, rpcTimeout); 567 } 568 569 @Override 570 public Message callBlockingMethod(Descriptors.MethodDescriptor md, RpcController controller, 571 Message param, Message returnType) throws ServiceException { 572 return rpcClient.callBlockingMethod(md, configureRpcController(controller), 573 param, returnType, ticket, addr); 574 } 575 } 576 577 /** 578 * Async rpc channel that goes via hbase rpc. 579 */ 580 public static class RpcChannelImplementation extends AbstractRpcChannel implements 581 RpcChannel { 582 583 protected RpcChannelImplementation(AbstractRpcClient<?> rpcClient, InetSocketAddress addr, 584 User ticket, int rpcTimeout) throws UnknownHostException { 585 super(rpcClient, addr, ticket, rpcTimeout); 586 } 587 588 @Override 589 public void callMethod(Descriptors.MethodDescriptor md, RpcController controller, 590 Message param, Message returnType, RpcCallback<Message> done) { 591 // This method does not throw any exceptions, so the caller must provide a 592 // HBaseRpcController which is used to pass the exceptions. 593 this.rpcClient.callMethod(md, 594 configureRpcController(Preconditions.checkNotNull(controller, 595 "RpcController can not be null for async rpc call")), 596 param, returnType, ticket, addr, done); 597 } 598 } 599}