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}