You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by st...@apache.org on 2021/04/07 01:18:12 UTC

[hbase] branch branch-2.4 updated: HBASE-25735 Add target Region to connection exceptions Signed-off-by: Wellington Chevreuil

This is an automated email from the ASF dual-hosted git repository.

stack pushed a commit to branch branch-2.4
in repository https://gitbox.apache.org/repos/asf/hbase.git


The following commit(s) were added to refs/heads/branch-2.4 by this push:
     new f120067  HBASE-25735 Add target Region to connection exceptions Signed-off-by: Wellington Chevreuil <wc...@apache.org>
f120067 is described below

commit f120067f174e632a8a01704a7915f96415420d27
Author: stack <st...@apache.org>
AuthorDate: Mon Apr 5 20:10:44 2021 -0700

    HBASE-25735 Add target Region to connection exceptions
    Signed-off-by: Wellington Chevreuil <wc...@apache.org>
---
 .../hadoop/hbase/client/RpcRetryingCallerImpl.java |  4 +-
 .../apache/hadoop/hbase/ipc/AbstractRpcClient.java |  2 +-
 .../java/org/apache/hadoop/hbase/ipc/Call.java     |  1 +
 .../hadoop/hbase/ipc/HBaseRpcController.java       | 21 +++++++-
 .../hadoop/hbase/ipc/HBaseRpcControllerImpl.java   | 46 +++++++++++++-----
 .../java/org/apache/hadoop/hbase/ipc/IPCUtil.java  | 56 +++++++++++++---------
 .../org/apache/hadoop/hbase/ipc/RpcConnection.java |  6 +--
 .../hadoop/hbase/ipc/RpcControllerFactory.java     | 12 +++--
 .../hbase/ipc/TestHBaseRpcControllerImpl.java      |  2 +-
 .../org/apache/hadoop/hbase/ipc/TestIPCUtil.java   | 17 +++++--
 .../hbase/client/TestRpcControllerFactory.java     |  9 ++--
 11 files changed, 119 insertions(+), 57 deletions(-)

diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RpcRetryingCallerImpl.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RpcRetryingCallerImpl.java
index 7a850ee..fb58bfa 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RpcRetryingCallerImpl.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RpcRetryingCallerImpl.java
@@ -25,6 +25,7 @@ import java.io.IOException;
 import java.io.InterruptedIOException;
 import java.lang.reflect.UndeclaredThrowableException;
 import java.net.SocketTimeoutException;
+import java.time.Instant;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.concurrent.atomic.AtomicBoolean;
@@ -246,7 +247,8 @@ public class RpcRetryingCallerImpl<T> implements RpcRetryingCaller<T> {
 
   @Override
   public String toString() {
-    return "RpcRetryingCaller{" + "globalStartTime=" + tracker.getStartTime() +
+    return "RpcRetryingCaller{" + "globalStartTime=" +
+      Instant.ofEpochMilli(tracker.getStartTime()) +
         ", pause=" + pause + ", maxAttempts=" + maxAttempts + '}';
   }
 }
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AbstractRpcClient.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AbstractRpcClient.java
index 1879f26..a57672f 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AbstractRpcClient.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AbstractRpcClient.java
@@ -380,7 +380,7 @@ public abstract class AbstractRpcClient<T extends RpcConnection> implements RpcC
         call.error.fillInStackTrace();
         hrc.setFailed(call.error);
       } else {
-        hrc.setFailed(wrapException(addr, call.error));
+        hrc.setFailed(wrapException(addr, hrc.getRegionInfo(), call.error));
       }
       callback.run(null);
     } else {
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/Call.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/Call.java
index ccd92cc..7793680 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/Call.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/Call.java
@@ -88,6 +88,7 @@ class Call {
 
   @Override
   public String toString() {
+    // Call[id=32153218,methodName=Get]
     return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE)
       .appendSuper(toShortString())
       .append("param", Optional.ofNullable(param)
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/HBaseRpcController.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/HBaseRpcController.java
index 55e85f0..843d9cd 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/HBaseRpcController.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/HBaseRpcController.java
@@ -1,4 +1,4 @@
-/**
+/*
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
  * distributed with this work for additional information
@@ -17,6 +17,7 @@
  */
 package org.apache.hadoop.hbase.ipc;
 
+import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hbase.thirdparty.com.google.protobuf.RpcCallback;
 import org.apache.hbase.thirdparty.com.google.protobuf.RpcController;
 
@@ -32,7 +33,9 @@ import org.apache.yetus.audience.InterfaceAudience;
  * Optionally carries Cells across the proxy/service interface down into ipc. On its way out it
  * optionally carries a set of result Cell data. We stick the Cells here when we want to avoid
  * having to protobuf them (for performance reasons). This class is used ferrying data across the
- * proxy/protobuf service chasm. Also does call timeout. Used by client and server ipc'ing.
+ * proxy/protobuf service chasm. Also does call timeout and on client-side, carries the target
+ * RegionInfo we're making the call against if relevant (useful adding info to exceptions and logs).
+ * Used by client and server ipc'ing.
  */
 @InterfaceAudience.Private
 public interface HBaseRpcController extends RpcController, CellScannable {
@@ -103,4 +106,18 @@ public interface HBaseRpcController extends RpcController, CellScannable {
    * cancellation state does not change during this call.
    */
   void notifyOnCancel(RpcCallback<Object> callback, CancellationCallback action) throws IOException;
+
+  /**
+   * @return True if this Controller is carrying the RPC target Region's RegionInfo.
+   */
+  default boolean hasRegionInfo() {
+    return false;
+  }
+
+  /**
+   * @return Target Region's RegionInfo or null if not available or pertinent.
+   */
+  default RegionInfo getRegionInfo() {
+    return null;
+  }
 }
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/HBaseRpcControllerImpl.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/HBaseRpcControllerImpl.java
index 237cef3..7dde67c 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/HBaseRpcControllerImpl.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/HBaseRpcControllerImpl.java
@@ -1,4 +1,4 @@
-/**
+/*
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
  * distributed with this work for additional information
@@ -17,6 +17,7 @@
  */
 package org.apache.hadoop.hbase.ipc;
 
+import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hbase.thirdparty.com.google.protobuf.RpcCallback;
 
 import java.io.IOException;
@@ -31,10 +32,8 @@ import org.apache.hadoop.hbase.TableName;
 import org.apache.yetus.audience.InterfaceAudience;
 
 /**
- * Optionally carries Cells across the proxy/service interface down into ipc. On its way out it
- * optionally carries a set of result Cell data. We stick the Cells here when we want to avoid
- * having to protobuf them (for performance reasons). This class is used ferrying data across the
- * proxy/protobuf service chasm. Also does call timeout. Used by client and server ipc'ing.
+ * Get instances via {@link RpcControllerFactory} on client-side.
+ * @see RpcControllerFactory
  */
 @InterfaceAudience.Private
 public class HBaseRpcControllerImpl implements HBaseRpcController {
@@ -52,6 +51,12 @@ public class HBaseRpcControllerImpl implements HBaseRpcController {
   private IOException exception;
 
   /**
+   * Rpc target Region's RegionInfo we are going against. May be null.
+   * @see #hasRegionInfo()
+   */
+  private RegionInfo regionInfo;
+
+  /**
    * Priority to set on this request. Set it here in controller so available composing the request.
    * This is the ordained way of setting priorities going forward. We will be undoing the old
    * annotation-based mechanism.
@@ -67,15 +72,34 @@ public class HBaseRpcControllerImpl implements HBaseRpcController {
   private CellScanner cellScanner;
 
   public HBaseRpcControllerImpl() {
-    this((CellScanner) null);
+    this(null, (CellScanner) null);
   }
 
+  /**
+   * Used server-side. Clients should go via {@link RpcControllerFactory}
+   */
   public HBaseRpcControllerImpl(final CellScanner cellScanner) {
+    this(null, cellScanner);
+  }
+
+  HBaseRpcControllerImpl(RegionInfo regionInfo, final CellScanner cellScanner) {
     this.cellScanner = cellScanner;
+    this.regionInfo = regionInfo;
   }
 
-  public HBaseRpcControllerImpl(final List<CellScannable> cellIterables) {
+  HBaseRpcControllerImpl(RegionInfo regionInfo, final List<CellScannable> cellIterables) {
     this.cellScanner = cellIterables == null ? null : CellUtil.createCellScanner(cellIterables);
+    this.regionInfo = null;
+  }
+
+  @Override
+  public boolean hasRegionInfo() {
+    return this.regionInfo != null;
+  }
+
+  @Override
+  public RegionInfo getRegionInfo() {
+    return this.regionInfo;
   }
 
   /**
@@ -118,6 +142,7 @@ public class HBaseRpcControllerImpl implements HBaseRpcController {
     cellScanner = null;
     exception = null;
     callTimeout = null;
+    regionInfo = null;
     // In the implementations of some callable with replicas, rpc calls are executed in a executor
     // and we could cancel the operation from outside which means there could be a race between
     // reset and startCancel. Although I think the race should be handled by the callable since the
@@ -131,11 +156,7 @@ public class HBaseRpcControllerImpl implements HBaseRpcController {
 
   @Override
   public int getCallTimeout() {
-    if (callTimeout != null) {
-      return callTimeout.intValue();
-    } else {
-      return 0;
-    }
+    return callTimeout != null? callTimeout: 0;
   }
 
   @Override
@@ -241,5 +262,4 @@ public class HBaseRpcControllerImpl implements HBaseRpcController {
       action.run(false);
     }
   }
-
 }
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/IPCUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/IPCUtil.java
index 1b1411c..d595b9b 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/IPCUtil.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/IPCUtil.java
@@ -29,6 +29,7 @@ import org.apache.commons.lang3.mutable.MutableInt;
 import org.apache.hadoop.hbase.DoNotRetryIOException;
 import org.apache.hadoop.hbase.HBaseIOException;
 import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.exceptions.ClientExceptionsUtil;
 import org.apache.hadoop.hbase.exceptions.ConnectionClosedException;
 import org.apache.hadoop.hbase.exceptions.ConnectionClosingException;
@@ -159,15 +160,21 @@ class IPCUtil {
     }
   }
 
+  private static String getCallTarget(InetSocketAddress addr, RegionInfo regionInfo) {
+    return "address=" + addr +
+      (regionInfo != null? ", region=" + regionInfo.getRegionNameAsString(): null);
+  }
+
   /**
-   * Takes an Exception and the address we were trying to connect to and return an IOException with
-   * the input exception as the cause. The new exception provides the stack trace of the place where
-   * the exception is thrown and some extra diagnostics information.
+   * Takes an Exception, the address, and if pertinent, the RegionInfo for the Region we were trying
+   * to connect to and returns an IOException with the input exception as the cause. The new
+   * exception provides the stack trace of the place where the exception is thrown and some extra
+   * diagnostics information.
    * <p/>
    * Notice that we will try our best to keep the original exception type when creating a new
    * exception, especially for the 'connection' exceptions, as it is used to determine whether this
-   * is a network issue or the remote side tells us clearly what is wrong, which is very important
-   * to decide whether to retry. If it is not possible to create a new exception with the same type,
+   * is a network issue or the remote side tells us clearly what is wrong, which is important
+   * deciding whether to retry. If it is not possible to create a new exception with the same type,
    * for example, the {@code error} is not an {@link IOException}, an {@link IOException} will be
    * created.
    * @param addr target address
@@ -175,17 +182,18 @@ class IPCUtil {
    * @return an exception to throw
    * @see ClientExceptionsUtil#isConnectionException(Throwable)
    */
-  static IOException wrapException(InetSocketAddress addr, Throwable error) {
+  static IOException wrapException(InetSocketAddress addr, RegionInfo regionInfo,
+      Throwable error) {
     if (error instanceof ConnectException) {
       // connection refused; include the host:port in the error
-      return (IOException) new ConnectException(
-        "Call to " + addr + " failed on connection exception: " + error).initCause(error);
+      return (IOException) new ConnectException("Call to " + getCallTarget(addr, regionInfo) +
+        " failed on connection exception: " + error).initCause(error);
     } else if (error instanceof SocketTimeoutException) {
       return (IOException) new SocketTimeoutException(
-        "Call to " + addr + " failed because " + error).initCause(error);
+        "Call to " + getCallTarget(addr, regionInfo) + " failed because " + error).initCause(error);
     } else if (error instanceof ConnectionClosingException) {
-      return new ConnectionClosingException("Call to " + addr + " failed on local exception: "
-        + error, error);
+      return new ConnectionClosingException("Call to " + getCallTarget(addr, regionInfo) +
+        " failed on local exception: " + error, error);
     } else if (error instanceof ServerTooBusyException) {
       // we already have address in the exception message
       return (IOException) error;
@@ -194,42 +202,44 @@ class IPCUtil {
       try {
         return (IOException) error.getClass().asSubclass(DoNotRetryIOException.class)
           .getConstructor(String.class)
-          .newInstance("Call to " + addr + " failed on local exception: " + error).initCause(error);
+          .newInstance("Call to " + getCallTarget(addr, regionInfo) +
+            " failed on local exception: " + error).initCause(error);
       } catch (InstantiationException | IllegalAccessException | IllegalArgumentException
           | InvocationTargetException | NoSuchMethodException | SecurityException e) {
         // just ignore, will just new a DoNotRetryIOException instead below
       }
-      return new DoNotRetryIOException("Call to " + addr + " failed on local exception: "
-        + error, error);
+      return new DoNotRetryIOException("Call to " + getCallTarget(addr, regionInfo) +
+        " failed on local exception: " + error, error);
     } else if (error instanceof ConnectionClosedException) {
-      return new ConnectionClosedException("Call to " + addr + " failed on local exception: "
-        + error, error);
+      return new ConnectionClosedException("Call to " + getCallTarget(addr, regionInfo) +
+        " failed on local exception: " + error, error);
     } else if (error instanceof CallTimeoutException) {
-      return new CallTimeoutException("Call to " + addr + " failed on local exception: "
-        + error, error);
+      return new CallTimeoutException("Call to " + getCallTarget(addr, regionInfo) +
+        " failed on local exception: " + error, error);
     } else if (error instanceof ClosedChannelException) {
       // ClosedChannelException does not have a constructor which takes a String but it is a
       // connection exception so we keep its original type
       return (IOException) error;
     } else if (error instanceof TimeoutException) {
       // TimeoutException is not an IOException, let's convert it to TimeoutIOException.
-      return new TimeoutIOException("Call to " + addr + " failed on local exception: "
-        + error, error);
+      return new TimeoutIOException("Call to " + getCallTarget(addr, regionInfo) +
+        " failed on local exception: " + error, error);
     } else {
       // try our best to keep the original exception type
       if (error instanceof IOException) {
         try {
           return (IOException) error.getClass().asSubclass(IOException.class)
             .getConstructor(String.class)
-            .newInstance("Call to " + addr + " failed on local exception: " + error)
+            .newInstance("Call to " + getCallTarget(addr, regionInfo) +
+              " failed on local exception: " + error)
             .initCause(error);
         } catch (InstantiationException | IllegalAccessException | IllegalArgumentException
             | InvocationTargetException | NoSuchMethodException | SecurityException e) {
           // just ignore, will just new an IOException instead below
         }
       }
-      return new HBaseIOException("Call to " + addr + " failed on local exception: "
-        + error, error);
+      return new HBaseIOException("Call to " + getCallTarget(addr, regionInfo) +
+        " failed on local exception: " + error, error);
     }
   }
 
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcConnection.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcConnection.java
index 195a16d..e3004e6 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcConnection.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcConnection.java
@@ -1,4 +1,4 @@
-/**
+/*
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
  * distributed with this work for additional information
@@ -134,8 +134,8 @@ abstract class RpcConnection {
         @Override
         public void run(Timeout timeout) throws Exception {
           call.setTimeout(new CallTimeoutException(call.toShortString() + ", waitTime="
-              + (EnvironmentEdgeManager.currentTime() - call.getStartTime()) + ", rpcTimeout="
-              + call.timeout));
+              + (EnvironmentEdgeManager.currentTime() - call.getStartTime()) + "ms, rpcTimeout="
+              + call.timeout + "ms"));
           callTimeout(call);
         }
       }, call.timeout, TimeUnit.MILLISECONDS);
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcControllerFactory.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcControllerFactory.java
index e944ec2..0dcb22f 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcControllerFactory.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcControllerFactory.java
@@ -1,4 +1,4 @@
-/**
+/*
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
  * distributed with this work for additional information
@@ -22,6 +22,7 @@ import java.util.List;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.CellScannable;
 import org.apache.hadoop.hbase.CellScanner;
+import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -51,12 +52,13 @@ public class RpcControllerFactory {
     return new HBaseRpcControllerImpl();
   }
 
-  public HBaseRpcController newController(final CellScanner cellScanner) {
-    return new HBaseRpcControllerImpl(cellScanner);
+  public HBaseRpcController newController(RegionInfo regionInfo, CellScanner cellScanner) {
+    return new HBaseRpcControllerImpl(regionInfo, cellScanner);
   }
 
-  public HBaseRpcController newController(final List<CellScannable> cellIterables) {
-    return new HBaseRpcControllerImpl(cellIterables);
+  public HBaseRpcController newController(RegionInfo regionInfo,
+      final List<CellScannable> cellIterables) {
+    return new HBaseRpcControllerImpl(regionInfo, cellIterables);
   }
 
 
diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/ipc/TestHBaseRpcControllerImpl.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/ipc/TestHBaseRpcControllerImpl.java
index 04fbe1b..d829b4b 100644
--- a/hbase-client/src/test/java/org/apache/hadoop/hbase/ipc/TestHBaseRpcControllerImpl.java
+++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/ipc/TestHBaseRpcControllerImpl.java
@@ -48,7 +48,7 @@ public class TestHBaseRpcControllerImpl {
     for (int i = 0; i < count; i++) {
       cells.add(createCell(i));
     }
-    HBaseRpcController controller = new HBaseRpcControllerImpl(cells);
+    HBaseRpcController controller = new HBaseRpcControllerImpl(null, cells);
     CellScanner cellScanner = controller.cellScanner();
     int index = 0;
     for (; cellScanner.advance(); index++) {
diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/ipc/TestIPCUtil.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/ipc/TestIPCUtil.java
index 9e1ab2e..7e2c59f 100644
--- a/hbase-client/src/test/java/org/apache/hadoop/hbase/ipc/TestIPCUtil.java
+++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/ipc/TestIPCUtil.java
@@ -19,6 +19,7 @@ package org.apache.hadoop.hbase.ipc;
 
 import static org.hamcrest.CoreMatchers.instanceOf;
 import static org.hamcrest.MatcherAssert.assertThat;
+import static org.junit.Assert.assertTrue;
 
 import java.io.IOException;
 import java.lang.reflect.Constructor;
@@ -30,6 +31,7 @@ import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.TimeoutException;
 import org.apache.commons.lang3.mutable.MutableInt;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.client.RegionInfoBuilder;
 import org.apache.hadoop.hbase.exceptions.ClientExceptionsUtil;
 import org.apache.hadoop.hbase.exceptions.TimeoutIOException;
 import org.apache.hadoop.hbase.testclassification.ClientTests;
@@ -91,8 +93,7 @@ public class TestIPCUtil {
   }
 
   /**
-   * See HBASE-21862, it is very important to keep the original exception type for connection
-   * exceptions.
+   * See HBASE-21862, it is important to keep original exception type for connection exceptions.
    */
   @Test
   public void testWrapConnectionException() throws Exception {
@@ -103,9 +104,17 @@ public class TestIPCUtil {
     InetSocketAddress addr = InetSocketAddress.createUnresolved("127.0.0.1", 12345);
     for (Throwable exception : exceptions) {
       if (exception instanceof TimeoutException) {
-        assertThat(IPCUtil.wrapException(addr, exception), instanceOf(TimeoutIOException.class));
+        assertThat(IPCUtil.wrapException(addr, null, exception), instanceOf(TimeoutIOException.class));
       } else {
-        assertThat(IPCUtil.wrapException(addr, exception), instanceOf(exception.getClass()));
+        IOException ioe = IPCUtil.wrapException(addr, RegionInfoBuilder.FIRST_META_REGIONINFO,
+          exception);
+        // Assert that the exception contains the Region name if supplied. HBASE-25735.
+        // Not all exceptions get the region stuffed into it.
+        if (ioe.getMessage() != null) {
+          assertTrue(ioe.getMessage().
+            contains(RegionInfoBuilder.FIRST_META_REGIONINFO.getRegionNameAsString()));
+        }
+        assertThat(ioe, instanceOf(exception.getClass()));
       }
     }
   }
diff --git a/hbase-endpoint/src/test/java/org/apache/hadoop/hbase/client/TestRpcControllerFactory.java b/hbase-endpoint/src/test/java/org/apache/hadoop/hbase/client/TestRpcControllerFactory.java
index 2d60733..30fce3c 100644
--- a/hbase-endpoint/src/test/java/org/apache/hadoop/hbase/client/TestRpcControllerFactory.java
+++ b/hbase-endpoint/src/test/java/org/apache/hadoop/hbase/client/TestRpcControllerFactory.java
@@ -71,13 +71,14 @@ public class TestRpcControllerFactory {
     }
 
     @Override
-    public HBaseRpcController newController(final CellScanner cellScanner) {
-      return new CountingRpcController(super.newController(cellScanner));
+    public HBaseRpcController newController(RegionInfo regionInfo, CellScanner cellScanner) {
+      return new CountingRpcController(super.newController(regionInfo, cellScanner));
     }
 
     @Override
-    public HBaseRpcController newController(final List<CellScannable> cellIterables) {
-      return new CountingRpcController(super.newController(cellIterables));
+    public HBaseRpcController newController(RegionInfo regionInfo,
+        List<CellScannable> cellIterables) {
+      return new CountingRpcController(super.newController(regionInfo, cellIterables));
     }
   }