You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by la...@apache.org on 2016/11/01 19:51:29 UTC

[35/50] hbase git commit: HBASE-15645 hbase.rpc.timeout is not used in operations of HTable

HBASE-15645 hbase.rpc.timeout is not used in operations of HTable

Signed-off-by: stack <st...@apache.org>


Project: http://git-wip-us.apache.org/repos/asf/hbase/repo
Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/48f158f0
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/48f158f0
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/48f158f0

Branch: refs/heads/branch-1.0
Commit: 48f158f0a834c28ec06a88aa39389da685ec7a0e
Parents: 0b20b27
Author: Phil Yang <ud...@gmail.com>
Authored: Tue Apr 26 18:02:22 2016 +0800
Committer: stack <st...@apache.org>
Committed: Tue Apr 26 10:55:36 2016 -0700

----------------------------------------------------------------------
 .../apache/hadoop/hbase/client/HBaseAdmin.java  |  5 ++-
 .../org/apache/hadoop/hbase/client/HTable.java  | 43 +++++++++++++++-----
 .../apache/hadoop/hbase/client/HTablePool.java  | 16 ++++++++
 .../hadoop/hbase/client/RpcRetryingCaller.java  | 16 ++++++--
 .../hbase/client/RpcRetryingCallerFactory.java  | 25 +++++++++++-
 .../client/StatsTrackingRpcRetryingCaller.java  |  2 +-
 .../org/apache/hadoop/hbase/client/Table.java   | 31 ++++++++++++++
 .../client/TestFastFailWithoutTestUtil.java     |  2 +-
 .../org/apache/hadoop/hbase/HConstants.java     |  4 +-
 .../src/main/resources/hbase-default.xml        | 11 ++++-
 .../hadoop/hbase/rest/client/RemoteHTable.java  | 16 ++++++++
 .../hadoop/hbase/client/HTableWrapper.java      | 16 ++++++++
 .../org/apache/hadoop/hbase/client/TestHCM.java | 26 +++++++++++-
 13 files changed, 191 insertions(+), 22 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/48f158f0/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
index 8cd3037..73d8c89 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
@@ -183,6 +183,7 @@ public class HBaseAdmin implements Admin {
   private boolean cleanupConnectionOnClose = false; // close the connection in close()
   private boolean closed = false;
   private int operationTimeout;
+  private int rpcTimeout;
 
   private RpcRetryingCallerFactory rpcCallerFactory;
 
@@ -237,6 +238,8 @@ public class HBaseAdmin implements Admin {
         "hbase.client.retries.longer.multiplier", 10);
     this.operationTimeout = this.conf.getInt(HConstants.HBASE_CLIENT_OPERATION_TIMEOUT,
         HConstants.DEFAULT_HBASE_CLIENT_OPERATION_TIMEOUT);
+    this.rpcTimeout = this.conf.getInt(HConstants.HBASE_RPC_TIMEOUT_KEY,
+        HConstants.DEFAULT_HBASE_RPC_TIMEOUT);
 
     this.rpcCallerFactory = RpcRetryingCallerFactory.instantiate(this.conf);
   }
@@ -3596,7 +3599,7 @@ public class HBaseAdmin implements Admin {
   }
 
   private <V> V executeCallable(MasterCallable<V> callable) throws IOException {
-    RpcRetryingCaller<V> caller = rpcCallerFactory.newCaller();
+    RpcRetryingCaller<V> caller = rpcCallerFactory.newCaller(rpcTimeout);
     try {
       return caller.callWithRetries(callable, operationTimeout);
     } finally {

http://git-wip-us.apache.org/repos/asf/hbase/blob/48f158f0/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java
index 533cbab..2bc0791 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java
@@ -129,7 +129,8 @@ public class HTable implements HTableInterface, RegionLocator {
   protected int scannerCaching;
   protected long scannerMaxResultSize;
   private ExecutorService pool;  // For Multi & Scan
-  private int operationTimeout;
+  private int operationTimeout; // global timeout for each blocking method with retrying rpc
+  private int rpcTimeout; // timeout for each rpc request
   private final boolean cleanupPoolOnClose; // shutdown the pool in close()
   private final boolean cleanupConnectionOnClose; // close the connection in close()
   private Consistency defaultConsistency = Consistency.STRONG;
@@ -360,6 +361,8 @@ public class HTable implements HTableInterface, RegionLocator {
 
     this.operationTimeout = tableName.isSystemTable() ?
         tableConfiguration.getMetaOperationTimeout() : tableConfiguration.getOperationTimeout();
+    this.rpcTimeout = configuration.getInt(HConstants.HBASE_RPC_TIMEOUT_KEY,
+        HConstants.DEFAULT_HBASE_RPC_TIMEOUT);
     this.scannerCaching = tableConfiguration.getScannerCaching();
     this.scannerMaxResultSize = tableConfiguration.getScannerMaxResultSize();
     if (this.rpcCallerFactory == null) {
@@ -808,7 +811,8 @@ public class HTable implements HTableInterface, RegionLocator {
          }
        }
      };
-     return rpcCallerFactory.<Result>newCaller().callWithRetries(callable, this.operationTimeout);
+     return rpcCallerFactory.<Result>newCaller(rpcTimeout).callWithRetries(callable,
+         this.operationTimeout);
    }
 
   /**
@@ -913,7 +917,8 @@ public class HTable implements HTableInterface, RegionLocator {
           }
         }
       };
-      return rpcCallerFactory.<Result>newCaller().callWithRetries(callable, this.operationTimeout);
+      return rpcCallerFactory.<Result>newCaller(rpcTimeout).callWithRetries(callable,
+          this.operationTimeout);
     }
 
     // Call that takes into account the replica
@@ -1029,7 +1034,8 @@ public class HTable implements HTableInterface, RegionLocator {
         }
       }
     };
-    rpcCallerFactory.<Boolean> newCaller().callWithRetries(callable, this.operationTimeout);
+    rpcCallerFactory.<Boolean> newCaller(rpcTimeout).callWithRetries(callable,
+        this.operationTimeout);
   }
 
   /**
@@ -1146,7 +1152,8 @@ public class HTable implements HTableInterface, RegionLocator {
           }
         }
       };
-    return rpcCallerFactory.<Result> newCaller().callWithRetries(callable, this.operationTimeout);
+    return rpcCallerFactory.<Result> newCaller(rpcTimeout).callWithRetries(callable,
+        this.operationTimeout);
   }
 
   /**
@@ -1177,7 +1184,8 @@ public class HTable implements HTableInterface, RegionLocator {
         }
       }
     };
-    return rpcCallerFactory.<Result> newCaller().callWithRetries(callable, this.operationTimeout);
+    return rpcCallerFactory.<Result> newCaller(rpcTimeout).callWithRetries(callable,
+        this.operationTimeout);
   }
 
   /**
@@ -1244,7 +1252,8 @@ public class HTable implements HTableInterface, RegionLocator {
           }
         }
       };
-    return rpcCallerFactory.<Long> newCaller().callWithRetries(callable, this.operationTimeout);
+    return rpcCallerFactory.<Long> newCaller(rpcTimeout).callWithRetries(callable,
+        this.operationTimeout);
   }
 
   /**
@@ -1273,7 +1282,8 @@ public class HTable implements HTableInterface, RegionLocator {
           }
         }
       };
-    return rpcCallerFactory.<Boolean> newCaller().callWithRetries(callable, this.operationTimeout);
+    return rpcCallerFactory.<Boolean> newCaller(rpcTimeout).callWithRetries(callable,
+        this.operationTimeout);
   }
 
   /**
@@ -1303,7 +1313,8 @@ public class HTable implements HTableInterface, RegionLocator {
           }
         }
       };
-    return rpcCallerFactory.<Boolean> newCaller().callWithRetries(callable, this.operationTimeout);
+    return rpcCallerFactory.<Boolean> newCaller(rpcTimeout).callWithRetries(callable,
+        this.operationTimeout);
   }
 
   /**
@@ -1332,7 +1343,8 @@ public class HTable implements HTableInterface, RegionLocator {
           }
         }
       };
-    return rpcCallerFactory.<Boolean> newCaller().callWithRetries(callable, this.operationTimeout);
+    return rpcCallerFactory.<Boolean> newCaller(rpcTimeout).callWithRetries(callable,
+        this.operationTimeout);
   }
 
   /**
@@ -1362,7 +1374,8 @@ public class HTable implements HTableInterface, RegionLocator {
           }
         }
       };
-    return rpcCallerFactory.<Boolean> newCaller().callWithRetries(callable, this.operationTimeout);
+    return rpcCallerFactory.<Boolean> newCaller(rpcTimeout).callWithRetries(callable,
+        this.operationTimeout);
   }
 
   /**
@@ -1815,6 +1828,14 @@ public class HTable implements HTableInterface, RegionLocator {
     return operationTimeout;
   }
 
+  @Override public void setRpcTimeout(int rpcTimeout) {
+    this.rpcTimeout = rpcTimeout;
+  }
+
+  @Override public int getRpcTimeout() {
+    return rpcTimeout;
+  }
+
   @Override
   public String toString() {
     return tableName + ";" + connection;

http://git-wip-us.apache.org/repos/asf/hbase/blob/48f158f0/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTablePool.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTablePool.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTablePool.java
index 4b998a6..d837bf8 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTablePool.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTablePool.java
@@ -672,5 +672,21 @@ public class HTablePool implements Closeable {
       checkState();
       return table.checkAndMutate(row, family, qualifier, compareOp, value, mutation);
     }
+
+    @Override public void setOperationTimeout(int operationTimeout) {
+      table.setOperationTimeout(operationTimeout);
+    }
+
+    @Override public int getOperationTimeout() {
+      return table.getOperationTimeout();
+    }
+
+    @Override public void setRpcTimeout(int rpcTimeout) {
+      table.setRpcTimeout(rpcTimeout);
+    }
+
+    @Override public int getRpcTimeout() {
+      return table.getRpcTimeout();
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/48f158f0/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RpcRetryingCaller.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RpcRetryingCaller.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RpcRetryingCaller.java
index 49c7efd..77ed031 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RpcRetryingCaller.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RpcRetryingCaller.java
@@ -63,21 +63,23 @@ public class RpcRetryingCaller<T> {
 
   private final long pause;
   private final int retries;
+  private final int rpcTimeout;// timeout for each rpc request
   private final AtomicBoolean cancelled = new AtomicBoolean(false);
   private final RetryingCallerInterceptor interceptor;
   private final RetryingCallerInterceptorContext context;
 
   public RpcRetryingCaller(long pause, int retries, int startLogErrorsCnt) {
-    this(pause, retries, RetryingCallerInterceptorFactory.NO_OP_INTERCEPTOR, startLogErrorsCnt);
+    this(pause, retries, RetryingCallerInterceptorFactory.NO_OP_INTERCEPTOR, startLogErrorsCnt, 0);
   }
 
   public RpcRetryingCaller(long pause, int retries,
-      RetryingCallerInterceptor interceptor, int startLogErrorsCnt) {
+      RetryingCallerInterceptor interceptor, int startLogErrorsCnt, int rpcTimeout) {
     this.pause = pause;
     this.retries = retries;
     this.interceptor = interceptor;
     context = interceptor.createEmptyContext();
     this.startLogErrorsCnt = startLogErrorsCnt;
+    this.rpcTimeout = rpcTimeout;
   }
 
   private int getRemainingTime(int callTimeout) {
@@ -97,6 +99,14 @@ public class RpcRetryingCaller<T> {
     }
   }
 
+  private int getTimeout(int callTimeout){
+    int timeout = getRemainingTime(callTimeout);
+    if (timeout <= 0 || rpcTimeout > 0 && rpcTimeout < timeout){
+      timeout = rpcTimeout;
+    }
+    return timeout;
+  }
+
   public void cancel(){
     cancelled.set(true);
     synchronized (cancelled){
@@ -123,7 +133,7 @@ public class RpcRetryingCaller<T> {
       try {
         callable.prepare(tries != 0); // if called with false, check table status on ZK
         interceptor.intercept(context.prepare(callable, tries));
-        return callable.call(getRemainingTime(callTimeout));
+        return callable.call(getTimeout(callTimeout));
       } catch (PreemptiveFastFailException e) {
         throw e;
       } catch (Throwable t) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/48f158f0/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RpcRetryingCallerFactory.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RpcRetryingCallerFactory.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RpcRetryingCallerFactory.java
index 1bf7bb0..09b70b8 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RpcRetryingCallerFactory.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RpcRetryingCallerFactory.java
@@ -33,6 +33,7 @@ public class RpcRetryingCallerFactory {
   protected final Configuration conf;
   private final long pause;
   private final int retries;
+  private final int rpcTimeout;
   private final RetryingCallerInterceptor interceptor;
   private final int startLogErrorsCnt;
   private final boolean enableBackPressure;
@@ -53,6 +54,7 @@ public class RpcRetryingCallerFactory {
     this.interceptor = interceptor;
     enableBackPressure = conf.getBoolean(HConstants.ENABLE_CLIENT_BACKPRESSURE,
         HConstants.DEFAULT_ENABLE_CLIENT_BACKPRESSURE);
+    rpcTimeout = conf.getInt(HConstants.HBASE_RPC_TIMEOUT_KEY,HConstants.DEFAULT_HBASE_RPC_TIMEOUT);
   }
 
   /**
@@ -62,11 +64,32 @@ public class RpcRetryingCallerFactory {
     this.stats = statisticTracker;
   }
 
+  /**
+   * Create a new RetryingCaller with specific rpc timeout.
+   */
+  public <T> RpcRetryingCaller<T> newCaller(int rpcTimeout) {
+    // We store the values in the factory instance. This way, constructing new objects
+    //  is cheap as it does not require parsing a complex structure.
+    RpcRetryingCaller<T> caller = new RpcRetryingCaller<T>(pause, retries, interceptor,
+        startLogErrorsCnt, rpcTimeout);
+
+    // wrap it with stats, if we are tracking them
+    if (enableBackPressure && this.stats != null) {
+      caller = new StatsTrackingRpcRetryingCaller<T>(pause, retries, interceptor,
+          startLogErrorsCnt, stats);
+    }
+
+    return caller;
+  }
+
+  /**
+   * Create a new RetryingCaller with configured rpc timeout.
+   */
   public <T> RpcRetryingCaller<T> newCaller() {
     // We store the values in the factory instance. This way, constructing new objects
     //  is cheap as it does not require parsing a complex structure.
     RpcRetryingCaller<T> caller = new RpcRetryingCaller<T>(pause, retries, interceptor,
-        startLogErrorsCnt);
+        startLogErrorsCnt, rpcTimeout);
 
     // wrap it with stats, if we are tracking them
     if (enableBackPressure && this.stats != null) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/48f158f0/hbase-client/src/main/java/org/apache/hadoop/hbase/client/StatsTrackingRpcRetryingCaller.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/StatsTrackingRpcRetryingCaller.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/StatsTrackingRpcRetryingCaller.java
index fc175bb..cbd625d 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/StatsTrackingRpcRetryingCaller.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/StatsTrackingRpcRetryingCaller.java
@@ -39,7 +39,7 @@ public class StatsTrackingRpcRetryingCaller<T> extends RpcRetryingCaller<T> {
   public StatsTrackingRpcRetryingCaller(long pause, int retries,
       RetryingCallerInterceptor interceptor, int startLogErrorsCnt,
       ServerStatisticTracker stats) {
-    super(pause, retries, interceptor, startLogErrorsCnt);
+    super(pause, retries, interceptor, startLogErrorsCnt, 0);
     this.stats = stats;
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/48f158f0/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Table.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Table.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Table.java
index 9a6744b..8c6169d 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Table.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Table.java
@@ -595,4 +595,35 @@ public interface Table extends Closeable {
    */
   boolean checkAndMutate(byte[] row, byte[] family, byte[] qualifier,
       CompareFilter.CompareOp compareOp, byte[] value, RowMutations mutation) throws IOException;
+
+  /**
+   * Set timeout (millisecond) of each operation in this Table instance, will override the value
+   * of hbase.client.operation.timeout in configuration.
+   * Operation timeout is a top-level restriction that makes sure a blocking method will not be
+   * blocked more than this. In each operation, if rpc request fails because of timeout or
+   * other reason, it will retry until success or throw a RetriesExhaustedException. But if the
+   * total time being blocking reach the operation timeout before retries exhausted, it will break
+   * early and throw SocketTimeoutException.
+   * @param operationTimeout the total timeout of each operation in millisecond.
+   */
+  public void setOperationTimeout(int operationTimeout);
+
+  /**
+   * Get timeout (millisecond) of each operation for in Table instance.
+   */
+  public int getOperationTimeout();
+
+  /**
+   * Set timeout (millisecond) of each rpc request in operations of this Table instance, will
+   * override the value of hbase.rpc.timeout in configuration.
+   * If a rpc request waiting too long, it will stop waiting and send a new request to retry until
+   * retries exhausted or operation timeout reached.
+   * @param rpcTimeout the timeout of each rpc request in millisecond.
+   */
+  public void setRpcTimeout(int rpcTimeout);
+
+  /**
+   * Get timeout (millisecond) of each rpc request in this Table instance.
+   */
+  public int getRpcTimeout();
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/48f158f0/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestFastFailWithoutTestUtil.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestFastFailWithoutTestUtil.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestFastFailWithoutTestUtil.java
index 7cb0be6..b387683 100644
--- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestFastFailWithoutTestUtil.java
+++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestFastFailWithoutTestUtil.java
@@ -563,7 +563,7 @@ public class TestFastFailWithoutTestUtil {
 
   public RpcRetryingCaller<Void> getRpcRetryingCaller(int pauseTime,
       int retries, RetryingCallerInterceptor interceptor) {
-    return new RpcRetryingCaller<Void>(pauseTime, retries, interceptor, 9) {
+    return new RpcRetryingCaller<Void>(pauseTime, retries, interceptor, 9, 0) {
       @Override
       public Void callWithRetries(RetryingCallable<Void> callable,
           int callTimeout) throws IOException, RuntimeException {

http://git-wip-us.apache.org/repos/asf/hbase/blob/48f158f0/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java
index f1f3e1a..4ad01c8 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java
@@ -272,10 +272,10 @@ public final class HConstants {
   /** Parameter name for HBase client IPC pool size */
   public static final String HBASE_CLIENT_IPC_POOL_SIZE = "hbase.client.ipc.pool.size";
 
-  /** Parameter name for HBase client operation timeout, which overrides RPC timeout */
+  /** Parameter name for HBase client operation timeout. */
   public static final String HBASE_CLIENT_OPERATION_TIMEOUT = "hbase.client.operation.timeout";
 
-  /** Parameter name for HBase client operation timeout, which overrides RPC timeout */
+  /** Parameter name for HBase client operation timeout. */
   public static final String HBASE_CLIENT_META_OPERATION_TIMEOUT =
     "hbase.client.meta.operation.timeout";
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/48f158f0/hbase-common/src/main/resources/hbase-default.xml
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/resources/hbase-default.xml b/hbase-common/src/main/resources/hbase-default.xml
index c2581bf..9cc2a9e 100644
--- a/hbase-common/src/main/resources/hbase-default.xml
+++ b/hbase-common/src/main/resources/hbase-default.xml
@@ -795,11 +795,20 @@ possible configurations would overwhelm and obscure the important.
   <property>
     <name>hbase.rpc.timeout</name>
     <value>60000</value>
-    <description>This is for the RPC layer to define how long HBase client applications
+    <description>This is for the RPC layer to define how long (millisecond) HBase client applications
         take for a remote call to time out. It uses pings to check connections
         but will eventually throw a TimeoutException.</description>
   </property>
   <property>
+    <name>hbase.client.operation.timeout</name>
+    <value>1200000</value>
+    <description>Operation timeout is a top-level restriction (millisecond) that makes sure a
+        blocking operation in Table will not be blocked more than this. In each operation, if rpc
+        request fails because of timeout or other reason, it will retry until success or throw
+        RetriesExhaustedException. But if the total time being blocking reach the operation timeout
+        before retries exhausted, it will break early and throw SocketTimeoutException.</description>
+  </property>
+  <property>
     <name>hbase.rpc.shortoperation.timeout</name>
     <value>10000</value>
     <description>This is another version of "hbase.rpc.timeout". For those RPC operation

http://git-wip-us.apache.org/repos/asf/hbase/blob/48f158f0/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/client/RemoteHTable.java
----------------------------------------------------------------------
diff --git a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/client/RemoteHTable.java b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/client/RemoteHTable.java
index 0300ea2..8429e12 100644
--- a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/client/RemoteHTable.java
+++ b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/client/RemoteHTable.java
@@ -856,4 +856,20 @@ public class RemoteHTable implements Table {
       CompareOp compareOp, byte[] value, RowMutations rm) throws IOException {
     throw new UnsupportedOperationException("checkAndMutate not implemented");
   }
+
+  @Override public void setOperationTimeout(int operationTimeout) {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override public int getOperationTimeout() {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override public void setRpcTimeout(int rpcTimeout) {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override public int getRpcTimeout() {
+    throw new UnsupportedOperationException();
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/48f158f0/hbase-server/src/main/java/org/apache/hadoop/hbase/client/HTableWrapper.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/client/HTableWrapper.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/client/HTableWrapper.java
index 1f84bb4..2d25f63 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/client/HTableWrapper.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/client/HTableWrapper.java
@@ -365,4 +365,20 @@ public class HTableWrapper implements HTableInterface {
       CompareOp compareOp, byte[] value, RowMutations rm) throws IOException {
     return table.checkAndMutate(row, family, qualifier, compareOp, value, rm);
   }
+
+  @Override public void setOperationTimeout(int operationTimeout) {
+    table.setOperationTimeout(operationTimeout);
+  }
+
+  @Override public int getOperationTimeout() {
+    return table.getOperationTimeout();
+  }
+
+  @Override public void setRpcTimeout(int rpcTimeout) {
+    table.setRpcTimeout(rpcTimeout);
+  }
+
+  @Override public int getRpcTimeout() {
+    return table.getRpcTimeout();
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/48f158f0/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHCM.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHCM.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHCM.java
index 306e139..832ee93 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHCM.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHCM.java
@@ -126,9 +126,19 @@ public class TestHCM {
     }
   }
 
+  public static class SleepCoprocessor extends BaseRegionObserver {
+    public static final int SLEEP_TIME = 5000;
+
+    @Override public void preGetOp(final ObserverContext<RegionCoprocessorEnvironment> e,
+        final Get get, final List<Cell> results) throws IOException {
+      Threads.sleep(SLEEP_TIME);
+    }
+  }
+
   @BeforeClass
   public static void setUpBeforeClass() throws Exception {
     TEST_UTIL.getConfiguration().setBoolean(HConstants.STATUS_PUBLISHED, true);
+    TEST_UTIL.getConfiguration().setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 5);
     TEST_UTIL.startMiniCluster(2);
   }
 
@@ -299,7 +309,7 @@ public class TestHCM {
     HTableDescriptor hdt = TEST_UTIL.createTableDescriptor("HCM-testOperationTimeout");
     hdt.addCoprocessor(SleepAndFailFirstTime.class.getName());
     HTable table = TEST_UTIL.createTable(hdt, new byte[][]{FAM_NAM}, TEST_UTIL.getConfiguration());
-
+    table.setRpcTimeout(Integer.MAX_VALUE);
     // Check that it works if the timeout is big enough
     table.setOperationTimeout(120 * 1000);
     table.get(new Get(FAM_NAM));
@@ -322,6 +332,20 @@ public class TestHCM {
     }
   }
 
+  @Test(expected = RetriesExhaustedException.class)
+  public void testRpcTimeout() throws Exception {
+    HTableDescriptor hdt = TEST_UTIL.createTableDescriptor("HCM-testRpcTimeout");
+    hdt.addCoprocessor(SleepCoprocessor.class.getName());
+    Configuration c = new Configuration(TEST_UTIL.getConfiguration());
+
+    try (Table t = TEST_UTIL.createTable(hdt, new byte[][] { FAM_NAM }, c)) {
+      assert t instanceof HTable;
+      HTable table = (HTable) t;
+      table.setRpcTimeout(SleepCoprocessor.SLEEP_TIME / 2);
+      table.setOperationTimeout(SleepCoprocessor.SLEEP_TIME * 100);
+      table.get(new Get(FAM_NAM));
+    }
+  }
 
   private void testConnectionClose(boolean allowsInterrupt) throws Exception {
     TableName tableName = TableName.valueOf("HCM-testConnectionClose" + allowsInterrupt);